From 9657b75f4f78333cf1ccda2433280f9075184696 Mon Sep 17 00:00:00 2001 From: NoeB Date: Thu, 14 Nov 2024 00:57:42 +0100 Subject: [PATCH 01/83] feat: support array_append (#1072) * feat: support array_append * formatted code * rewrite array_append plan to match spark behaviour and fixed bug in QueryPlan serde * remove unwrap * Fix for Spark 3.3 * refactor array_append binary expression serde code * Disabled array_append test for spark 4.0+ --- .../core/src/execution/datafusion/planner.rs | 31 ++++++++++++++++++- native/proto/src/proto/expr.proto | 1 + .../apache/comet/serde/QueryPlanSerde.scala | 7 ++++- .../apache/comet/CometExpressionSuite.scala | 24 ++++++++++++++ 4 files changed, 61 insertions(+), 2 deletions(-) diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index 6f41bf0ad2..d80e22f1bf 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -82,6 +82,7 @@ use datafusion::{ }, prelude::SessionContext, }; +use datafusion_functions_nested::concat::ArrayAppend; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use datafusion_comet_proto::{ @@ -107,7 +108,8 @@ use datafusion_common::{ }; use datafusion_expr::expr::find_df_window_func; use datafusion_expr::{ - AggregateUDF, WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, + AggregateUDF, ScalarUDF, WindowFrame, WindowFrameBound, WindowFrameUnits, + WindowFunctionDefinition, }; use datafusion_physical_expr::expressions::{Literal, StatsType}; use datafusion_physical_expr::window::WindowExpr; @@ -691,6 +693,33 @@ impl PhysicalPlanner { expr.ordinal as usize, ))) } + ExprStruct::ArrayAppend(expr) => { + let left = + self.create_expr(expr.left.as_ref().unwrap(), Arc::clone(&input_schema))?; + let right = + self.create_expr(expr.right.as_ref().unwrap(), Arc::clone(&input_schema))?; + let return_type = left.data_type(&input_schema)?; + let args = vec![Arc::clone(&left), right]; + let datafusion_array_append = + Arc::new(ScalarUDF::new_from_impl(ArrayAppend::new())); + let array_append_expr: Arc = Arc::new(ScalarFunctionExpr::new( + "array_append", + datafusion_array_append, + args, + return_type, + )); + + let is_null_expr: Arc = Arc::new(IsNullExpr::new(left)); + let null_literal_expr: Arc = + Arc::new(Literal::new(ScalarValue::Null)); + + let case_expr = CaseExpr::try_new( + None, + vec![(is_null_expr, null_literal_expr)], + Some(array_append_expr), + )?; + Ok(Arc::new(case_expr)) + } expr => Err(ExecutionError::GeneralError(format!( "Not implemented: {:?}", expr diff --git a/native/proto/src/proto/expr.proto b/native/proto/src/proto/expr.proto index 3a8193f4af..220f5e521f 100644 --- a/native/proto/src/proto/expr.proto +++ b/native/proto/src/proto/expr.proto @@ -82,6 +82,7 @@ message Expr { ToJson to_json = 55; ListExtract list_extract = 56; GetArrayStructFields get_array_struct_fields = 57; + BinaryExpr array_append = 58; } } 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 2a86c5c363..4a130ad0db 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -2237,7 +2237,12 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim withInfo(expr, "unsupported arguments for GetArrayStructFields", child) None } - + case _ if expr.prettyName == "array_append" => + createBinaryExpr( + expr.children(0), + expr.children(1), + inputs, + (builder, binaryExpr) => builder.setArrayAppend(binaryExpr)) case _ => withInfo(expr, s"${expr.prettyName} is not supported", expr.children: _*) None diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index 0d00867d1e..5079f1910a 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -2313,4 +2313,28 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } } + + test("array_append") { + // array append has been added in Spark 3.4 and in Spark 4.0 it gets written to ArrayInsert + assume(isSpark34Plus && !isSpark40Plus) + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "test.parquet") + makeParquetFileAllTypes(path, dictionaryEnabled = dictionaryEnabled, 10000) + spark.read.parquet(path.toString).createOrReplaceTempView("t1"); + checkSparkAnswerAndOperator(spark.sql("Select array_append(array(_1),false) from t1")) + checkSparkAnswerAndOperator( + spark.sql("SELECT array_append(array(_2, _3, _4), 4) FROM t1")) + checkSparkAnswerAndOperator( + spark.sql("SELECT array_append(array(_2, _3, _4), null) FROM t1")); + checkSparkAnswerAndOperator( + spark.sql("SELECT array_append(array(_6, _7), CAST(6.5 AS DOUBLE)) FROM t1")); + checkSparkAnswerAndOperator(spark.sql("SELECT array_append(array(_8), 'test') FROM t1")); + checkSparkAnswerAndOperator(spark.sql("SELECT array_append(array(_19), _19) FROM t1")); + checkSparkAnswerAndOperator( + spark.sql("SELECT array_append((CASE WHEN _2 =_3 THEN array(_4) END), _4) FROM t1")); + } + + } + } } From c32bf0c949e7ab4a99e5a98d07639812267c97ee Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 13 Nov 2024 22:18:56 -0800 Subject: [PATCH 02/83] chore: Simplify CometShuffleMemoryAllocator to use Spark unified memory allocator (#1063) --- .../scala/org/apache/comet/CometConf.scala | 19 +- docs/source/user-guide/configs.md | 1 - .../comet/CometShuffleMemoryAllocator.java | 182 ++++------------ .../CometShuffleMemoryAllocatorTrait.java | 41 ++++ .../CometTestShuffleMemoryAllocator.java | 194 ++++++++++++++++++ .../sort/CometShuffleExternalSorter.java | 3 +- .../shuffle/CometDiskBlockWriter.java | 10 +- .../comet/execution/shuffle/SpillWriter.java | 8 +- .../comet/CometSparkSessionExtensions.scala | 16 +- .../exec/CometColumnarShuffleSuite.scala | 10 + 10 files changed, 336 insertions(+), 148 deletions(-) create mode 100644 spark/src/main/java/org/apache/spark/shuffle/comet/CometShuffleMemoryAllocatorTrait.java create mode 100644 spark/src/main/java/org/apache/spark/shuffle/comet/CometTestShuffleMemoryAllocator.java diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 7450d27a6d..b602d7cf1c 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -322,8 +322,10 @@ object CometConf extends ShimCometConf { val COMET_COLUMNAR_SHUFFLE_MEMORY_SIZE: OptionalConfigEntry[Long] = conf("spark.comet.columnar.shuffle.memorySize") + .internal() .doc( - "The optional maximum size of the memory used for Comet columnar shuffle, in MiB. " + + "Test-only config. This is only used to test Comet shuffle with Spark tests. " + + "The optional maximum size of the memory used for Comet columnar shuffle, in MiB. " + "Note that this config is only used when `spark.comet.exec.shuffle.mode` is " + "`jvm`. Once allocated memory size reaches this config, the current batch will be " + "flushed to disk immediately. If this is not configured, Comet will use " + @@ -335,8 +337,10 @@ object CometConf extends ShimCometConf { val COMET_COLUMNAR_SHUFFLE_MEMORY_FACTOR: ConfigEntry[Double] = conf("spark.comet.columnar.shuffle.memory.factor") + .internal() .doc( - "Fraction of Comet memory to be allocated per executor process for Comet shuffle. " + + "Test-only config. This is only used to test Comet shuffle with Spark tests. " + + "Fraction of Comet memory to be allocated per executor process for Comet shuffle. " + "Comet memory size is specified by `spark.comet.memoryOverhead` or " + "calculated by `spark.comet.memory.overhead.factor` * `spark.executor.memory`.") .doubleConf @@ -345,6 +349,17 @@ object CometConf extends ShimCometConf { "Ensure that Comet shuffle memory overhead factor is a double greater than 0") .createWithDefault(1.0) + val COMET_COLUMNAR_SHUFFLE_UNIFIED_MEMORY_ALLOCATOR_IN_TEST: ConfigEntry[Boolean] = + conf("spark.comet.columnar.shuffle.unifiedMemoryAllocatorTest") + .doc("Whether to use Spark unified memory allocator for Comet columnar shuffle in tests." + + "If not configured, Comet will use a test-only memory allocator for Comet columnar " + + "shuffle when Spark test env detected. The test-ony allocator is proposed to run with " + + "Spark tests as these tests require on-heap memory configuration. " + + "By default, this config is false.") + .internal() + .booleanConf + .createWithDefault(false) + val COMET_COLUMNAR_SHUFFLE_BATCH_SIZE: ConfigEntry[Int] = conf("spark.comet.columnar.shuffle.batch.size") .internal() diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index dc97e3c598..69da792223 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -29,7 +29,6 @@ Comet provides the following configuration settings. | spark.comet.columnar.shuffle.async.enabled | Whether to enable asynchronous shuffle for Arrow-based shuffle. | false | | spark.comet.columnar.shuffle.async.max.thread.num | Maximum number of threads on an executor used for Comet async columnar shuffle. This is the upper bound of total number of shuffle threads per executor. In other words, if the number of cores * the number of shuffle threads per task `spark.comet.columnar.shuffle.async.thread.num` is larger than this config. Comet will use this config as the number of shuffle threads per executor instead. | 100 | | spark.comet.columnar.shuffle.async.thread.num | Number of threads used for Comet async columnar shuffle per shuffle task. Note that more threads means more memory requirement to buffer shuffle data before flushing to disk. Also, more threads may not always improve performance, and should be set based on the number of cores available. | 3 | -| spark.comet.columnar.shuffle.memory.factor | Fraction of Comet memory to be allocated per executor process for Comet shuffle. Comet memory size is specified by `spark.comet.memoryOverhead` or calculated by `spark.comet.memory.overhead.factor` * `spark.executor.memory`. | 1.0 | | spark.comet.convert.csv.enabled | When enabled, data from Spark (non-native) CSV v1 and v2 scans will be converted to Arrow format. Note that to enable native vectorized execution, both this config and 'spark.comet.exec.enabled' need to be enabled. | false | | spark.comet.convert.json.enabled | When enabled, data from Spark (non-native) JSON v1 and v2 scans will be converted to Arrow format. Note that to enable native vectorized execution, both this config and 'spark.comet.exec.enabled' need to be enabled. | false | | spark.comet.convert.parquet.enabled | When enabled, data from Spark (non-native) Parquet v1 and v2 scans will be converted to Arrow format. Note that to enable native vectorized execution, both this config and 'spark.comet.exec.enabled' need to be enabled. | false | diff --git a/spark/src/main/java/org/apache/spark/shuffle/comet/CometShuffleMemoryAllocator.java b/spark/src/main/java/org/apache/spark/shuffle/comet/CometShuffleMemoryAllocator.java index 2837fa369a..54e349c137 100644 --- a/spark/src/main/java/org/apache/spark/shuffle/comet/CometShuffleMemoryAllocator.java +++ b/spark/src/main/java/org/apache/spark/shuffle/comet/CometShuffleMemoryAllocator.java @@ -20,157 +20,75 @@ package org.apache.spark.shuffle.comet; import java.io.IOException; -import java.util.BitSet; import org.apache.spark.SparkConf; import org.apache.spark.memory.MemoryConsumer; import org.apache.spark.memory.MemoryMode; -import org.apache.spark.memory.SparkOutOfMemoryError; import org.apache.spark.memory.TaskMemoryManager; -import org.apache.spark.sql.internal.SQLConf; -import org.apache.spark.unsafe.array.LongArray; import org.apache.spark.unsafe.memory.MemoryBlock; -import org.apache.spark.unsafe.memory.UnsafeMemoryAllocator; +import org.apache.spark.util.Utils; -import org.apache.comet.CometSparkSessionExtensions$; +import org.apache.comet.CometConf$; /** * A simple memory allocator used by `CometShuffleExternalSorter` to allocate memory blocks which - * store serialized rows. We don't rely on Spark memory allocator because we need to allocate - * off-heap memory no matter memory mode is on-heap or off-heap. This allocator is configured with - * fixed size of memory, and it will throw `SparkOutOfMemoryError` if the memory is not enough. - * - *

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

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

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


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

{ /// A utility page iterator which stores page readers in memory, used for tests. #[derive(Clone)] pub struct InMemoryPageIterator>> { - schema: SchemaDescPtr, - column_desc: ColumnDescPtr, page_reader_iter: I, } impl>> InMemoryPageIterator { - pub fn new( - schema: SchemaDescPtr, - column_desc: ColumnDescPtr, - pages: impl IntoIterator, IntoIter = I>, - ) -> Self { + pub fn new(pages: impl IntoIterator, IntoIter = I>) -> Self { Self { - schema, - column_desc, page_reader_iter: pages.into_iter(), } } From 2c1a6b9c576042f2e0a73cf76a3ad93db46b6a0b Mon Sep 17 00:00:00 2001 From: KAZUYUKI TANIMURA Date: Wed, 11 Dec 2024 09:44:12 -0800 Subject: [PATCH 32/83] fix: Spark 4.0-preview1 SPARK-47120 (#1156) ## Which issue does this PR close? Part of https://github.com/apache/datafusion-comet/issues/372 and https://github.com/apache/datafusion-comet/issues/551 ## Rationale for this change To be ready for Spark 4.0 ## What changes are included in this PR? This PR fixes the new test SPARK-47120 added in Spark 4.0 ## How are these changes tested? tests enabled --- dev/diffs/4.0.0-preview1.diff | 16 +++------------- docs/source/user-guide/source.md | 2 +- .../apache/comet/parquet/ParquetFilters.scala | 10 ++++++---- 3 files changed, 10 insertions(+), 18 deletions(-) diff --git a/dev/diffs/4.0.0-preview1.diff b/dev/diffs/4.0.0-preview1.diff index 1a6810a7ab..ba68d2a7bd 100644 --- a/dev/diffs/4.0.0-preview1.diff +++ b/dev/diffs/4.0.0-preview1.diff @@ -2088,7 +2088,7 @@ index cd6f41b4ef4..4b6a17344bc 100644 ParquetOutputFormat.WRITER_VERSION -> ParquetProperties.WriterVersion.PARQUET_2_0.toString ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala -index 795e9f46a8d..6285a1e388b 100644 +index 795e9f46a8d..5306c94a686 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -1100,7 +1100,11 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared @@ -2150,17 +2150,7 @@ index 795e9f46a8d..6285a1e388b 100644 // block 1: // null count min max // page-0 0 0 99 -@@ -2211,7 +2225,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared - } - } - -- test("SPARK-47120: subquery literal filter pushdown") { -+ test("SPARK-47120: subquery literal filter pushdown", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { - withTable("t1", "t2") { - sql("create table t1(d date) using parquet") - sql("create table t2(d date) using parquet") -@@ -2301,7 +2316,11 @@ class ParquetV1FilterSuite extends ParquetFilterSuite { +@@ -2301,7 +2315,11 @@ class ParquetV1FilterSuite extends ParquetFilterSuite { assert(pushedParquetFilters.exists(_.getClass === filterClass), s"${pushedParquetFilters.map(_.getClass).toList} did not contain ${filterClass}.") @@ -2173,7 +2163,7 @@ index 795e9f46a8d..6285a1e388b 100644 } else { assert(selectedFilters.isEmpty, "There is filter pushed down") } -@@ -2362,7 +2381,11 @@ class ParquetV2FilterSuite extends ParquetFilterSuite { +@@ -2362,7 +2380,11 @@ class ParquetV2FilterSuite extends ParquetFilterSuite { assert(pushedParquetFilters.exists(_.getClass === filterClass), s"${pushedParquetFilters.map(_.getClass).toList} did not contain ${filterClass}.") diff --git a/docs/source/user-guide/source.md b/docs/source/user-guide/source.md index 71c9060cb5..5f5d03d906 100644 --- a/docs/source/user-guide/source.md +++ b/docs/source/user-guide/source.md @@ -27,7 +27,7 @@ Official source releases can be downloaded from https://dist.apache.org/repos/di ```console # Pick the latest version -export COMET_VERSION=0.3.0 +export COMET_VERSION=0.4.0 # Download the tarball curl -O "https://dist.apache.org/repos/dist/release/datafusion/datafusion-comet-$COMET_VERSION/apache-datafusion-comet-$COMET_VERSION.tar.gz" # Unpack diff --git a/spark/src/main/scala/org/apache/comet/parquet/ParquetFilters.scala b/spark/src/main/scala/org/apache/comet/parquet/ParquetFilters.scala index 17844aba82..bcb23986f3 100644 --- a/spark/src/main/scala/org/apache/comet/parquet/ParquetFilters.scala +++ b/spark/src/main/scala/org/apache/comet/parquet/ParquetFilters.scala @@ -723,20 +723,22 @@ class ParquetFilters( .lift(nameToParquetField(name).fieldType) .map(_(nameToParquetField(name).fieldNames, value)) - case sources.LessThan(name, value) if canMakeFilterOn(name, value) => + case sources.LessThan(name, value) if (value != null) && canMakeFilterOn(name, value) => makeLt .lift(nameToParquetField(name).fieldType) .map(_(nameToParquetField(name).fieldNames, value)) - case sources.LessThanOrEqual(name, value) if canMakeFilterOn(name, value) => + case sources.LessThanOrEqual(name, value) + if (value != null) && canMakeFilterOn(name, value) => makeLtEq .lift(nameToParquetField(name).fieldType) .map(_(nameToParquetField(name).fieldNames, value)) - case sources.GreaterThan(name, value) if canMakeFilterOn(name, value) => + case sources.GreaterThan(name, value) if (value != null) && canMakeFilterOn(name, value) => makeGt .lift(nameToParquetField(name).fieldType) .map(_(nameToParquetField(name).fieldNames, value)) - case sources.GreaterThanOrEqual(name, value) if canMakeFilterOn(name, value) => + case sources.GreaterThanOrEqual(name, value) + if (value != null) && canMakeFilterOn(name, value) => makeGtEq .lift(nameToParquetField(name).fieldType) .map(_(nameToParquetField(name).fieldNames, value)) From 49cf0d7f32813b6cb5bd3999378c960bd1171fca Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 12 Dec 2024 07:45:06 -0700 Subject: [PATCH 33/83] chore: Move string kernels and expressions to spark-expr crate (#1164) * Move string kernels and expressions to spark-expr crate * remove unused hash kernel * remove unused dependencies --- native/Cargo.lock | 2 - native/core/Cargo.toml | 6 - native/core/benches/hash.rs | 137 ------------- .../execution/datafusion/expressions/mod.rs | 1 - .../core/src/execution/datafusion/planner.rs | 15 +- native/core/src/execution/kernels/hash.rs | 187 ------------------ native/core/src/execution/kernels/mod.rs | 23 --- native/core/src/execution/mod.rs | 3 - native/spark-expr/src/kernels/mod.rs | 1 + .../src}/kernels/strings.rs | 7 +- native/spark-expr/src/lib.rs | 2 + .../expressions => spark-expr/src}/strings.rs | 2 +- 12 files changed, 13 insertions(+), 373 deletions(-) delete mode 100644 native/core/benches/hash.rs delete mode 100644 native/core/src/execution/kernels/hash.rs delete mode 100644 native/core/src/execution/kernels/mod.rs rename native/{core/src/execution => spark-expr/src}/kernels/strings.rs (96%) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/strings.rs (99%) diff --git a/native/Cargo.lock b/native/Cargo.lock index 67d041a399..9a8eab83e6 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -871,7 +871,6 @@ dependencies = [ name = "datafusion-comet" version = "0.5.0" dependencies = [ - "ahash", "arrow", "arrow-array", "arrow-buffer", @@ -893,7 +892,6 @@ dependencies = [ "datafusion-physical-expr", "flate2", "futures", - "half", "hex", "itertools 0.11.0", "jni", diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 4b9753ec51..489da46d47 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -41,7 +41,6 @@ arrow-buffer = { workspace = true } arrow-data = { workspace = true } arrow-schema = { workspace = true } parquet = { workspace = true, default-features = false, features = ["experimental"] } -half = { version = "2.4.1", default-features = false } futures = { workspace = true } mimalloc = { version = "*", default-features = false, optional = true } tokio = { version = "1", features = ["rt-multi-thread"] } @@ -62,7 +61,6 @@ rand = { workspace = true} num = { workspace = true } bytes = "1.5.0" tempfile = "3.8.0" -ahash = { version = "0.8", default-features = false } itertools = "0.11.0" paste = "1.0.14" datafusion-common = { workspace = true } @@ -102,10 +100,6 @@ harness = false name = "bit_util" harness = false -[[bench]] -name = "hash" -harness = false - [[bench]] name = "row_columnar" harness = false diff --git a/native/core/benches/hash.rs b/native/core/benches/hash.rs deleted file mode 100644 index 039a6d5d8e..0000000000 --- a/native/core/benches/hash.rs +++ /dev/null @@ -1,137 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#[path = "common.rs"] -mod common; - -use arrow_array::ArrayRef; -use comet::execution::kernels::hash; -use common::*; -use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; -use datafusion_comet_spark_expr::scalar_funcs::{spark_murmur3_hash, spark_xxhash64}; -use datafusion_common::ScalarValue; -use datafusion_expr::ColumnarValue; -use std::sync::Arc; - -const BATCH_SIZE: usize = 1024 * 8; -const NUM_ITER: usize = 10; -const NULL_FRACTION: f32 = 0.1; - -fn criterion_benchmark(c: &mut Criterion) { - let mut group = c.benchmark_group("hash"); - - let a1: ArrayRef = Arc::new(create_int64_array(BATCH_SIZE, 0.0, 0, BATCH_SIZE as i64)); - let a2: ArrayRef = Arc::new(create_int64_array(BATCH_SIZE, 0.0, 0, BATCH_SIZE as i64)); - let a3: ArrayRef = Arc::new(create_int64_array( - BATCH_SIZE, - NULL_FRACTION, - 0, - BATCH_SIZE as i64, - )); - let a4: ArrayRef = Arc::new(create_int64_array( - BATCH_SIZE, - NULL_FRACTION, - 0, - BATCH_SIZE as i64, - )); - - group.bench_function( - BenchmarkId::new("hash_i64_single_nonnull", BATCH_SIZE), - |b| { - let input = vec![a1.clone()]; - let mut dst = vec![0; BATCH_SIZE]; - - b.iter(|| { - for _ in 0..NUM_ITER { - hash(&input, &mut dst); - } - }); - }, - ); - group.bench_function(BenchmarkId::new("hash_i64_single_null", BATCH_SIZE), |b| { - let input = vec![a3.clone()]; - let mut dst = vec![0; BATCH_SIZE]; - - b.iter(|| { - for _ in 0..NUM_ITER { - hash(&input, &mut dst); - } - }); - }); - group.bench_function( - BenchmarkId::new("hash_i64_multiple_nonnull", BATCH_SIZE), - |b| { - let input = vec![a1.clone(), a2.clone()]; - let mut dst = vec![0; BATCH_SIZE]; - - b.iter(|| { - for _ in 0..NUM_ITER { - hash(&input, &mut dst); - } - }); - }, - ); - group.bench_function( - BenchmarkId::new("hash_i64_multiple_null", BATCH_SIZE), - |b| { - let input = vec![a3.clone(), a4.clone()]; - let mut dst = vec![0; BATCH_SIZE]; - - b.iter(|| { - for _ in 0..NUM_ITER { - hash(&input, &mut dst); - } - }); - }, - ); - group.bench_function(BenchmarkId::new("xxhash64", BATCH_SIZE), |b| { - let inputs = &[ - ColumnarValue::Array(a3.clone()), - ColumnarValue::Array(a4.clone()), - ColumnarValue::Scalar(ScalarValue::Int64(Some(42i64))), - ]; - - b.iter(|| { - for _ in 0..NUM_ITER { - spark_xxhash64(inputs).unwrap(); - } - }); - }); - group.bench_function(BenchmarkId::new("murmur3", BATCH_SIZE), |b| { - let inputs = &[ - ColumnarValue::Array(a3.clone()), - ColumnarValue::Array(a4.clone()), - ColumnarValue::Scalar(ScalarValue::Int32(Some(42))), - ]; - b.iter(|| { - for _ in 0..NUM_ITER { - spark_murmur3_hash(inputs).unwrap(); - } - }); - }); -} - -fn config() -> Criterion { - Criterion::default() -} - -criterion_group! { - name = benches; - config = config(); - targets = criterion_benchmark -} -criterion_main!(benches); diff --git a/native/core/src/execution/datafusion/expressions/mod.rs b/native/core/src/execution/datafusion/expressions/mod.rs index 2bb14df36e..5f9f322b2e 100644 --- a/native/core/src/execution/datafusion/expressions/mod.rs +++ b/native/core/src/execution/datafusion/expressions/mod.rs @@ -23,7 +23,6 @@ use crate::errors::CometError; pub mod bloom_filter_agg; pub mod bloom_filter_might_contain; pub mod negative; -pub mod strings; pub mod subquery; pub mod unbound; diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index 5e77b3f653..0e64ed6afa 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -25,12 +25,8 @@ use crate::{ datafusion::{ expressions::{ bloom_filter_agg::BloomFilterAgg, - bloom_filter_might_contain::BloomFilterMightContain, - checkoverflow::CheckOverflow, - negative, - strings::{Contains, EndsWith, Like, StartsWith, StringSpaceExpr, SubstringExpr}, - subquery::Subquery, - unbound::UnboundColumn, + bloom_filter_might_contain::BloomFilterMightContain, checkoverflow::CheckOverflow, + negative, subquery::Subquery, unbound::UnboundColumn, }, operators::expand::CometExpandExec, shuffle_writer::ShuffleWriterExec, @@ -90,9 +86,10 @@ use datafusion_comet_proto::{ spark_partitioning::{partitioning::PartitioningStruct, Partitioning as SparkPartitioning}, }; use datafusion_comet_spark_expr::{ - ArrayInsert, Avg, AvgDecimal, BitwiseNotExpr, Cast, Correlation, Covariance, CreateNamedStruct, - DateTruncExpr, GetArrayStructFields, GetStructField, HourExpr, IfExpr, ListExtract, MinuteExpr, - NormalizeNaNAndZero, RLike, SecondExpr, SparkCastOptions, Stddev, SumDecimal, + ArrayInsert, Avg, AvgDecimal, BitwiseNotExpr, Cast, Contains, Correlation, Covariance, + CreateNamedStruct, DateTruncExpr, EndsWith, GetArrayStructFields, GetStructField, HourExpr, + IfExpr, Like, ListExtract, MinuteExpr, NormalizeNaNAndZero, RLike, SecondExpr, + SparkCastOptions, StartsWith, Stddev, StringSpaceExpr, SubstringExpr, SumDecimal, TimestampTruncExpr, ToJson, Variance, }; use datafusion_common::scalar::ScalarStructBuilder; diff --git a/native/core/src/execution/kernels/hash.rs b/native/core/src/execution/kernels/hash.rs deleted file mode 100644 index b39fd62243..0000000000 --- a/native/core/src/execution/kernels/hash.rs +++ /dev/null @@ -1,187 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use crate::common::bit; -use ahash::RandomState; -use arrow::datatypes::{i256, ArrowNativeType}; -use arrow_array::{ - downcast_dictionary_array, downcast_primitive_array, Array, ArrayAccessor, ArrayRef, - ArrowPrimitiveType, PrimitiveArray, -}; -use arrow_buffer::{IntervalDayTime, IntervalMonthDayNano}; -use std::fmt::Debug; - -pub fn hash(src: &[ArrayRef], dst: &mut [u64]) { - let state = RandomState::with_seed(42); - src.iter().enumerate().for_each(|(idx, v)| { - downcast_dictionary_array!( - v => { - let keys = v.keys(); - let values = v.values(); - downcast_primitive_array!( - values => hash_dict_typed(&state, idx > 0, keys, values, dst), - dt => panic!("Expected only primitive type but found {}", dt) - ) - }, - dt => { - downcast_primitive_array!( - v => hash_typed(&state, idx > 0, v, dst), - _ => panic!("Expected only primitive type but found {}", dt) - ) - } - ) - }); -} - -fn hash_typed(state: &RandomState, mix: bool, array: T, dst: &mut [u64]) -where - T: ArrayAccessor, - T::Item: Hashable + Debug, -{ - let nullable = array.null_count() > 0; - let num_values = array.len(); - if nullable { - for i in 0..num_values { - if !array.is_null(i) { - unsafe { - let value = array.value_unchecked(i); - hash1(state, mix, i, value, dst); - } - } - } - } else { - for i in 0..num_values { - unsafe { - let value = array.value_unchecked(i); - hash1(state, mix, i, value, dst); - } - } - } -} - -fn hash_dict_typed( - state: &RandomState, - mix: bool, - keys: &PrimitiveArray, - values: V, - dst: &mut [u64], -) where - K: ArrowPrimitiveType, - V: ArrayAccessor, - V::Item: Hashable + Debug, -{ - let nullable = keys.null_count() > 0; - let num_keys = keys.len(); - let mut value_hashes = vec![0; values.len()]; - - for (i, value_hash) in value_hashes.iter_mut().enumerate() { - unsafe { - *value_hash = values.value_unchecked(i).create_hash(state); - } - } - - if nullable { - for i in 0..num_keys { - if !keys.is_null(i) { - unsafe { - let idx = keys.value_unchecked(i); - let hash = value_hashes[idx.as_usize()]; - hash1_helper(mix, i, hash, dst); - } - } - } - } else { - for i in 0..num_keys { - unsafe { - let idx = keys.value_unchecked(i); - let hash = value_hashes[idx.as_usize()]; - hash1_helper(mix, i, hash, dst); - } - } - } -} - -#[inline(always)] -fn hash1(state: &RandomState, mix: bool, i: usize, value: T, dst: &mut [u64]) { - let hash = value.create_hash(state); - hash1_helper(mix, i, hash, dst); -} - -#[inline(always)] -fn hash1_helper(mix: bool, i: usize, hash: u64, dst: &mut [u64]) { - dst[i] = if mix { - bit::mix_hash(dst[i], hash) - } else { - hash - } -} - -pub(crate) trait Hashable { - fn create_hash(&self, state: &RandomState) -> u64; -} - -macro_rules! impl_hashable { - ($($t:ty),+) => { - $(impl Hashable for $t { - #[inline] - fn create_hash(&self, state: &RandomState) -> u64 { - state.hash_one(self) - } - })+ - }; -} - -impl_hashable!(i8, i16, i32, u8, u16, u32, u64, i128, i256); - -impl Hashable for i64 { - fn create_hash(&self, state: &RandomState) -> u64 { - state.hash_one(self) - } -} - -impl Hashable for half::f16 { - fn create_hash(&self, _: &RandomState) -> u64 { - unimplemented!("hashing on f16 is not supported") - } -} - -impl Hashable for f32 { - fn create_hash(&self, state: &RandomState) -> u64 { - state.hash_one(u32::from_ne_bytes(self.to_ne_bytes())) - } -} - -impl Hashable for f64 { - fn create_hash(&self, state: &RandomState) -> u64 { - state.hash_one(u64::from_ne_bytes(self.to_ne_bytes())) - } -} - -impl Hashable for IntervalDayTime { - fn create_hash(&self, state: &RandomState) -> u64 { - state.hash_one(self.days); - state.hash_one(self.milliseconds) - } -} - -impl Hashable for IntervalMonthDayNano { - fn create_hash(&self, state: &RandomState) -> u64 { - state.hash_one(self.months); - state.hash_one(self.days); - state.hash_one(self.nanoseconds) - } -} diff --git a/native/core/src/execution/kernels/mod.rs b/native/core/src/execution/kernels/mod.rs deleted file mode 100644 index 675dcd4893..0000000000 --- a/native/core/src/execution/kernels/mod.rs +++ /dev/null @@ -1,23 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Kernels - -mod hash; -pub use hash::hash; - -pub(crate) mod strings; diff --git a/native/core/src/execution/mod.rs b/native/core/src/execution/mod.rs index f17935702c..3dba747f2d 100644 --- a/native/core/src/execution/mod.rs +++ b/native/core/src/execution/mod.rs @@ -18,9 +18,6 @@ //! PoC of vectorization execution through JNI to Rust. pub mod datafusion; pub mod jni_api; - -pub mod kernels; // for benchmarking - mod metrics; pub mod operators; pub mod serde; diff --git a/native/spark-expr/src/kernels/mod.rs b/native/spark-expr/src/kernels/mod.rs index 88aa34b1a3..3669ff13ad 100644 --- a/native/spark-expr/src/kernels/mod.rs +++ b/native/spark-expr/src/kernels/mod.rs @@ -17,4 +17,5 @@ //! Kernels +pub mod strings; pub(crate) mod temporal; diff --git a/native/core/src/execution/kernels/strings.rs b/native/spark-expr/src/kernels/strings.rs similarity index 96% rename from native/core/src/execution/kernels/strings.rs rename to native/spark-expr/src/kernels/strings.rs index d63b2c4778..bb275fbb9f 100644 --- a/native/core/src/execution/kernels/strings.rs +++ b/native/spark-expr/src/kernels/strings.rs @@ -25,15 +25,14 @@ use arrow::{ compute::kernels::substring::{substring as arrow_substring, substring_by_char}, datatypes::{DataType, Int32Type}, }; - -use crate::errors::ExpressionError; +use datafusion_common::DataFusionError; /// Returns an ArrayRef with a string consisting of `length` spaces. /// /// # Preconditions /// /// - elements in `length` must not be negative -pub fn string_space(length: &dyn Array) -> Result { +pub fn string_space(length: &dyn Array) -> Result { match length.data_type() { DataType::Int32 => { let array = length.as_any().downcast_ref::().unwrap(); @@ -52,7 +51,7 @@ pub fn string_space(length: &dyn Array) -> Result { } } -pub fn substring(array: &dyn Array, start: i64, length: u64) -> Result { +pub fn substring(array: &dyn Array, start: i64, length: u64) -> Result { match array.data_type() { DataType::LargeUtf8 => substring_by_char( array diff --git a/native/spark-expr/src/lib.rs b/native/spark-expr/src/lib.rs index 15f446ef35..5dff6e0b8f 100644 --- a/native/spark-expr/src/lib.rs +++ b/native/spark-expr/src/lib.rs @@ -33,6 +33,8 @@ mod correlation; pub use correlation::Correlation; mod covariance; pub use covariance::Covariance; +mod strings; +pub use strings::{Contains, EndsWith, Like, StartsWith, StringSpaceExpr, SubstringExpr}; mod kernels; mod list; mod regexp; diff --git a/native/core/src/execution/datafusion/expressions/strings.rs b/native/spark-expr/src/strings.rs similarity index 99% rename from native/core/src/execution/datafusion/expressions/strings.rs rename to native/spark-expr/src/strings.rs index 200b4ec5a3..a8aab6aee9 100644 --- a/native/core/src/execution/datafusion/expressions/strings.rs +++ b/native/spark-expr/src/strings.rs @@ -17,7 +17,7 @@ #![allow(deprecated)] -use crate::execution::kernels::strings::{string_space, substring}; +use crate::kernels::strings::{string_space, substring}; use arrow::{ compute::{ contains_dyn, contains_utf8_scalar_dyn, ends_with_dyn, ends_with_utf8_scalar_dyn, like_dyn, From 7db9aa67ed0049bb25e783bc4ba991e53df34bbf Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 12 Dec 2024 14:51:19 -0700 Subject: [PATCH 34/83] chore: Move remaining expressions to spark-expr crate + some minor refactoring (#1165) * move CheckOverflow to spark-expr crate * move NegativeExpr to spark-expr crate * move UnboundColumn to spark-expr crate * move ExpandExec from execution::datafusion::operators to execution::operators * refactoring to remove datafusion subpackage * update imports in benches * fix * fix --- native/Cargo.lock | 1 + native/core/benches/bloom_filter_agg.rs | 2 +- native/core/benches/shuffle_writer.rs | 2 +- .../execution/datafusion/expressions/mod.rs | 35 ------------------- .../src/execution/datafusion/operators/mod.rs | 18 ---------- .../expressions/bloom_filter_agg.rs | 4 +-- .../expressions/bloom_filter_might_contain.rs | 4 +-- .../{datafusion => expressions}/mod.rs | 13 ++++--- .../{datafusion => }/expressions/subquery.rs | 0 native/core/src/execution/jni_api.rs | 6 ++-- native/core/src/execution/metrics/utils.rs | 2 +- native/core/src/execution/mod.rs | 5 ++- .../{datafusion => }/operators/expand.rs | 10 +++--- native/core/src/execution/operators/mod.rs | 2 ++ native/core/src/execution/operators/scan.rs | 3 +- .../src/execution/{datafusion => }/planner.rs | 32 ++++++++--------- native/core/src/execution/shuffle/mod.rs | 2 ++ native/core/src/execution/shuffle/row.rs | 2 +- .../{datafusion => shuffle}/shuffle_writer.rs | 0 .../execution/{datafusion => }/spark_plan.rs | 0 .../execution/{datafusion => }/util/mod.rs | 0 .../{datafusion => }/util/spark_bit_array.rs | 0 .../util/spark_bloom_filter.rs | 4 +-- native/spark-expr/Cargo.toml | 1 + .../src}/checkoverflow.rs | 0 native/spark-expr/src/lib.rs | 12 +++++++ .../src}/negative.rs | 7 ++-- .../expressions => spark-expr/src}/unbound.rs | 0 28 files changed, 63 insertions(+), 104 deletions(-) delete mode 100644 native/core/src/execution/datafusion/expressions/mod.rs delete mode 100644 native/core/src/execution/datafusion/operators/mod.rs rename native/core/src/execution/{datafusion => }/expressions/bloom_filter_agg.rs (97%) rename native/core/src/execution/{datafusion => }/expressions/bloom_filter_might_contain.rs (97%) rename native/core/src/execution/{datafusion => expressions}/mod.rs (83%) rename native/core/src/execution/{datafusion => }/expressions/subquery.rs (100%) rename native/core/src/execution/{datafusion => }/operators/expand.rs (97%) rename native/core/src/execution/{datafusion => }/planner.rs (98%) rename native/core/src/execution/{datafusion => shuffle}/shuffle_writer.rs (100%) rename native/core/src/execution/{datafusion => }/spark_plan.rs (100%) rename native/core/src/execution/{datafusion => }/util/mod.rs (100%) rename native/core/src/execution/{datafusion => }/util/spark_bit_array.rs (100%) rename native/core/src/execution/{datafusion => }/util/spark_bloom_filter.rs (98%) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/checkoverflow.rs (100%) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/negative.rs (98%) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/unbound.rs (100%) diff --git a/native/Cargo.lock b/native/Cargo.lock index 9a8eab83e6..7966bb80bb 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -931,6 +931,7 @@ version = "0.5.0" dependencies = [ "arrow", "arrow-array", + "arrow-buffer", "arrow-data", "arrow-schema", "chrono", diff --git a/native/core/benches/bloom_filter_agg.rs b/native/core/benches/bloom_filter_agg.rs index 25d27d174e..b83ff3fad1 100644 --- a/native/core/benches/bloom_filter_agg.rs +++ b/native/core/benches/bloom_filter_agg.rs @@ -19,7 +19,7 @@ use arrow::datatypes::{DataType, Field, Schema}; use arrow_array::builder::Int64Builder; use arrow_array::{ArrayRef, RecordBatch}; use arrow_schema::SchemaRef; -use comet::execution::datafusion::expressions::bloom_filter_agg::BloomFilterAgg; +use comet::execution::expressions::bloom_filter_agg::BloomFilterAgg; use criterion::{black_box, criterion_group, criterion_main, Criterion}; use datafusion::physical_expr::PhysicalExpr; use datafusion::physical_plan::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; diff --git a/native/core/benches/shuffle_writer.rs b/native/core/benches/shuffle_writer.rs index 6f28718618..272887238e 100644 --- a/native/core/benches/shuffle_writer.rs +++ b/native/core/benches/shuffle_writer.rs @@ -17,7 +17,7 @@ use arrow_array::{builder::StringBuilder, RecordBatch}; use arrow_schema::{DataType, Field, Schema}; -use comet::execution::datafusion::shuffle_writer::ShuffleWriterExec; +use comet::execution::shuffle::ShuffleWriterExec; use criterion::{criterion_group, criterion_main, Criterion}; use datafusion::{ physical_plan::{common::collect, memory::MemoryExec, ExecutionPlan}, diff --git a/native/core/src/execution/datafusion/expressions/mod.rs b/native/core/src/execution/datafusion/expressions/mod.rs deleted file mode 100644 index 5f9f322b2e..0000000000 --- a/native/core/src/execution/datafusion/expressions/mod.rs +++ /dev/null @@ -1,35 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Native DataFusion expressions - -pub mod checkoverflow; - -use crate::errors::CometError; -pub mod bloom_filter_agg; -pub mod bloom_filter_might_contain; -pub mod negative; -pub mod subquery; -pub mod unbound; - -pub use datafusion_comet_spark_expr::{EvalMode, SparkError}; - -fn arithmetic_overflow_error(from_type: &str) -> CometError { - CometError::Spark(SparkError::ArithmeticOverflow { - from_type: from_type.to_string(), - }) -} diff --git a/native/core/src/execution/datafusion/operators/mod.rs b/native/core/src/execution/datafusion/operators/mod.rs deleted file mode 100644 index 3d28a266a6..0000000000 --- a/native/core/src/execution/datafusion/operators/mod.rs +++ /dev/null @@ -1,18 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -pub mod expand; diff --git a/native/core/src/execution/datafusion/expressions/bloom_filter_agg.rs b/native/core/src/execution/expressions/bloom_filter_agg.rs similarity index 97% rename from native/core/src/execution/datafusion/expressions/bloom_filter_agg.rs rename to native/core/src/execution/expressions/bloom_filter_agg.rs index 1300e08c22..ea8bb3647f 100644 --- a/native/core/src/execution/datafusion/expressions/bloom_filter_agg.rs +++ b/native/core/src/execution/expressions/bloom_filter_agg.rs @@ -19,8 +19,8 @@ use arrow_schema::Field; use datafusion::{arrow::datatypes::DataType, logical_expr::Volatility}; use std::{any::Any, sync::Arc}; -use crate::execution::datafusion::util::spark_bloom_filter; -use crate::execution::datafusion::util::spark_bloom_filter::SparkBloomFilter; +use crate::execution::util::spark_bloom_filter; +use crate::execution::util::spark_bloom_filter::SparkBloomFilter; use arrow::array::ArrayRef; use arrow_array::BinaryArray; use datafusion::error::Result; diff --git a/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs b/native/core/src/execution/expressions/bloom_filter_might_contain.rs similarity index 97% rename from native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs rename to native/core/src/execution/expressions/bloom_filter_might_contain.rs index de922d8312..af6a5a47a3 100644 --- a/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs +++ b/native/core/src/execution/expressions/bloom_filter_might_contain.rs @@ -15,9 +15,7 @@ // specific language governing permissions and limitations // under the License. -use crate::{ - execution::datafusion::util::spark_bloom_filter::SparkBloomFilter, parquet::data_type::AsBytes, -}; +use crate::{execution::util::spark_bloom_filter::SparkBloomFilter, parquet::data_type::AsBytes}; use arrow::record_batch::RecordBatch; use arrow_array::cast::as_primitive_array; use arrow_schema::{DataType, Schema}; diff --git a/native/core/src/execution/datafusion/mod.rs b/native/core/src/execution/expressions/mod.rs similarity index 83% rename from native/core/src/execution/datafusion/mod.rs rename to native/core/src/execution/expressions/mod.rs index ca41fa0aa0..e2f811fa2e 100644 --- a/native/core/src/execution/datafusion/mod.rs +++ b/native/core/src/execution/expressions/mod.rs @@ -15,11 +15,10 @@ // specific language governing permissions and limitations // under the License. -//! Native execution through DataFusion +//! Native DataFusion expressions -pub mod expressions; -mod operators; -pub mod planner; -pub mod shuffle_writer; -pub(crate) mod spark_plan; -mod util; +pub mod bloom_filter_agg; +pub mod bloom_filter_might_contain; +pub mod subquery; + +pub use datafusion_comet_spark_expr::EvalMode; diff --git a/native/core/src/execution/datafusion/expressions/subquery.rs b/native/core/src/execution/expressions/subquery.rs similarity index 100% rename from native/core/src/execution/datafusion/expressions/subquery.rs rename to native/core/src/execution/expressions/subquery.rs diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index 5103f5ce4a..491b389c99 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -45,8 +45,8 @@ use super::{serde, utils::SparkArrowConvert, CometMemoryPool}; use crate::{ errors::{try_unwrap_or_throw, CometError, CometResult}, execution::{ - datafusion::planner::PhysicalPlanner, metrics::utils::update_comet_metric, - serde::to_arrow_datatype, shuffle::row::process_sorted_row_partition, sort::RdxSort, + metrics::utils::update_comet_metric, planner::PhysicalPlanner, serde::to_arrow_datatype, + shuffle::row::process_sorted_row_partition, sort::RdxSort, }, jvm_bridge::{jni_new_global_ref, JVMClasses}, }; @@ -59,8 +59,8 @@ use jni::{ }; use tokio::runtime::Runtime; -use crate::execution::datafusion::spark_plan::SparkPlan; use crate::execution::operators::ScanExec; +use crate::execution::spark_plan::SparkPlan; use log::info; /// Comet native execution context. Kept alive across JNI calls. diff --git a/native/core/src/execution/metrics/utils.rs b/native/core/src/execution/metrics/utils.rs index 4bb1c4474c..0eb4b631dd 100644 --- a/native/core/src/execution/metrics/utils.rs +++ b/native/core/src/execution/metrics/utils.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use crate::execution::datafusion::spark_plan::SparkPlan; +use crate::execution::spark_plan::SparkPlan; use crate::jvm_bridge::jni_new_global_ref; use crate::{ errors::CometError, diff --git a/native/core/src/execution/mod.rs b/native/core/src/execution/mod.rs index 3dba747f2d..a74ec3017e 100644 --- a/native/core/src/execution/mod.rs +++ b/native/core/src/execution/mod.rs @@ -16,13 +16,16 @@ // under the License. //! PoC of vectorization execution through JNI to Rust. -pub mod datafusion; +pub mod expressions; pub mod jni_api; mod metrics; pub mod operators; +pub(crate) mod planner; pub mod serde; pub mod shuffle; pub(crate) mod sort; +pub(crate) mod spark_plan; +pub(crate) mod util; pub use datafusion_comet_spark_expr::timezone; pub(crate) mod utils; diff --git a/native/core/src/execution/datafusion/operators/expand.rs b/native/core/src/execution/operators/expand.rs similarity index 97% rename from native/core/src/execution/datafusion/operators/expand.rs rename to native/core/src/execution/operators/expand.rs index a3dd06507e..fb43a6e49f 100644 --- a/native/core/src/execution/datafusion/operators/expand.rs +++ b/native/core/src/execution/operators/expand.rs @@ -37,14 +37,14 @@ use std::{ /// A Comet native operator that expands a single row into multiple rows. This behaves as same as /// Spark Expand operator. #[derive(Debug)] -pub struct CometExpandExec { +pub struct ExpandExec { projections: Vec>>, child: Arc, schema: SchemaRef, cache: PlanProperties, } -impl CometExpandExec { +impl ExpandExec { /// Create a new ExpandExec pub fn new( projections: Vec>>, @@ -66,7 +66,7 @@ impl CometExpandExec { } } -impl DisplayAs for CometExpandExec { +impl DisplayAs for ExpandExec { fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { @@ -87,7 +87,7 @@ impl DisplayAs for CometExpandExec { } } -impl ExecutionPlan for CometExpandExec { +impl ExecutionPlan for ExpandExec { fn as_any(&self) -> &dyn Any { self } @@ -104,7 +104,7 @@ impl ExecutionPlan for CometExpandExec { self: Arc, children: Vec>, ) -> datafusion_common::Result> { - let new_expand = CometExpandExec::new( + let new_expand = ExpandExec::new( self.projections.clone(), Arc::clone(&children[0]), Arc::clone(&self.schema), diff --git a/native/core/src/execution/operators/mod.rs b/native/core/src/execution/operators/mod.rs index bdc233e942..4e15e4341a 100644 --- a/native/core/src/execution/operators/mod.rs +++ b/native/core/src/execution/operators/mod.rs @@ -27,6 +27,8 @@ pub use filter::FilterExec; pub use scan::*; mod copy; +mod expand; +pub use expand::ExpandExec; mod filter; mod scan; diff --git a/native/core/src/execution/operators/scan.rs b/native/core/src/execution/operators/scan.rs index 0d35859dfb..a297f87c1f 100644 --- a/native/core/src/execution/operators/scan.rs +++ b/native/core/src/execution/operators/scan.rs @@ -18,8 +18,7 @@ use crate::{ errors::CometError, execution::{ - datafusion::planner::TEST_EXEC_CONTEXT_ID, operators::ExecutionError, - utils::SparkArrowConvert, + operators::ExecutionError, planner::TEST_EXEC_CONTEXT_ID, utils::SparkArrowConvert, }, jvm_bridge::{jni_call, JVMClasses}, }; diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/planner.rs similarity index 98% rename from native/core/src/execution/datafusion/planner.rs rename to native/core/src/execution/planner.rs index 0e64ed6afa..3ac830c04e 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/planner.rs @@ -22,17 +22,13 @@ use crate::execution::operators::{CopyMode, FilterExec}; use crate::{ errors::ExpressionError, execution::{ - datafusion::{ - expressions::{ - bloom_filter_agg::BloomFilterAgg, - bloom_filter_might_contain::BloomFilterMightContain, checkoverflow::CheckOverflow, - negative, subquery::Subquery, unbound::UnboundColumn, - }, - operators::expand::CometExpandExec, - shuffle_writer::ShuffleWriterExec, + expressions::{ + bloom_filter_agg::BloomFilterAgg, bloom_filter_might_contain::BloomFilterMightContain, + subquery::Subquery, }, - operators::{CopyExec, ExecutionError, ScanExec}, + operators::{CopyExec, ExecutionError, ExpandExec, ScanExec}, serde::to_arrow_datatype, + shuffle::ShuffleWriterExec, }, }; use arrow::compute::CastOptions; @@ -68,11 +64,11 @@ use datafusion::{ }, prelude::SessionContext, }; -use datafusion_comet_spark_expr::create_comet_physical_fun; +use datafusion_comet_spark_expr::{create_comet_physical_fun, create_negate_expr}; use datafusion_functions_nested::concat::ArrayAppend; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; -use crate::execution::datafusion::spark_plan::SparkPlan; +use crate::execution::spark_plan::SparkPlan; use datafusion_comet_proto::{ spark_expression::{ self, agg_expr::ExprStruct as AggExprStruct, expr::ExprStruct, literal::Value, AggExpr, @@ -86,11 +82,11 @@ use datafusion_comet_proto::{ spark_partitioning::{partitioning::PartitioningStruct, Partitioning as SparkPartitioning}, }; use datafusion_comet_spark_expr::{ - ArrayInsert, Avg, AvgDecimal, BitwiseNotExpr, Cast, Contains, Correlation, Covariance, - CreateNamedStruct, DateTruncExpr, EndsWith, GetArrayStructFields, GetStructField, HourExpr, - IfExpr, Like, ListExtract, MinuteExpr, NormalizeNaNAndZero, RLike, SecondExpr, + ArrayInsert, Avg, AvgDecimal, BitwiseNotExpr, Cast, CheckOverflow, Contains, Correlation, + Covariance, CreateNamedStruct, DateTruncExpr, EndsWith, GetArrayStructFields, GetStructField, + HourExpr, IfExpr, Like, ListExtract, MinuteExpr, NormalizeNaNAndZero, RLike, SecondExpr, SparkCastOptions, StartsWith, Stddev, StringSpaceExpr, SubstringExpr, SumDecimal, - TimestampTruncExpr, ToJson, Variance, + TimestampTruncExpr, ToJson, UnboundColumn, Variance, }; use datafusion_common::scalar::ScalarStructBuilder; use datafusion_common::{ @@ -611,7 +607,7 @@ impl PhysicalPlanner { ExprStruct::UnaryMinus(expr) => { let child: Arc = self.create_expr(expr.child.as_ref().unwrap(), Arc::clone(&input_schema))?; - let result = negative::create_negate_expr(child, expr.fail_on_error); + let result = create_negate_expr(child, expr.fail_on_error); result.map_err(|e| ExecutionError::GeneralError(e.to_string())) } ExprStruct::NormalizeNanAndZero(expr) => { @@ -1118,7 +1114,7 @@ impl PhysicalPlanner { } else { Arc::clone(&child.native_plan) }; - let expand = Arc::new(CometExpandExec::new(projections, input, schema)); + let expand = Arc::new(ExpandExec::new(projections, input, schema)); Ok(( scans, Arc::new(SparkPlan::new(spark_plan.plan_id, expand, vec![child])), @@ -2270,7 +2266,7 @@ mod tests { use datafusion::{physical_plan::common::collect, prelude::SessionContext}; use tokio::sync::mpsc; - use crate::execution::{datafusion::planner::PhysicalPlanner, operators::InputBatch}; + use crate::execution::{operators::InputBatch, planner::PhysicalPlanner}; use crate::execution::operators::ExecutionError; use datafusion_comet_proto::{ diff --git a/native/core/src/execution/shuffle/mod.rs b/native/core/src/execution/shuffle/mod.rs index b052df29b3..8721ead74b 100644 --- a/native/core/src/execution/shuffle/mod.rs +++ b/native/core/src/execution/shuffle/mod.rs @@ -18,3 +18,5 @@ mod list; mod map; pub mod row; +mod shuffle_writer; +pub use shuffle_writer::ShuffleWriterExec; diff --git a/native/core/src/execution/shuffle/row.rs b/native/core/src/execution/shuffle/row.rs index 17b180e9d0..ce752e68af 100644 --- a/native/core/src/execution/shuffle/row.rs +++ b/native/core/src/execution/shuffle/row.rs @@ -20,10 +20,10 @@ use crate::{ errors::CometError, execution::{ - datafusion::shuffle_writer::{write_ipc_compressed, Checksum}, shuffle::{ list::{append_list_element, SparkUnsafeArray}, map::{append_map_elements, get_map_key_value_dt, SparkUnsafeMap}, + shuffle_writer::{write_ipc_compressed, Checksum}, }, utils::bytes_to_i128, }, diff --git a/native/core/src/execution/datafusion/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs similarity index 100% rename from native/core/src/execution/datafusion/shuffle_writer.rs rename to native/core/src/execution/shuffle/shuffle_writer.rs diff --git a/native/core/src/execution/datafusion/spark_plan.rs b/native/core/src/execution/spark_plan.rs similarity index 100% rename from native/core/src/execution/datafusion/spark_plan.rs rename to native/core/src/execution/spark_plan.rs diff --git a/native/core/src/execution/datafusion/util/mod.rs b/native/core/src/execution/util/mod.rs similarity index 100% rename from native/core/src/execution/datafusion/util/mod.rs rename to native/core/src/execution/util/mod.rs diff --git a/native/core/src/execution/datafusion/util/spark_bit_array.rs b/native/core/src/execution/util/spark_bit_array.rs similarity index 100% rename from native/core/src/execution/datafusion/util/spark_bit_array.rs rename to native/core/src/execution/util/spark_bit_array.rs diff --git a/native/core/src/execution/datafusion/util/spark_bloom_filter.rs b/native/core/src/execution/util/spark_bloom_filter.rs similarity index 98% rename from native/core/src/execution/datafusion/util/spark_bloom_filter.rs rename to native/core/src/execution/util/spark_bloom_filter.rs index 35fa23b460..2c3af16916 100644 --- a/native/core/src/execution/datafusion/util/spark_bloom_filter.rs +++ b/native/core/src/execution/util/spark_bloom_filter.rs @@ -15,8 +15,8 @@ // specific language governing permissions and limitations // under the License. -use crate::execution::datafusion::util::spark_bit_array; -use crate::execution::datafusion::util::spark_bit_array::SparkBitArray; +use crate::execution::util::spark_bit_array; +use crate::execution::util::spark_bit_array::SparkBitArray; use arrow_array::{ArrowNativeTypeOp, BooleanArray, Int64Array}; use arrow_buffer::ToByteSlice; use datafusion_comet_spark_expr::spark_hash::spark_compatible_murmur3_hash; diff --git a/native/spark-expr/Cargo.toml b/native/spark-expr/Cargo.toml index 65517431d2..d0bc2fd9dd 100644 --- a/native/spark-expr/Cargo.toml +++ b/native/spark-expr/Cargo.toml @@ -29,6 +29,7 @@ edition = { workspace = true } [dependencies] arrow = { workspace = true } arrow-array = { workspace = true } +arrow-buffer = { workspace = true } arrow-data = { workspace = true } arrow-schema = { workspace = true } chrono = { workspace = true } diff --git a/native/core/src/execution/datafusion/expressions/checkoverflow.rs b/native/spark-expr/src/checkoverflow.rs similarity index 100% rename from native/core/src/execution/datafusion/expressions/checkoverflow.rs rename to native/spark-expr/src/checkoverflow.rs diff --git a/native/spark-expr/src/lib.rs b/native/spark-expr/src/lib.rs index 5dff6e0b8f..8a57480587 100644 --- a/native/spark-expr/src/lib.rs +++ b/native/spark-expr/src/lib.rs @@ -29,6 +29,8 @@ mod bitwise_not; pub use bitwise_not::{bitwise_not, BitwiseNotExpr}; mod avg_decimal; pub use avg_decimal::AvgDecimal; +mod checkoverflow; +pub use checkoverflow::CheckOverflow; mod correlation; pub use correlation::Correlation; mod covariance; @@ -45,10 +47,14 @@ pub use stddev::Stddev; mod structs; mod sum_decimal; pub use sum_decimal::SumDecimal; +mod negative; +pub use negative::{create_negate_expr, NegativeExpr}; mod normalize_nan; mod temporal; pub mod timezone; mod to_json; +mod unbound; +pub use unbound::UnboundColumn; pub mod utils; pub use normalize_nan::NormalizeNaNAndZero; @@ -83,3 +89,9 @@ pub enum EvalMode { /// failing the entire query. Try, } + +pub(crate) fn arithmetic_overflow_error(from_type: &str) -> SparkError { + SparkError::ArithmeticOverflow { + from_type: from_type.to_string(), + } +} diff --git a/native/core/src/execution/datafusion/expressions/negative.rs b/native/spark-expr/src/negative.rs similarity index 98% rename from native/core/src/execution/datafusion/expressions/negative.rs rename to native/spark-expr/src/negative.rs index 8dfe717422..3d9063e783 100644 --- a/native/core/src/execution/datafusion/expressions/negative.rs +++ b/native/spark-expr/src/negative.rs @@ -16,7 +16,7 @@ // under the License. use super::arithmetic_overflow_error; -use crate::errors::CometError; +use crate::SparkError; use arrow::{compute::kernels::numeric::neg_wrapping, datatypes::IntervalDayTimeType}; use arrow_array::RecordBatch; use arrow_buffer::IntervalDayTime; @@ -26,8 +26,7 @@ use datafusion::{ logical_expr::{interval_arithmetic::Interval, ColumnarValue}, physical_expr::PhysicalExpr, }; -use datafusion_comet_spark_expr::SparkError; -use datafusion_common::{Result, ScalarValue}; +use datafusion_common::{DataFusionError, Result, ScalarValue}; use datafusion_expr::sort_properties::ExprProperties; use std::{ any::Any, @@ -38,7 +37,7 @@ use std::{ pub fn create_negate_expr( expr: Arc, fail_on_error: bool, -) -> Result, CometError> { +) -> Result, DataFusionError> { Ok(Arc::new(NegativeExpr::new(expr, fail_on_error))) } diff --git a/native/core/src/execution/datafusion/expressions/unbound.rs b/native/spark-expr/src/unbound.rs similarity index 100% rename from native/core/src/execution/datafusion/expressions/unbound.rs rename to native/spark-expr/src/unbound.rs From f1d08791e0603e5543702fa952365d4e61f8df4c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 12 Dec 2024 16:41:12 -0700 Subject: [PATCH 35/83] chore: Add ignored tests for reading complex types from Parquet (#1167) * Add ignored tests for reading structs from Parquet * add basic map test * add tests for Map and Array --- .../apache/comet/CometExpressionSuite.scala | 127 ++++++++++++++++++ 1 file changed, 127 insertions(+) diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index 35f374bf0f..cce7cb20a1 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -2195,6 +2195,133 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } + ignore("get_struct_field - select primitive fields") { + withTempPath { dir => + // create input file with Comet disabled + withSQLConf(CometConf.COMET_ENABLED.key -> "false") { + val df = spark + .range(5) + // Add both a null struct and null inner value + .select(when(col("id") > 1, struct(when(col("id") > 2, col("id")).alias("id"))) + .alias("nested1")) + + df.write.parquet(dir.toString()) + } + + Seq("", "parquet").foreach { v1List => + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> v1List) { + val df = spark.read.parquet(dir.toString()) + checkSparkAnswerAndOperator(df.select("nested1.id")) + } + } + } + } + + ignore("get_struct_field - select subset of struct") { + withTempPath { dir => + // create input file with Comet disabled + withSQLConf(CometConf.COMET_ENABLED.key -> "false") { + val df = spark + .range(5) + // Add both a null struct and null inner value + .select( + when( + col("id") > 1, + struct( + when(col("id") > 2, col("id")).alias("id"), + when(col("id") > 2, struct(when(col("id") > 3, col("id")).alias("id"))) + .as("nested2"))) + .alias("nested1")) + + df.write.parquet(dir.toString()) + } + + Seq("", "parquet").foreach { v1List => + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> v1List) { + val df = spark.read.parquet(dir.toString()) + checkSparkAnswerAndOperator(df.select("nested1.id")) + checkSparkAnswerAndOperator(df.select("nested1.nested2")) + checkSparkAnswerAndOperator(df.select("nested1.nested2.id")) + checkSparkAnswerAndOperator(df.select("nested1.id", "nested1.nested2.id")) + } + } + } + } + + ignore("get_struct_field - read entire struct") { + withTempPath { dir => + // create input file with Comet disabled + withSQLConf(CometConf.COMET_ENABLED.key -> "false") { + val df = spark + .range(5) + // Add both a null struct and null inner value + .select( + when( + col("id") > 1, + struct( + when(col("id") > 2, col("id")).alias("id"), + when(col("id") > 2, struct(when(col("id") > 3, col("id")).alias("id"))) + .as("nested2"))) + .alias("nested1")) + + df.write.parquet(dir.toString()) + } + + Seq("", "parquet").foreach { v1List => + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> v1List) { + val df = spark.read.parquet(dir.toString()) + checkSparkAnswerAndOperator(df.select("nested1")) + } + } + } + } + + ignore("read map[int, int] from parquet") { + withTempPath { dir => + // create input file with Comet disabled + withSQLConf(CometConf.COMET_ENABLED.key -> "false") { + val df = spark + .range(5) + // Spark does not allow null as a key but does allow null as a + // value, and the entire map be null + .select( + when(col("id") > 1, map(col("id"), when(col("id") > 2, col("id")))).alias("map1")) + df.write.parquet(dir.toString()) + } + + Seq("", "parquet").foreach { v1List => + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> v1List) { + val df = spark.read.parquet(dir.toString()) + checkSparkAnswerAndOperator(df.select("map1")) + checkSparkAnswerAndOperator(df.select(map_keys(col("map1")))) + checkSparkAnswerAndOperator(df.select(map_values(col("map1")))) + } + } + } + } + + ignore("read array[int] from parquet") { + withTempPath { dir => + // create input file with Comet disabled + withSQLConf(CometConf.COMET_ENABLED.key -> "false") { + val df = spark + .range(5) + // Spark does not allow null as a key but does allow null as a + // value, and the entire map be null + .select(when(col("id") > 1, sequence(lit(0), col("id") * 2)).alias("array1")) + df.write.parquet(dir.toString()) + } + + Seq("", "parquet").foreach { v1List => + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> v1List) { + val df = spark.read.parquet(dir.toString()) + checkSparkAnswerAndOperator(df.select("array1")) + checkSparkAnswerAndOperator(df.select(element_at(col("array1"), lit(1)))) + } + } + } + } + test("CreateArray") { Seq(true, false).foreach { dictionaryEnabled => withTempDir { dir => From b9ac78beffe8b71388ceb8d7e842fd6a07395829 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 17 Dec 2024 16:35:54 -0700 Subject: [PATCH 36/83] feat: Add Spark-compatible implementation of SchemaAdapterFactory (#1169) * Add Spark-compatible SchemaAdapterFactory implementation * remove prototype code * fix * refactor * implement more cast logic * implement more cast logic * add basic test * improve test * cleanup * fmt * add support for casting unsigned int to signed int * clippy * address feedback * fix test --- native/Cargo.lock | 67 +++- native/Cargo.toml | 2 +- .../core/src/parquet/util/test_common/mod.rs | 3 +- native/spark-expr/Cargo.toml | 4 +- native/spark-expr/src/cast.rs | 353 +++++++++++++--- native/spark-expr/src/lib.rs | 5 + native/spark-expr/src/schema_adapter.rs | 376 ++++++++++++++++++ .../src}/test_common/file_util.rs | 0 native/spark-expr/src/test_common/mod.rs | 17 + 9 files changed, 758 insertions(+), 69 deletions(-) create mode 100644 native/spark-expr/src/schema_adapter.rs rename native/{core/src/parquet/util => spark-expr/src}/test_common/file_util.rs (100%) create mode 100644 native/spark-expr/src/test_common/mod.rs diff --git a/native/Cargo.lock b/native/Cargo.lock index 7966bb80bb..538c40ee23 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -436,7 +436,18 @@ checksum = "d640d25bc63c50fb1f0b545ffd80207d2e10a4c965530809b40ba3386825c391" dependencies = [ "alloc-no-stdlib", "alloc-stdlib", - "brotli-decompressor", + "brotli-decompressor 2.5.1", +] + +[[package]] +name = "brotli" +version = "7.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cc97b8f16f944bba54f0433f07e30be199b6dc2bd25937444bbad560bcea29bd" +dependencies = [ + "alloc-no-stdlib", + "alloc-stdlib", + "brotli-decompressor 4.0.1", ] [[package]] @@ -449,6 +460,16 @@ dependencies = [ "alloc-stdlib", ] +[[package]] +name = "brotli-decompressor" +version = "4.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9a45bd2e4095a8b518033b128020dd4a55aab1c0a381ba4404a472630f4bc362" +dependencies = [ + "alloc-no-stdlib", + "alloc-stdlib", +] + [[package]] name = "bumpalo" version = "3.16.0" @@ -842,6 +863,7 @@ dependencies = [ "num_cpus", "object_store", "parking_lot", + "parquet", "paste", "pin-project-lite", "rand", @@ -878,7 +900,7 @@ dependencies = [ "arrow-schema", "assertables", "async-trait", - "brotli", + "brotli 3.5.0", "bytes", "crc32fast", "criterion", @@ -914,7 +936,7 @@ dependencies = [ "tempfile", "thiserror", "tokio", - "zstd", + "zstd 0.11.2+zstd.1.5.2", ] [[package]] @@ -943,6 +965,7 @@ dependencies = [ "datafusion-physical-expr", "futures", "num", + "parquet", "rand", "regex", "thiserror", @@ -969,6 +992,7 @@ dependencies = [ "libc", "num_cpus", "object_store", + "parquet", "paste", "sqlparser", "tokio", @@ -2350,16 +2374,33 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dea02606ba6f5e856561d8d507dba8bac060aefca2a6c0f1aa1d361fed91ff3e" dependencies = [ "ahash", + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-ipc", + "arrow-schema", + "arrow-select", + "base64", + "brotli 7.0.0", "bytes", "chrono", + "flate2", + "futures", "half", "hashbrown 0.14.5", + "lz4_flex", "num", "num-bigint", + "object_store", "paste", "seq-macro", + "snap", "thrift", + "tokio", "twox-hash 1.6.3", + "zstd 0.13.2", + "zstd-sys", ] [[package]] @@ -3652,7 +3693,16 @@ version = "0.11.2+zstd.1.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "20cc960326ece64f010d2d2107537f26dc589a6573a316bd5b1dba685fa5fde4" dependencies = [ - "zstd-safe", + "zstd-safe 5.0.2+zstd.1.5.2", +] + +[[package]] +name = "zstd" +version = "0.13.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fcf2b778a664581e31e389454a7072dab1647606d44f7feea22cd5abb9c9f3f9" +dependencies = [ + "zstd-safe 7.2.1", ] [[package]] @@ -3665,6 +3715,15 @@ dependencies = [ "zstd-sys", ] +[[package]] +name = "zstd-safe" +version = "7.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "54a3ab4db68cea366acc5c897c7b4d4d1b8994a9cd6e6f841f8964566a419059" +dependencies = [ + "zstd-sys", +] + [[package]] name = "zstd-sys" version = "2.0.13+zstd.1.5.6" diff --git a/native/Cargo.toml b/native/Cargo.toml index 4ac85479f2..bd46cf0c9f 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -39,8 +39,8 @@ arrow-buffer = { version = "53.2.0" } arrow-data = { version = "53.2.0" } arrow-schema = { version = "53.2.0" } parquet = { version = "53.2.0", default-features = false, features = ["experimental"] } -datafusion-common = { version = "43.0.0" } datafusion = { version = "43.0.0", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } +datafusion-common = { version = "43.0.0" } datafusion-functions = { version = "43.0.0", features = ["crypto_expressions"] } datafusion-functions-nested = { version = "43.0.0", default-features = false } datafusion-expr = { version = "43.0.0", default-features = false } diff --git a/native/core/src/parquet/util/test_common/mod.rs b/native/core/src/parquet/util/test_common/mod.rs index e46d732239..d92544608e 100644 --- a/native/core/src/parquet/util/test_common/mod.rs +++ b/native/core/src/parquet/util/test_common/mod.rs @@ -15,10 +15,9 @@ // specific language governing permissions and limitations // under the License. -pub mod file_util; pub mod page_util; pub mod rand_gen; pub use self::rand_gen::{random_bools, random_bytes, random_numbers, random_numbers_range}; -pub use self::file_util::{get_temp_file, get_temp_filename}; +pub use datafusion_comet_spark_expr::test_common::file_util::{get_temp_file, get_temp_filename}; diff --git a/native/spark-expr/Cargo.toml b/native/spark-expr/Cargo.toml index d0bc2fd9dd..27367d83e1 100644 --- a/native/spark-expr/Cargo.toml +++ b/native/spark-expr/Cargo.toml @@ -33,7 +33,7 @@ arrow-buffer = { workspace = true } arrow-data = { workspace = true } arrow-schema = { workspace = true } chrono = { workspace = true } -datafusion = { workspace = true } +datafusion = { workspace = true, features = ["parquet"] } datafusion-common = { workspace = true } datafusion-expr = { workspace = true } datafusion-physical-expr = { workspace = true } @@ -43,9 +43,11 @@ regex = { workspace = true } thiserror = { workspace = true } futures = { workspace = true } twox-hash = "2.0.0" +rand = { workspace = true } [dev-dependencies] arrow-data = {workspace = true} +parquet = { workspace = true, features = ["arrow"] } criterion = "0.5.1" rand = { workspace = true} tokio = { version = "1", features = ["rt-multi-thread"] } diff --git a/native/spark-expr/src/cast.rs b/native/spark-expr/src/cast.rs index f62d0220c9..d96bcbbdb6 100644 --- a/native/spark-expr/src/cast.rs +++ b/native/spark-expr/src/cast.rs @@ -15,6 +15,9 @@ // specific language governing permissions and limitations // under the License. +use crate::timezone; +use crate::utils::array_with_timezone; +use crate::{EvalMode, SparkError, SparkResult}; use arrow::{ array::{ cast::AsArray, @@ -35,11 +38,18 @@ use arrow::{ use arrow_array::builder::StringBuilder; use arrow_array::{DictionaryArray, StringArray, StructArray}; use arrow_schema::{DataType, Field, Schema}; +use chrono::{NaiveDate, NaiveDateTime, TimeZone, Timelike}; +use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{ cast::as_generic_string_array, internal_err, Result as DataFusionResult, ScalarValue, }; use datafusion_expr::ColumnarValue; use datafusion_physical_expr::PhysicalExpr; +use num::{ + cast::AsPrimitive, integer::div_floor, traits::CheckedNeg, CheckedSub, Integer, Num, + ToPrimitive, +}; +use regex::Regex; use std::str::FromStr; use std::{ any::Any, @@ -49,19 +59,6 @@ use std::{ sync::Arc, }; -use chrono::{NaiveDate, NaiveDateTime, TimeZone, Timelike}; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; -use num::{ - cast::AsPrimitive, integer::div_floor, traits::CheckedNeg, CheckedSub, Integer, Num, - ToPrimitive, -}; -use regex::Regex; - -use crate::timezone; -use crate::utils::array_with_timezone; - -use crate::{EvalMode, SparkError, SparkResult}; - static TIMESTAMP_FORMAT: Option<&str> = Some("%Y-%m-%d %H:%M:%S%.f"); const MICROS_PER_SECOND: i64 = 1000000; @@ -141,6 +138,240 @@ pub struct Cast { pub cast_options: SparkCastOptions, } +/// Determine if Comet supports a cast, taking options such as EvalMode and Timezone into account. +pub fn cast_supported( + from_type: &DataType, + to_type: &DataType, + options: &SparkCastOptions, +) -> bool { + use DataType::*; + + let from_type = if let Dictionary(_, dt) = from_type { + dt + } else { + from_type + }; + + let to_type = if let Dictionary(_, dt) = to_type { + dt + } else { + to_type + }; + + if from_type == to_type { + return true; + } + + match (from_type, to_type) { + (Boolean, _) => can_cast_from_boolean(to_type, options), + (UInt8 | UInt16 | UInt32 | UInt64, Int8 | Int16 | Int32 | Int64) + if options.allow_cast_unsigned_ints => + { + true + } + (Int8, _) => can_cast_from_byte(to_type, options), + (Int16, _) => can_cast_from_short(to_type, options), + (Int32, _) => can_cast_from_int(to_type, options), + (Int64, _) => can_cast_from_long(to_type, options), + (Float32, _) => can_cast_from_float(to_type, options), + (Float64, _) => can_cast_from_double(to_type, options), + (Decimal128(p, s), _) => can_cast_from_decimal(p, s, to_type, options), + (Timestamp(_, None), _) => can_cast_from_timestamp_ntz(to_type, options), + (Timestamp(_, Some(_)), _) => can_cast_from_timestamp(to_type, options), + (Utf8 | LargeUtf8, _) => can_cast_from_string(to_type, options), + (_, Utf8 | LargeUtf8) => can_cast_to_string(from_type, options), + (Struct(from_fields), Struct(to_fields)) => from_fields + .iter() + .zip(to_fields.iter()) + .all(|(a, b)| cast_supported(a.data_type(), b.data_type(), options)), + _ => false, + } +} + +fn can_cast_from_string(to_type: &DataType, options: &SparkCastOptions) -> bool { + use DataType::*; + match to_type { + Boolean | Int8 | Int16 | Int32 | Int64 | Binary => true, + Float32 | Float64 => { + // https://github.com/apache/datafusion-comet/issues/326 + // Does not support inputs ending with 'd' or 'f'. Does not support 'inf'. + // Does not support ANSI mode. + options.allow_incompat + } + Decimal128(_, _) => { + // https://github.com/apache/datafusion-comet/issues/325 + // Does not support inputs ending with 'd' or 'f'. Does not support 'inf'. + // Does not support ANSI mode. Returns 0.0 instead of null if input contains no digits + + options.allow_incompat + } + Date32 | Date64 => { + // https://github.com/apache/datafusion-comet/issues/327 + // Only supports years between 262143 BC and 262142 AD + options.allow_incompat + } + Timestamp(_, _) if options.eval_mode == EvalMode::Ansi => { + // ANSI mode not supported + false + } + Timestamp(_, Some(tz)) if tz.as_ref() != "UTC" => { + // Cast will use UTC instead of $timeZoneId + options.allow_incompat + } + Timestamp(_, _) => { + // https://github.com/apache/datafusion-comet/issues/328 + // Not all valid formats are supported + options.allow_incompat + } + _ => false, + } +} + +fn can_cast_to_string(from_type: &DataType, options: &SparkCastOptions) -> bool { + use DataType::*; + match from_type { + Boolean | Int8 | Int16 | Int32 | Int64 | Date32 | Date64 | Timestamp(_, _) => true, + Float32 | Float64 => { + // There can be differences in precision. + // For example, the input \"1.4E-45\" will produce 1.0E-45 " + + // instead of 1.4E-45")) + true + } + Decimal128(_, _) => { + // https://github.com/apache/datafusion-comet/issues/1068 + // There can be formatting differences in some case due to Spark using + // scientific notation where Comet does not + true + } + Binary => { + // https://github.com/apache/datafusion-comet/issues/377 + // Only works for binary data representing valid UTF-8 strings + options.allow_incompat + } + Struct(fields) => fields + .iter() + .all(|f| can_cast_to_string(f.data_type(), options)), + _ => false, + } +} + +fn can_cast_from_timestamp_ntz(to_type: &DataType, options: &SparkCastOptions) -> bool { + use DataType::*; + match to_type { + Timestamp(_, _) | Date32 | Date64 | Utf8 => { + // incompatible + options.allow_incompat + } + _ => { + // unsupported + false + } + } +} + +fn can_cast_from_timestamp(to_type: &DataType, _options: &SparkCastOptions) -> bool { + use DataType::*; + match to_type { + Boolean | Int8 | Int16 => { + // https://github.com/apache/datafusion-comet/issues/352 + // this seems like an edge case that isn't important for us to support + false + } + Int64 => { + // https://github.com/apache/datafusion-comet/issues/352 + true + } + Date32 | Date64 | Utf8 | Decimal128(_, _) => true, + _ => { + // unsupported + false + } + } +} + +fn can_cast_from_boolean(to_type: &DataType, _: &SparkCastOptions) -> bool { + use DataType::*; + matches!(to_type, Int8 | Int16 | Int32 | Int64 | Float32 | Float64) +} + +fn can_cast_from_byte(to_type: &DataType, _: &SparkCastOptions) -> bool { + use DataType::*; + matches!( + to_type, + Boolean | Int8 | Int16 | Int32 | Int64 | Float32 | Float64 | Decimal128(_, _) + ) +} + +fn can_cast_from_short(to_type: &DataType, _: &SparkCastOptions) -> bool { + use DataType::*; + matches!( + to_type, + Boolean | Int8 | Int16 | Int32 | Int64 | Float32 | Float64 | Decimal128(_, _) + ) +} + +fn can_cast_from_int(to_type: &DataType, options: &SparkCastOptions) -> bool { + use DataType::*; + match to_type { + Boolean | Int8 | Int16 | Int32 | Int64 | Float32 | Float64 | Utf8 => true, + Decimal128(_, _) => { + // incompatible: no overflow check + options.allow_incompat + } + _ => false, + } +} + +fn can_cast_from_long(to_type: &DataType, options: &SparkCastOptions) -> bool { + use DataType::*; + match to_type { + Boolean | Int8 | Int16 | Int32 | Int64 | Float32 | Float64 => true, + Decimal128(_, _) => { + // incompatible: no overflow check + options.allow_incompat + } + _ => false, + } +} + +fn can_cast_from_float(to_type: &DataType, _: &SparkCastOptions) -> bool { + use DataType::*; + matches!( + to_type, + Boolean | Int8 | Int16 | Int32 | Int64 | Float64 | Decimal128(_, _) + ) +} + +fn can_cast_from_double(to_type: &DataType, _: &SparkCastOptions) -> bool { + use DataType::*; + matches!( + to_type, + Boolean | Int8 | Int16 | Int32 | Int64 | Float32 | Decimal128(_, _) + ) +} + +fn can_cast_from_decimal( + p1: &u8, + _s1: &i8, + to_type: &DataType, + options: &SparkCastOptions, +) -> bool { + use DataType::*; + match to_type { + Int8 | Int16 | Int32 | Int64 | Float32 | Float64 => true, + Decimal128(p2, _) => { + if p2 < p1 { + // https://github.com/apache/datafusion/issues/13492 + // Incompatible(Some("Casting to smaller precision is not supported")) + options.allow_incompat + } else { + true + } + } + _ => false, + } +} + macro_rules! cast_utf8_to_int { ($array:expr, $eval_mode:expr, $array_type:ty, $cast_method:ident) => {{ let len = $array.len(); @@ -560,6 +791,8 @@ pub struct SparkCastOptions { pub timezone: String, /// Allow casts that are supported but not guaranteed to be 100% compatible pub allow_incompat: bool, + /// Support casting unsigned ints to signed ints (used by Parquet SchemaAdapter) + pub allow_cast_unsigned_ints: bool, } impl SparkCastOptions { @@ -568,6 +801,7 @@ impl SparkCastOptions { eval_mode, timezone: timezone.to_string(), allow_incompat, + allow_cast_unsigned_ints: false, } } @@ -576,6 +810,7 @@ impl SparkCastOptions { eval_mode, timezone: "".to_string(), allow_incompat, + allow_cast_unsigned_ints: false, } } } @@ -611,14 +846,14 @@ fn cast_array( to_type: &DataType, cast_options: &SparkCastOptions, ) -> DataFusionResult { + use DataType::*; let array = array_with_timezone(array, cast_options.timezone.clone(), Some(to_type))?; let from_type = array.data_type().clone(); let array = match &from_type { - DataType::Dictionary(key_type, value_type) - if key_type.as_ref() == &DataType::Int32 - && (value_type.as_ref() == &DataType::Utf8 - || value_type.as_ref() == &DataType::LargeUtf8) => + Dictionary(key_type, value_type) + if key_type.as_ref() == &Int32 + && (value_type.as_ref() == &Utf8 || value_type.as_ref() == &LargeUtf8) => { let dict_array = array .as_any() @@ -631,7 +866,7 @@ fn cast_array( ); let casted_result = match to_type { - DataType::Dictionary(_, _) => Arc::new(casted_dictionary.clone()), + Dictionary(_, _) => Arc::new(casted_dictionary.clone()), _ => take(casted_dictionary.values().as_ref(), dict_array.keys(), None)?, }; return Ok(spark_cast_postprocess(casted_result, &from_type, to_type)); @@ -642,70 +877,66 @@ fn cast_array( let eval_mode = cast_options.eval_mode; let cast_result = match (from_type, to_type) { - (DataType::Utf8, DataType::Boolean) => spark_cast_utf8_to_boolean::(&array, eval_mode), - (DataType::LargeUtf8, DataType::Boolean) => { - spark_cast_utf8_to_boolean::(&array, eval_mode) - } - (DataType::Utf8, DataType::Timestamp(_, _)) => { + (Utf8, Boolean) => spark_cast_utf8_to_boolean::(&array, eval_mode), + (LargeUtf8, Boolean) => spark_cast_utf8_to_boolean::(&array, eval_mode), + (Utf8, Timestamp(_, _)) => { cast_string_to_timestamp(&array, to_type, eval_mode, &cast_options.timezone) } - (DataType::Utf8, DataType::Date32) => cast_string_to_date(&array, to_type, eval_mode), - (DataType::Int64, DataType::Int32) - | (DataType::Int64, DataType::Int16) - | (DataType::Int64, DataType::Int8) - | (DataType::Int32, DataType::Int16) - | (DataType::Int32, DataType::Int8) - | (DataType::Int16, DataType::Int8) + (Utf8, Date32) => cast_string_to_date(&array, to_type, eval_mode), + (Int64, Int32) + | (Int64, Int16) + | (Int64, Int8) + | (Int32, Int16) + | (Int32, Int8) + | (Int16, Int8) if eval_mode != EvalMode::Try => { spark_cast_int_to_int(&array, eval_mode, from_type, to_type) } - (DataType::Utf8, DataType::Int8 | DataType::Int16 | DataType::Int32 | DataType::Int64) => { + (Utf8, Int8 | Int16 | Int32 | Int64) => { cast_string_to_int::(to_type, &array, eval_mode) } - ( - DataType::LargeUtf8, - DataType::Int8 | DataType::Int16 | DataType::Int32 | DataType::Int64, - ) => cast_string_to_int::(to_type, &array, eval_mode), - (DataType::Float64, DataType::Utf8) => spark_cast_float64_to_utf8::(&array, eval_mode), - (DataType::Float64, DataType::LargeUtf8) => { - spark_cast_float64_to_utf8::(&array, eval_mode) - } - (DataType::Float32, DataType::Utf8) => spark_cast_float32_to_utf8::(&array, eval_mode), - (DataType::Float32, DataType::LargeUtf8) => { - spark_cast_float32_to_utf8::(&array, eval_mode) - } - (DataType::Float32, DataType::Decimal128(precision, scale)) => { + (LargeUtf8, Int8 | Int16 | Int32 | Int64) => { + cast_string_to_int::(to_type, &array, eval_mode) + } + (Float64, Utf8) => spark_cast_float64_to_utf8::(&array, eval_mode), + (Float64, LargeUtf8) => spark_cast_float64_to_utf8::(&array, eval_mode), + (Float32, Utf8) => spark_cast_float32_to_utf8::(&array, eval_mode), + (Float32, LargeUtf8) => spark_cast_float32_to_utf8::(&array, eval_mode), + (Float32, Decimal128(precision, scale)) => { cast_float32_to_decimal128(&array, *precision, *scale, eval_mode) } - (DataType::Float64, DataType::Decimal128(precision, scale)) => { + (Float64, Decimal128(precision, scale)) => { cast_float64_to_decimal128(&array, *precision, *scale, eval_mode) } - (DataType::Float32, DataType::Int8) - | (DataType::Float32, DataType::Int16) - | (DataType::Float32, DataType::Int32) - | (DataType::Float32, DataType::Int64) - | (DataType::Float64, DataType::Int8) - | (DataType::Float64, DataType::Int16) - | (DataType::Float64, DataType::Int32) - | (DataType::Float64, DataType::Int64) - | (DataType::Decimal128(_, _), DataType::Int8) - | (DataType::Decimal128(_, _), DataType::Int16) - | (DataType::Decimal128(_, _), DataType::Int32) - | (DataType::Decimal128(_, _), DataType::Int64) + (Float32, Int8) + | (Float32, Int16) + | (Float32, Int32) + | (Float32, Int64) + | (Float64, Int8) + | (Float64, Int16) + | (Float64, Int32) + | (Float64, Int64) + | (Decimal128(_, _), Int8) + | (Decimal128(_, _), Int16) + | (Decimal128(_, _), Int32) + | (Decimal128(_, _), Int64) if eval_mode != EvalMode::Try => { spark_cast_nonintegral_numeric_to_integral(&array, eval_mode, from_type, to_type) } - (DataType::Struct(_), DataType::Utf8) => { - Ok(casts_struct_to_string(array.as_struct(), cast_options)?) - } - (DataType::Struct(_), DataType::Struct(_)) => Ok(cast_struct_to_struct( + (Struct(_), Utf8) => Ok(casts_struct_to_string(array.as_struct(), cast_options)?), + (Struct(_), Struct(_)) => Ok(cast_struct_to_struct( array.as_struct(), from_type, to_type, cast_options, )?), + (UInt8 | UInt16 | UInt32 | UInt64, Int8 | Int16 | Int32 | Int64) + if cast_options.allow_cast_unsigned_ints => + { + Ok(cast_with_options(&array, to_type, &CAST_OPTIONS)?) + } _ if is_datafusion_spark_compatible(from_type, to_type, cast_options.allow_incompat) => { // use DataFusion cast only when we know that it is compatible with Spark Ok(cast_with_options(&array, to_type, &CAST_OPTIONS)?) diff --git a/native/spark-expr/src/lib.rs b/native/spark-expr/src/lib.rs index 8a57480587..f358731004 100644 --- a/native/spark-expr/src/lib.rs +++ b/native/spark-expr/src/lib.rs @@ -41,6 +41,9 @@ mod kernels; mod list; mod regexp; pub mod scalar_funcs; +mod schema_adapter; +pub use schema_adapter::SparkSchemaAdapterFactory; + pub mod spark_hash; mod stddev; pub use stddev::Stddev; @@ -51,6 +54,8 @@ mod negative; pub use negative::{create_negate_expr, NegativeExpr}; mod normalize_nan; mod temporal; + +pub mod test_common; pub mod timezone; mod to_json; mod unbound; diff --git a/native/spark-expr/src/schema_adapter.rs b/native/spark-expr/src/schema_adapter.rs new file mode 100644 index 0000000000..161ad6f164 --- /dev/null +++ b/native/spark-expr/src/schema_adapter.rs @@ -0,0 +1,376 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Custom schema adapter that uses Spark-compatible casts + +use crate::cast::cast_supported; +use crate::{spark_cast, SparkCastOptions}; +use arrow_array::{new_null_array, Array, RecordBatch, RecordBatchOptions}; +use arrow_schema::{Schema, SchemaRef}; +use datafusion::datasource::schema_adapter::{SchemaAdapter, SchemaAdapterFactory, SchemaMapper}; +use datafusion_common::plan_err; +use datafusion_expr::ColumnarValue; +use std::sync::Arc; + +/// An implementation of DataFusion's `SchemaAdapterFactory` that uses a Spark-compatible +/// `cast` implementation. +#[derive(Clone, Debug)] +pub struct SparkSchemaAdapterFactory { + /// Spark cast options + cast_options: SparkCastOptions, +} + +impl SparkSchemaAdapterFactory { + pub fn new(options: SparkCastOptions) -> Self { + Self { + cast_options: options, + } + } +} + +impl SchemaAdapterFactory for SparkSchemaAdapterFactory { + /// Create a new factory for mapping batches from a file schema to a table + /// schema. + /// + /// This is a convenience for [`DefaultSchemaAdapterFactory::create`] with + /// the same schema for both the projected table schema and the table + /// schema. + fn create( + &self, + required_schema: SchemaRef, + table_schema: SchemaRef, + ) -> Box { + Box::new(SparkSchemaAdapter { + required_schema, + table_schema, + cast_options: self.cast_options.clone(), + }) + } +} + +/// This SchemaAdapter requires both the table schema and the projected table +/// schema. See [`SchemaMapping`] for more details +#[derive(Clone, Debug)] +pub struct SparkSchemaAdapter { + /// The schema for the table, projected to include only the fields being output (projected) by the + /// associated ParquetExec + required_schema: SchemaRef, + /// The entire table schema for the table we're using this to adapt. + /// + /// This is used to evaluate any filters pushed down into the scan + /// which may refer to columns that are not referred to anywhere + /// else in the plan. + table_schema: SchemaRef, + /// Spark cast options + cast_options: SparkCastOptions, +} + +impl SchemaAdapter for SparkSchemaAdapter { + /// Map a column index in the table schema to a column index in a particular + /// file schema + /// + /// Panics if index is not in range for the table schema + fn map_column_index(&self, index: usize, file_schema: &Schema) -> Option { + let field = self.required_schema.field(index); + Some(file_schema.fields.find(field.name())?.0) + } + + /// Creates a `SchemaMapping` for casting or mapping the columns from the + /// file schema to the table schema. + /// + /// If the provided `file_schema` contains columns of a different type to + /// the expected `table_schema`, the method will attempt to cast the array + /// data from the file schema to the table schema where possible. + /// + /// Returns a [`SchemaMapping`] that can be applied to the output batch + /// along with an ordered list of columns to project from the file + fn map_schema( + &self, + file_schema: &Schema, + ) -> datafusion_common::Result<(Arc, Vec)> { + let mut projection = Vec::with_capacity(file_schema.fields().len()); + let mut field_mappings = vec![None; self.required_schema.fields().len()]; + + for (file_idx, file_field) in file_schema.fields.iter().enumerate() { + if let Some((table_idx, table_field)) = + self.required_schema.fields().find(file_field.name()) + { + if cast_supported( + file_field.data_type(), + table_field.data_type(), + &self.cast_options, + ) { + field_mappings[table_idx] = Some(projection.len()); + projection.push(file_idx); + } else { + return plan_err!( + "Cannot cast file schema field {} of type {:?} to required schema field of type {:?}", + file_field.name(), + file_field.data_type(), + table_field.data_type() + ); + } + } + } + + Ok(( + Arc::new(SchemaMapping { + required_schema: Arc::::clone(&self.required_schema), + field_mappings, + table_schema: Arc::::clone(&self.table_schema), + cast_options: self.cast_options.clone(), + }), + projection, + )) + } +} + +// TODO SchemaMapping is mostly copied from DataFusion but calls spark_cast +// instead of arrow cast - can we reduce the amount of code copied here and make +// the DataFusion version more extensible? + +/// The SchemaMapping struct holds a mapping from the file schema to the table +/// schema and any necessary type conversions. +/// +/// Note, because `map_batch` and `map_partial_batch` functions have different +/// needs, this struct holds two schemas: +/// +/// 1. The projected **table** schema +/// 2. The full table schema +/// +/// [`map_batch`] is used by the ParquetOpener to produce a RecordBatch which +/// has the projected schema, since that's the schema which is supposed to come +/// out of the execution of this query. Thus `map_batch` uses +/// `projected_table_schema` as it can only operate on the projected fields. +/// +/// [`map_partial_batch`] is used to create a RecordBatch with a schema that +/// can be used for Parquet predicate pushdown, meaning that it may contain +/// fields which are not in the projected schema (as the fields that parquet +/// pushdown filters operate can be completely distinct from the fields that are +/// projected (output) out of the ParquetExec). `map_partial_batch` thus uses +/// `table_schema` to create the resulting RecordBatch (as it could be operating +/// on any fields in the schema). +/// +/// [`map_batch`]: Self::map_batch +/// [`map_partial_batch`]: Self::map_partial_batch +#[derive(Debug)] +pub struct SchemaMapping { + /// The schema of the table. This is the expected schema after conversion + /// and it should match the schema of the query result. + required_schema: SchemaRef, + /// Mapping from field index in `projected_table_schema` to index in + /// projected file_schema. + /// + /// They are Options instead of just plain `usize`s because the table could + /// have fields that don't exist in the file. + field_mappings: Vec>, + /// The entire table schema, as opposed to the projected_table_schema (which + /// only contains the columns that we are projecting out of this query). + /// This contains all fields in the table, regardless of if they will be + /// projected out or not. + table_schema: SchemaRef, + /// Spark cast options + cast_options: SparkCastOptions, +} + +impl SchemaMapper for SchemaMapping { + /// Adapts a `RecordBatch` to match the `projected_table_schema` using the stored mapping and + /// conversions. The produced RecordBatch has a schema that contains only the projected + /// columns, so if one needs a RecordBatch with a schema that references columns which are not + /// in the projected, it would be better to use `map_partial_batch` + fn map_batch(&self, batch: RecordBatch) -> datafusion_common::Result { + let batch_rows = batch.num_rows(); + let batch_cols = batch.columns().to_vec(); + + let cols = self + .required_schema + // go through each field in the projected schema + .fields() + .iter() + // and zip it with the index that maps fields from the projected table schema to the + // projected file schema in `batch` + .zip(&self.field_mappings) + // and for each one... + .map(|(field, file_idx)| { + file_idx.map_or_else( + // If this field only exists in the table, and not in the file, then we know + // that it's null, so just return that. + || Ok(new_null_array(field.data_type(), batch_rows)), + // However, if it does exist in both, then try to cast it to the correct output + // type + |batch_idx| { + spark_cast( + ColumnarValue::Array(Arc::clone(&batch_cols[batch_idx])), + field.data_type(), + &self.cast_options, + )? + .into_array(batch_rows) + }, + ) + }) + .collect::, _>>()?; + + // Necessary to handle empty batches + let options = RecordBatchOptions::new().with_row_count(Some(batch.num_rows())); + + let schema = Arc::::clone(&self.required_schema); + let record_batch = RecordBatch::try_new_with_options(schema, cols, &options)?; + Ok(record_batch) + } + + /// Adapts a [`RecordBatch`]'s schema into one that has all the correct output types and only + /// contains the fields that exist in both the file schema and table schema. + /// + /// Unlike `map_batch` this method also preserves the columns that + /// may not appear in the final output (`projected_table_schema`) but may + /// appear in push down predicates + fn map_partial_batch(&self, batch: RecordBatch) -> datafusion_common::Result { + let batch_cols = batch.columns().to_vec(); + let schema = batch.schema(); + + // for each field in the batch's schema (which is based on a file, not a table)... + let (cols, fields) = schema + .fields() + .iter() + .zip(batch_cols.iter()) + .flat_map(|(field, batch_col)| { + self.table_schema + // try to get the same field from the table schema that we have stored in self + .field_with_name(field.name()) + // and if we don't have it, that's fine, ignore it. This may occur when we've + // created an external table whose fields are a subset of the fields in this + // file, then tried to read data from the file into this table. If that is the + // case here, it's fine to ignore because we don't care about this field + // anyways + .ok() + // but if we do have it, + .map(|table_field| { + // try to cast it into the correct output type. we don't want to ignore this + // error, though, so it's propagated. + spark_cast( + ColumnarValue::Array(Arc::clone(batch_col)), + table_field.data_type(), + &self.cast_options, + )? + .into_array(batch_col.len()) + // and if that works, return the field and column. + .map(|new_col| (new_col, table_field.clone())) + }) + }) + .collect::, _>>()? + .into_iter() + .unzip::<_, _, Vec<_>, Vec<_>>(); + + // Necessary to handle empty batches + let options = RecordBatchOptions::new().with_row_count(Some(batch.num_rows())); + + let schema = Arc::new(Schema::new_with_metadata(fields, schema.metadata().clone())); + let record_batch = RecordBatch::try_new_with_options(schema, cols, &options)?; + Ok(record_batch) + } +} + +#[cfg(test)] +mod test { + use crate::test_common::file_util::get_temp_filename; + use crate::{EvalMode, SparkCastOptions, SparkSchemaAdapterFactory}; + use arrow::array::{Int32Array, StringArray}; + use arrow::datatypes::{DataType, Field, Schema}; + use arrow::record_batch::RecordBatch; + use arrow_array::UInt32Array; + use arrow_schema::SchemaRef; + use datafusion::datasource::listing::PartitionedFile; + use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; + use datafusion::execution::object_store::ObjectStoreUrl; + use datafusion::execution::TaskContext; + use datafusion::physical_plan::ExecutionPlan; + use datafusion_common::DataFusionError; + use futures::StreamExt; + use parquet::arrow::ArrowWriter; + use std::fs::File; + use std::sync::Arc; + + #[tokio::test] + async fn parquet_roundtrip_int_as_string() -> Result<(), DataFusionError> { + let file_schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, false), + ])); + + let ids = Arc::new(Int32Array::from(vec![1, 2, 3])) as Arc; + let names = Arc::new(StringArray::from(vec!["Alice", "Bob", "Charlie"])) + as Arc; + let batch = RecordBatch::try_new(Arc::clone(&file_schema), vec![ids, names])?; + + let required_schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Utf8, false), + Field::new("name", DataType::Utf8, false), + ])); + + let _ = roundtrip(&batch, required_schema).await?; + + Ok(()) + } + + #[tokio::test] + async fn parquet_roundtrip_unsigned_int() -> Result<(), DataFusionError> { + let file_schema = Arc::new(Schema::new(vec![Field::new("id", DataType::UInt32, false)])); + + let ids = Arc::new(UInt32Array::from(vec![1, 2, 3])) as Arc; + let batch = RecordBatch::try_new(Arc::clone(&file_schema), vec![ids])?; + + let required_schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, false)])); + + let _ = roundtrip(&batch, required_schema).await?; + + Ok(()) + } + + /// Create a Parquet file containing a single batch and then read the batch back using + /// the specified required_schema. This will cause the SchemaAdapter code to be used. + async fn roundtrip( + batch: &RecordBatch, + required_schema: SchemaRef, + ) -> Result { + let filename = get_temp_filename(); + let filename = filename.as_path().as_os_str().to_str().unwrap().to_string(); + let file = File::create(&filename)?; + let mut writer = ArrowWriter::try_new(file, Arc::clone(&batch.schema()), None)?; + writer.write(batch)?; + writer.close()?; + + let object_store_url = ObjectStoreUrl::local_filesystem(); + let file_scan_config = FileScanConfig::new(object_store_url, required_schema) + .with_file_groups(vec![vec![PartitionedFile::from_path( + filename.to_string(), + )?]]); + + let mut spark_cast_options = SparkCastOptions::new(EvalMode::Legacy, "UTC", false); + spark_cast_options.allow_cast_unsigned_ints = true; + + let parquet_exec = ParquetExec::builder(file_scan_config) + .with_schema_adapter_factory(Arc::new(SparkSchemaAdapterFactory::new( + spark_cast_options, + ))) + .build(); + + let mut stream = parquet_exec + .execute(0, Arc::new(TaskContext::default())) + .unwrap(); + stream.next().await.unwrap() + } +} diff --git a/native/core/src/parquet/util/test_common/file_util.rs b/native/spark-expr/src/test_common/file_util.rs similarity index 100% rename from native/core/src/parquet/util/test_common/file_util.rs rename to native/spark-expr/src/test_common/file_util.rs diff --git a/native/spark-expr/src/test_common/mod.rs b/native/spark-expr/src/test_common/mod.rs new file mode 100644 index 0000000000..efd25a4a2a --- /dev/null +++ b/native/spark-expr/src/test_common/mod.rs @@ -0,0 +1,17 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +pub mod file_util; From 46a28db5e5014fc7fabe8eafc7dbeb5027974c47 Mon Sep 17 00:00:00 2001 From: Parth Chandra Date: Tue, 17 Dec 2024 15:50:36 -0800 Subject: [PATCH 37/83] fix: Document enabling comet explain plan usage in Spark (4.0) (#1176) --- docs/source/user-guide/tuning.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/docs/source/user-guide/tuning.md b/docs/source/user-guide/tuning.md index af722494f9..f10a0dde9e 100644 --- a/docs/source/user-guide/tuning.md +++ b/docs/source/user-guide/tuning.md @@ -127,3 +127,13 @@ Here is a guide to some of the native metrics. | `elapsed_compute` | Total time spent in this operator, fetching batches from a JVM iterator. | | `jvm_fetch_time` | Time spent in the JVM fetching input batches to be read by this `ScanExec` instance. | | `arrow_ffi_time` | Time spent using Arrow FFI to create Arrow batches from the memory addresses returned from the JVM. | + +## Explain Plan +### Extended Explain +With Spark 4.0.0 and newer, Comet can provide extended explain plan information in the Spark UI. Currently this lists +reasons why Comet may not have been enabled for specific operations. +To enable this, in the Spark configuration, set the following: +```shell +-c spark.sql.extendedExplainProviders=org.apache.comet.ExtendedExplainInfo +``` +This will add a section to the detailed plan displayed in the Spark SQL UI page. \ No newline at end of file From 655081b66808d8f70abebd4d85af7c401c579a3a Mon Sep 17 00:00:00 2001 From: KAZUYUKI TANIMURA Date: Wed, 18 Dec 2024 09:10:34 -0800 Subject: [PATCH 38/83] test: enabling Spark tests with offHeap requirement (#1177) ## Which issue does this PR close? ## Rationale for this change After https://github.com/apache/datafusion-comet/pull/1062 We have not running Spark tests for native execution ## What changes are included in this PR? Removed the off heap requirement for testing ## How are these changes tested? Bringing back Spark tests for native execution --- dev/diffs/4.0.0-preview1.diff | 103 +++++++++++++++++- native/core/src/execution/jni_api.rs | 26 ++++- .../org/apache/comet/CometExecIterator.scala | 9 +- .../comet/CometSparkSessionExtensions.scala | 10 +- .../main/scala/org/apache/comet/Native.scala | 5 + 5 files changed, 142 insertions(+), 11 deletions(-) diff --git a/dev/diffs/4.0.0-preview1.diff b/dev/diffs/4.0.0-preview1.diff index ba68d2a7bd..db62ed607e 100644 --- a/dev/diffs/4.0.0-preview1.diff +++ b/dev/diffs/4.0.0-preview1.diff @@ -146,6 +146,77 @@ index 698ca009b4f..57d774a3617 100644 -- Test tables CREATE table explain_temp1 (key int, val int) USING PARQUET; +diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql +index 3a409eea348..26e9aaf215c 100644 +--- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql ++++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql +@@ -6,6 +6,9 @@ + -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/int4.sql + -- + ++-- TODO: https://github.com/apache/datafusion-comet/issues/551 ++--SET spark.comet.enabled = false ++ + CREATE TABLE INT4_TBL(f1 int) USING parquet; + + -- [SPARK-28023] Trim the string when cast string type to other types +diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql +index fac23b4a26f..98b12ae5ccc 100644 +--- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql ++++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql +@@ -6,6 +6,10 @@ + -- Test int8 64-bit integers. + -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/int8.sql + -- ++ ++-- TODO: https://github.com/apache/datafusion-comet/issues/551 ++--SET spark.comet.enabled = false ++ + CREATE TABLE INT8_TBL(q1 bigint, q2 bigint) USING parquet; + + -- PostgreSQL implicitly casts string literals to data with integral types, but +diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/select_having.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/select_having.sql +index 0efe0877e9b..f9df0400c99 100644 +--- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/select_having.sql ++++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/select_having.sql +@@ -6,6 +6,9 @@ + -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_having.sql + -- + ++-- TODO: https://github.com/apache/datafusion-comet/issues/551 ++--SET spark.comet.enabled = false ++ + -- load test data + CREATE TABLE test_having (a int, b int, c string, d string) USING parquet; + INSERT INTO test_having VALUES (0, 1, 'XXXX', 'A'); +diff --git a/sql/core/src/test/resources/sql-tests/inputs/view-schema-binding-config.sql b/sql/core/src/test/resources/sql-tests/inputs/view-schema-binding-config.sql +index e803254ea64..74db78aee38 100644 +--- a/sql/core/src/test/resources/sql-tests/inputs/view-schema-binding-config.sql ++++ b/sql/core/src/test/resources/sql-tests/inputs/view-schema-binding-config.sql +@@ -1,6 +1,9 @@ + -- This test suits check the spark.sql.viewSchemaBindingMode configuration. + -- It can be DISABLED and COMPENSATION + ++-- TODO: https://github.com/apache/datafusion-comet/issues/551 ++--SET spark.comet.enabled = false ++ + -- Verify the default binding is true + SET spark.sql.legacy.viewSchemaBindingMode; + +diff --git a/sql/core/src/test/resources/sql-tests/inputs/view-schema-compensation.sql b/sql/core/src/test/resources/sql-tests/inputs/view-schema-compensation.sql +index 21a3ce1e122..f4762ab98f0 100644 +--- a/sql/core/src/test/resources/sql-tests/inputs/view-schema-compensation.sql ++++ b/sql/core/src/test/resources/sql-tests/inputs/view-schema-compensation.sql +@@ -1,5 +1,9 @@ + -- This test suite checks the WITH SCHEMA COMPENSATION clause + -- Disable ANSI mode to ensure we are forcing it explicitly in the CASTS ++ ++-- TODO: https://github.com/apache/datafusion-comet/issues/551 ++--SET spark.comet.enabled = false ++ + SET spark.sql.ansi.enabled = false; + + -- In COMPENSATION views get invalidated if the type can't cast diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index d023fb82185..0f4f03bda6c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -917,7 +988,7 @@ index 34c6c49bc49..f5dea07a213 100644 protected val baseResourcePath = { // use the same way as `SQLQueryTestSuite` to get the resource path diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala -index 56c364e2084..a00a50e020a 100644 +index 56c364e2084..fc3abd7cdc4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1510,7 +1510,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark @@ -930,6 +1001,36 @@ index 56c364e2084..a00a50e020a 100644 AccumulatorSuite.verifyPeakExecutionMemorySet(sparkContext, "external sort") { sql("SELECT * FROM testData2 ORDER BY a ASC, b ASC").collect() } +@@ -4454,7 +4455,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark + } + + test("SPARK-39166: Query context of binary arithmetic should be serialized to executors" + +- " when WSCG is off") { ++ " when WSCG is off", ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false", + SQLConf.ANSI_ENABLED.key -> "true") { + withTable("t") { +@@ -4475,7 +4477,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark + } + + test("SPARK-39175: Query context of Cast should be serialized to executors" + +- " when WSCG is off") { ++ " when WSCG is off", ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false", + SQLConf.ANSI_ENABLED.key -> "true") { + withTable("t") { +@@ -4502,7 +4505,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark + } + + test("SPARK-39190,SPARK-39208,SPARK-39210: Query context of decimal overflow error should " + +- "be serialized to executors when WSCG is off") { ++ "be serialized to executors when WSCG is off", ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false", + SQLConf.ANSI_ENABLED.key -> "true") { + withTable("t") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 68f14f13bbd..174636cefb5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index 491b389c99..eb73675b5b 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -106,6 +106,9 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( metrics_node: JObject, comet_task_memory_manager_obj: JObject, batch_size: jint, + use_unified_memory_manager: jboolean, + memory_limit: jlong, + memory_fraction: jdouble, debug_native: jboolean, explain_native: jboolean, worker_threads: jint, @@ -147,7 +150,13 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( // We need to keep the session context alive. Some session state like temporary // dictionaries are stored in session context. If it is dropped, the temporary // dictionaries will be dropped as well. - let session = prepare_datafusion_session_context(batch_size as usize, task_memory_manager)?; + let session = prepare_datafusion_session_context( + batch_size as usize, + use_unified_memory_manager == 1, + memory_limit as usize, + memory_fraction, + task_memory_manager, + )?; let plan_creation_time = start.elapsed(); @@ -174,13 +183,22 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( /// Configure DataFusion session context. fn prepare_datafusion_session_context( batch_size: usize, + use_unified_memory_manager: bool, + memory_limit: usize, + memory_fraction: f64, comet_task_memory_manager: Arc, ) -> CometResult { let mut rt_config = RuntimeConfig::new().with_disk_manager(DiskManagerConfig::NewOs); - // Set Comet memory pool for native - let memory_pool = CometMemoryPool::new(comet_task_memory_manager); - rt_config = rt_config.with_memory_pool(Arc::new(memory_pool)); + // Check if we are using unified memory manager integrated with Spark. + if use_unified_memory_manager { + // Set Comet memory pool for native + let memory_pool = CometMemoryPool::new(comet_task_memory_manager); + rt_config = rt_config.with_memory_pool(Arc::new(memory_pool)); + } else { + // Use the memory pool from DF + rt_config = rt_config.with_memory_limit(memory_limit, memory_fraction) + } // Get Datafusion configuration from Spark Execution context // can be configured in Comet Spark JVM using Spark --conf parameters diff --git a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala index d57e9e2b87..04d9306951 100644 --- a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala +++ b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala @@ -23,7 +23,7 @@ import org.apache.spark._ import org.apache.spark.sql.comet.CometMetricNode import org.apache.spark.sql.vectorized._ -import org.apache.comet.CometConf.{COMET_BATCH_SIZE, COMET_BLOCKING_THREADS, COMET_DEBUG_ENABLED, COMET_EXPLAIN_NATIVE_ENABLED, COMET_WORKER_THREADS} +import org.apache.comet.CometConf.{COMET_BATCH_SIZE, COMET_BLOCKING_THREADS, COMET_DEBUG_ENABLED, COMET_EXEC_MEMORY_FRACTION, COMET_EXPLAIN_NATIVE_ENABLED, COMET_WORKER_THREADS} import org.apache.comet.vector.NativeUtil /** @@ -60,6 +60,10 @@ class CometExecIterator( new CometBatchIterator(iterator, nativeUtil) }.toArray private val plan = { + val conf = SparkEnv.get.conf + // Only enable unified memory manager when off-heap mode is enabled. Otherwise, + // we'll use the built-in memory pool from DF, and initializes with `memory_limit` + // and `memory_fraction` below. nativeLib.createPlan( id, cometBatchIterators, @@ -67,6 +71,9 @@ class CometExecIterator( nativeMetrics, new CometTaskMemoryManager(id), batchSize = COMET_BATCH_SIZE.get(), + use_unified_memory_manager = conf.getBoolean("spark.memory.offHeap.enabled", false), + memory_limit = CometSparkSessionExtensions.getCometMemoryOverhead(conf), + memory_fraction = COMET_EXEC_MEMORY_FRACTION.get(), debug = COMET_DEBUG_ENABLED.get(), explain = COMET_EXPLAIN_NATIVE_ENABLED.get(), workerThreads = COMET_WORKER_THREADS.get(), diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index 61c45daff0..8bff6b5fbd 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -53,7 +53,7 @@ import org.apache.spark.sql.types.{DoubleType, FloatType} import org.apache.comet.CometConf._ import org.apache.comet.CometExplainInfo.getActualPlan -import org.apache.comet.CometSparkSessionExtensions.{createMessage, getCometBroadcastNotEnabledReason, getCometShuffleNotEnabledReason, isANSIEnabled, isCometBroadCastForceEnabled, isCometEnabled, isCometExecEnabled, isCometJVMShuffleMode, isCometNativeShuffleMode, isCometScan, isCometScanEnabled, isCometShuffleEnabled, isSpark34Plus, isSpark40Plus, shouldApplySparkToColumnar, withInfo, withInfos} +import org.apache.comet.CometSparkSessionExtensions.{createMessage, getCometBroadcastNotEnabledReason, getCometShuffleNotEnabledReason, isANSIEnabled, isCometBroadCastForceEnabled, isCometEnabled, isCometExecEnabled, isCometJVMShuffleMode, isCometNativeShuffleMode, isCometScan, isCometScanEnabled, isCometShuffleEnabled, isOffHeapEnabled, isSpark34Plus, isSpark40Plus, isTesting, shouldApplySparkToColumnar, withInfo, withInfos} import org.apache.comet.parquet.{CometParquetScan, SupportsComet} import org.apache.comet.rules.RewriteJoin import org.apache.comet.serde.OperatorOuterClass.Operator @@ -921,8 +921,9 @@ class CometSparkSessionExtensions override def apply(plan: SparkPlan): SparkPlan = { // Comet required off-heap memory to be enabled - if ("true" != conf.getConfString("spark.memory.offHeap.enabled", "false")) { - logInfo("Comet extension disabled because spark.memory.offHeap.enabled=false") + if (!isOffHeapEnabled(conf) && !isTesting) { + logWarning("Comet native exec disabled because spark.memory.offHeap.enabled=false") + withInfo(plan, "Comet native exec disabled because spark.memory.offHeap.enabled=false") return plan } @@ -1174,8 +1175,7 @@ object CometSparkSessionExtensions extends Logging { } private[comet] def isOffHeapEnabled(conf: SQLConf): Boolean = - conf.contains("spark.memory.offHeap.enabled") && - conf.getConfString("spark.memory.offHeap.enabled").toBoolean + conf.getConfString("spark.memory.offHeap.enabled", "false").toBoolean // Copied from org.apache.spark.util.Utils which is private to Spark. private[comet] def isTesting: Boolean = { diff --git a/spark/src/main/scala/org/apache/comet/Native.scala b/spark/src/main/scala/org/apache/comet/Native.scala index 64ada91adc..083c0f2b50 100644 --- a/spark/src/main/scala/org/apache/comet/Native.scala +++ b/spark/src/main/scala/org/apache/comet/Native.scala @@ -43,6 +43,7 @@ class Native extends NativeBase { * @return * the address to native query plan. */ + // scalastyle:off @native def createPlan( id: Long, iterators: Array[CometBatchIterator], @@ -50,10 +51,14 @@ class Native extends NativeBase { metrics: CometMetricNode, taskMemoryManager: CometTaskMemoryManager, batchSize: Int, + use_unified_memory_manager: Boolean, + memory_limit: Long, + memory_fraction: Double, debug: Boolean, explain: Boolean, workerThreads: Int, blockingThreads: Int): Long + // scalastyle:on /** * Execute a native query plan based on given input Arrow arrays. From e297d23bd38bc306c90ed21a154d1495f985683e Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 18 Dec 2024 10:50:07 -0700 Subject: [PATCH 39/83] feat: Improve shuffle metrics (second attempt) (#1175) * improve shuffle metrics * docs * more metrics * refactor * address feedback --- docs/source/index.rst | 1 + docs/source/user-guide/metrics.md | 66 +++++ docs/source/user-guide/tuning.md | 25 -- native/core/src/execution/shuffle/row.rs | 6 +- .../src/execution/shuffle/shuffle_writer.rs | 271 +++++++++++------- .../sql/comet/CometCollectLimitExec.scala | 3 +- .../spark/sql/comet/CometMetricNode.scala | 11 + .../CometTakeOrderedAndProjectExec.scala | 3 +- .../shuffle/CometShuffleExchangeExec.scala | 15 +- 9 files changed, 261 insertions(+), 140 deletions(-) create mode 100644 docs/source/user-guide/metrics.md diff --git a/docs/source/index.rst b/docs/source/index.rst index 39ad27a57c..21ec36ca95 100644 --- a/docs/source/index.rst +++ b/docs/source/index.rst @@ -51,6 +51,7 @@ as a native runtime to achieve improvement in terms of query efficiency and quer Configuration Settings Compatibility Guide Tuning Guide + Metrics Guide .. _toc.contributor-guide-links: .. toctree:: diff --git a/docs/source/user-guide/metrics.md b/docs/source/user-guide/metrics.md new file mode 100644 index 0000000000..509d0ae8c0 --- /dev/null +++ b/docs/source/user-guide/metrics.md @@ -0,0 +1,66 @@ + + +# Comet Metrics + +## Spark SQL Metrics + +Set `spark.comet.metrics.detailed=true` to see all available Comet metrics. + +### CometScanExec + +| Metric | Description | +| ----------- | ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| `scan time` | Total time to scan a Parquet file. This is not comparable to the same metric in Spark because Comet's scan metric is more accurate. Although both Comet and Spark measure the time in nanoseconds, Spark rounds this time to the nearest millisecond per batch and Comet does not. | + +### Exchange + +Comet adds some additional metrics: + +| Metric | Description | +| ------------------------------- | ------------------------------------------------------------- | +| `native shuffle time` | Total time in native code excluding any child operators. | +| `repartition time` | Time to repartition batches. | +| `memory pool time` | Time interacting with memory pool. | +| `encoding and compression time` | Time to encode batches in IPC format and compress using ZSTD. | + +## Native Metrics + +Setting `spark.comet.explain.native.enabled=true` will cause native plans to be logged in each executor. Metrics are +logged for each native plan (and there is one plan per task, so this is very verbose). + +Here is a guide to some of the native metrics. + +### ScanExec + +| Metric | Description | +| ----------------- | --------------------------------------------------------------------------------------------------- | +| `elapsed_compute` | Total time spent in this operator, fetching batches from a JVM iterator. | +| `jvm_fetch_time` | Time spent in the JVM fetching input batches to be read by this `ScanExec` instance. | +| `arrow_ffi_time` | Time spent using Arrow FFI to create Arrow batches from the memory addresses returned from the JVM. | + +### ShuffleWriterExec + +| Metric | Description | +| ----------------- | ------------------------------------------------------------- | +| `elapsed_compute` | Total time excluding any child operators. | +| `repart_time` | Time to repartition batches. | +| `ipc_time` | Time to encode batches in IPC format and compress using ZSTD. | +| `mempool_time` | Time interacting with memory pool. | +| `write_time` | Time spent writing bytes to disk. | diff --git a/docs/source/user-guide/tuning.md b/docs/source/user-guide/tuning.md index f10a0dde9e..d68481d172 100644 --- a/docs/source/user-guide/tuning.md +++ b/docs/source/user-guide/tuning.md @@ -103,31 +103,6 @@ native shuffle currently only supports `HashPartitioning` and `SinglePartitionin To enable native shuffle, set `spark.comet.exec.shuffle.mode` to `native`. If this mode is explicitly set, then any shuffle operations that cannot be supported in this mode will fall back to Spark. -## Metrics - -### Spark SQL Metrics - -Some Comet metrics are not directly comparable to Spark metrics in some cases: - -- `CometScanExec` uses nanoseconds for total scan time. Spark also measures scan time in nanoseconds but converts to - milliseconds _per batch_ which can result in a large loss of precision, making it difficult to compare scan times - between Spark and Comet. - -### Native Metrics - -Setting `spark.comet.explain.native.enabled=true` will cause native plans to be logged in each executor. Metrics are -logged for each native plan (and there is one plan per task, so this is very verbose). - -Here is a guide to some of the native metrics. - -### ScanExec - -| Metric | Description | -| ----------------- | --------------------------------------------------------------------------------------------------- | -| `elapsed_compute` | Total time spent in this operator, fetching batches from a JVM iterator. | -| `jvm_fetch_time` | Time spent in the JVM fetching input batches to be read by this `ScanExec` instance. | -| `arrow_ffi_time` | Time spent using Arrow FFI to create Arrow batches from the memory addresses returned from the JVM. | - ## Explain Plan ### Extended Explain With Spark 4.0.0 and newer, Comet can provide extended explain plan information in the Spark UI. Currently this lists diff --git a/native/core/src/execution/shuffle/row.rs b/native/core/src/execution/shuffle/row.rs index ce752e68af..ecab77d966 100644 --- a/native/core/src/execution/shuffle/row.rs +++ b/native/core/src/execution/shuffle/row.rs @@ -40,6 +40,7 @@ use arrow_array::{ Array, ArrayRef, RecordBatch, RecordBatchOptions, }; use arrow_schema::{ArrowError, DataType, Field, Schema, TimeUnit}; +use datafusion::physical_plan::metrics::Time; use jni::sys::{jint, jlong}; use std::{ fs::OpenOptions, @@ -3354,7 +3355,10 @@ pub fn process_sorted_row_partition( let mut frozen: Vec = vec![]; let mut cursor = Cursor::new(&mut frozen); cursor.seek(SeekFrom::End(0))?; - written += write_ipc_compressed(&batch, &mut cursor)?; + + // we do not collect metrics in Native_writeSortedFileNative + let ipc_time = Time::default(); + written += write_ipc_compressed(&batch, &mut cursor, &ipc_time)?; if let Some(checksum) = &mut current_checksum { checksum.update(&mut cursor)?; diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 7587ff06dc..fcc8c51f60 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -17,22 +17,14 @@ //! Defines the External shuffle repartition plan. -use std::{ - any::Any, - fmt, - fmt::{Debug, Formatter}, - fs::{File, OpenOptions}, - io::{BufReader, BufWriter, Cursor, Read, Seek, SeekFrom, Write}, - path::Path, - sync::Arc, - task::{Context, Poll}, +use crate::{ + common::bit::ceil, + errors::{CometError, CometResult}, }; - use arrow::{datatypes::*, ipc::writer::StreamWriter}; use async_trait::async_trait; use bytes::Buf; use crc32fast::Hasher; -use datafusion::physical_plan::metrics::Time; use datafusion::{ arrow::{ array::*, @@ -48,23 +40,32 @@ use datafusion::{ runtime_env::RuntimeEnv, }, physical_plan::{ - metrics::{BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}, + metrics::{ + BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, Time, + }, stream::RecordBatchStreamAdapter, DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }, }; +use datafusion_comet_spark_expr::spark_hash::create_murmur3_hashes; use datafusion_physical_expr::EquivalenceProperties; use futures::executor::block_on; use futures::{lock::Mutex, Stream, StreamExt, TryFutureExt, TryStreamExt}; use itertools::Itertools; use simd_adler32::Adler32; - -use crate::{ - common::bit::ceil, - errors::{CometError, CometResult}, +use std::io::Error; +use std::{ + any::Any, + fmt, + fmt::{Debug, Formatter}, + fs::{File, OpenOptions}, + io::{BufReader, BufWriter, Cursor, Read, Seek, SeekFrom, Write}, + path::Path, + sync::Arc, + task::{Context, Poll}, }; -use datafusion_comet_spark_expr::spark_hash::create_murmur3_hashes; +use tokio::time::Instant; /// The status of appending rows to a partition buffer. enum AppendRowStatus { @@ -242,7 +243,7 @@ impl PartitionBuffer { /// Initializes active builders if necessary. /// Returns error if memory reservation fails. - fn init_active_if_necessary(&mut self) -> Result { + fn init_active_if_necessary(&mut self, metrics: &ShuffleRepartitionerMetrics) -> Result { let mut mem_diff = 0; if self.active.is_empty() { @@ -256,9 +257,13 @@ impl PartitionBuffer { .sum::(); } + let mut mempool_timer = metrics.mempool_time.timer(); self.reservation.try_grow(self.active_slots_mem_size)?; + mempool_timer.stop(); + let mut repart_timer = metrics.repart_time.timer(); self.active = new_array_builders(&self.schema, self.batch_size); + repart_timer.stop(); mem_diff += self.active_slots_mem_size as isize; } @@ -271,13 +276,13 @@ impl PartitionBuffer { columns: &[ArrayRef], indices: &[usize], start_index: usize, - time_metric: &Time, + metrics: &ShuffleRepartitionerMetrics, ) -> AppendRowStatus { let mut mem_diff = 0; let mut start = start_index; // lazy init because some partition may be empty - let init = self.init_active_if_necessary(); + let init = self.init_active_if_necessary(metrics); if init.is_err() { return AppendRowStatus::StartIndex(start); } @@ -285,6 +290,8 @@ impl PartitionBuffer { while start < indices.len() { let end = (start + self.batch_size).min(indices.len()); + + let mut repart_timer = metrics.repart_time.timer(); self.active .iter_mut() .zip(columns) @@ -292,16 +299,16 @@ impl PartitionBuffer { append_columns(builder, column, &indices[start..end], column.data_type()); }); self.num_active_rows += end - start; + repart_timer.stop(); + if self.num_active_rows >= self.batch_size { - let mut timer = time_metric.timer(); - let flush = self.flush(); + let flush = self.flush(&metrics.ipc_time); if let Err(e) = flush { return AppendRowStatus::MemDiff(Err(e)); } mem_diff += flush.unwrap(); - timer.stop(); - let init = self.init_active_if_necessary(); + let init = self.init_active_if_necessary(metrics); if init.is_err() { return AppendRowStatus::StartIndex(end); } @@ -313,7 +320,7 @@ impl PartitionBuffer { } /// flush active data into frozen bytes - fn flush(&mut self) -> Result { + fn flush(&mut self, ipc_time: &Time) -> Result { if self.num_active_rows == 0 { return Ok(0); } @@ -330,7 +337,7 @@ impl PartitionBuffer { let frozen_capacity_old = self.frozen.capacity(); let mut cursor = Cursor::new(&mut self.frozen); cursor.seek(SeekFrom::End(0))?; - write_ipc_compressed(&frozen_batch, &mut cursor)?; + write_ipc_compressed(&frozen_batch, &mut cursor, ipc_time)?; mem_diff += (self.frozen.capacity() - frozen_capacity_old) as isize; Ok(mem_diff) @@ -628,6 +635,21 @@ struct ShuffleRepartitionerMetrics { /// metrics baseline: BaselineMetrics, + /// Time to perform repartitioning + repart_time: Time, + + /// Time interacting with memory pool + mempool_time: Time, + + /// Time encoding batches to IPC format + ipc_time: Time, + + /// Time spent writing to disk. Maps to "shuffleWriteTime" in Spark SQL Metrics. + write_time: Time, + + /// Number of input batches + input_batches: Count, + /// count of spills during the execution of the operator spill_count: Count, @@ -642,6 +664,11 @@ impl ShuffleRepartitionerMetrics { fn new(metrics: &ExecutionPlanMetricsSet, partition: usize) -> Self { Self { baseline: BaselineMetrics::new(metrics, partition), + repart_time: MetricBuilder::new(metrics).subset_time("repart_time", partition), + mempool_time: MetricBuilder::new(metrics).subset_time("mempool_time", partition), + ipc_time: MetricBuilder::new(metrics).subset_time("ipc_time", partition), + write_time: MetricBuilder::new(metrics).subset_time("write_time", partition), + input_batches: MetricBuilder::new(metrics).counter("input_batches", partition), spill_count: MetricBuilder::new(metrics).spill_count(partition), spilled_bytes: MetricBuilder::new(metrics).spilled_bytes(partition), data_size: MetricBuilder::new(metrics).counter("data_size", partition), @@ -701,6 +728,7 @@ impl ShuffleRepartitioner { /// This function will slice input batch according to configured batch size and then /// shuffle rows into corresponding partition buffer. async fn insert_batch(&mut self, batch: RecordBatch) -> Result<()> { + let start_time = Instant::now(); let mut start = 0; while start < batch.num_rows() { let end = (start + self.batch_size).min(batch.num_rows()); @@ -708,6 +736,11 @@ impl ShuffleRepartitioner { self.partitioning_batch(batch).await?; start = end; } + self.metrics.input_batches.add(1); + self.metrics + .baseline + .elapsed_compute() + .add_duration(start_time.elapsed()); Ok(()) } @@ -738,53 +771,61 @@ impl ShuffleRepartitioner { let num_output_partitions = self.num_output_partitions; match &self.partitioning { Partitioning::Hash(exprs, _) => { - let arrays = exprs - .iter() - .map(|expr| expr.evaluate(&input)?.into_array(input.num_rows())) - .collect::>>()?; - - // use identical seed as spark hash partition - let hashes_buf = &mut self.hashes_buf[..arrays[0].len()]; - hashes_buf.fill(42_u32); - - // Hash arrays and compute buckets based on number of partitions - let partition_ids = &mut self.partition_ids[..arrays[0].len()]; - create_murmur3_hashes(&arrays, hashes_buf)? - .iter() - .enumerate() - .for_each(|(idx, hash)| { - partition_ids[idx] = pmod(*hash, num_output_partitions) as u64 + let (partition_starts, shuffled_partition_ids): (Vec, Vec) = { + let mut timer = self.metrics.repart_time.timer(); + let arrays = exprs + .iter() + .map(|expr| expr.evaluate(&input)?.into_array(input.num_rows())) + .collect::>>()?; + + // use identical seed as spark hash partition + let hashes_buf = &mut self.hashes_buf[..arrays[0].len()]; + hashes_buf.fill(42_u32); + + // Hash arrays and compute buckets based on number of partitions + let partition_ids = &mut self.partition_ids[..arrays[0].len()]; + create_murmur3_hashes(&arrays, hashes_buf)? + .iter() + .enumerate() + .for_each(|(idx, hash)| { + partition_ids[idx] = pmod(*hash, num_output_partitions) as u64 + }); + + // count each partition size + let mut partition_counters = vec![0usize; num_output_partitions]; + partition_ids + .iter() + .for_each(|partition_id| partition_counters[*partition_id as usize] += 1); + + // accumulate partition counters into partition ends + // e.g. partition counter: [1, 3, 2, 1] => [1, 4, 6, 7] + let mut partition_ends = partition_counters; + let mut accum = 0; + partition_ends.iter_mut().for_each(|v| { + *v += accum; + accum = *v; }); - // count each partition size - let mut partition_counters = vec![0usize; num_output_partitions]; - partition_ids - .iter() - .for_each(|partition_id| partition_counters[*partition_id as usize] += 1); - - // accumulate partition counters into partition ends - // e.g. partition counter: [1, 3, 2, 1] => [1, 4, 6, 7] - let mut partition_ends = partition_counters; - let mut accum = 0; - partition_ends.iter_mut().for_each(|v| { - *v += accum; - accum = *v; - }); - - // calculate shuffled partition ids - // e.g. partition ids: [3, 1, 1, 1, 2, 2, 0] => [6, 1, 2, 3, 4, 5, 0] which is the - // row indices for rows ordered by their partition id. For example, first partition - // 0 has one row index [6], partition 1 has row indices [1, 2, 3], etc. - let mut shuffled_partition_ids = vec![0usize; input.num_rows()]; - for (index, partition_id) in partition_ids.iter().enumerate().rev() { - partition_ends[*partition_id as usize] -= 1; - let end = partition_ends[*partition_id as usize]; - shuffled_partition_ids[end] = index; - } + // calculate shuffled partition ids + // e.g. partition ids: [3, 1, 1, 1, 2, 2, 0] => [6, 1, 2, 3, 4, 5, 0] which is the + // row indices for rows ordered by their partition id. For example, first partition + // 0 has one row index [6], partition 1 has row indices [1, 2, 3], etc. + let mut shuffled_partition_ids = vec![0usize; input.num_rows()]; + for (index, partition_id) in partition_ids.iter().enumerate().rev() { + partition_ends[*partition_id as usize] -= 1; + let end = partition_ends[*partition_id as usize]; + shuffled_partition_ids[end] = index; + } - // after calculating, partition ends become partition starts - let mut partition_starts = partition_ends; - partition_starts.push(input.num_rows()); + // after calculating, partition ends become partition starts + let mut partition_starts = partition_ends; + partition_starts.push(input.num_rows()); + timer.stop(); + Ok::<(Vec, Vec), DataFusionError>(( + partition_starts, + shuffled_partition_ids, + )) + }?; // For each interval of row indices of partition, taking rows from input batch and // appending into output buffer. @@ -804,11 +845,20 @@ impl ShuffleRepartitioner { if mem_diff > 0 { let mem_increase = mem_diff as usize; - if self.reservation.try_grow(mem_increase).is_err() { + + let try_grow = { + let mut mempool_timer = self.metrics.mempool_time.timer(); + let result = self.reservation.try_grow(mem_increase); + mempool_timer.stop(); + result + }; + + if try_grow.is_err() { self.spill().await?; + let mut mempool_timer = self.metrics.mempool_time.timer(); self.reservation.free(); self.reservation.try_grow(mem_increase)?; - + mempool_timer.stop(); mem_diff = 0; } } @@ -816,7 +866,9 @@ impl ShuffleRepartitioner { if mem_diff < 0 { let mem_used = self.reservation.size(); let mem_decrease = mem_used.min(-mem_diff as usize); + let mut mempool_timer = self.metrics.mempool_time.timer(); self.reservation.shrink(mem_decrease); + mempool_timer.stop(); } } } @@ -848,12 +900,13 @@ impl ShuffleRepartitioner { /// Writes buffered shuffled record batches into Arrow IPC bytes. async fn shuffle_write(&mut self) -> Result { + let mut elapsed_compute = self.metrics.baseline.elapsed_compute().timer(); let num_output_partitions = self.num_output_partitions; let buffered_partitions = &mut self.buffered_partitions; let mut output_batches: Vec> = vec![vec![]; num_output_partitions]; - + let mut offsets = vec![0; num_output_partitions + 1]; for i in 0..num_output_partitions { - buffered_partitions[i].flush()?; + buffered_partitions[i].flush(&self.metrics.ipc_time)?; output_batches[i] = std::mem::take(&mut buffered_partitions[i].frozen); } @@ -863,53 +916,38 @@ impl ShuffleRepartitioner { let data_file = self.output_data_file.clone(); let index_file = self.output_index_file.clone(); - let mut offsets = vec![0; num_output_partitions + 1]; - let mut output_data = OpenOptions::new() + let mut write_time = self.metrics.write_time.timer(); + + let output_data = OpenOptions::new() .write(true) .create(true) .truncate(true) .open(data_file) .map_err(|e| DataFusionError::Execution(format!("shuffle write error: {:?}", e)))?; - for i in 0..num_output_partitions { - let mut timer = self.metrics.baseline.elapsed_compute().timer(); + let mut output_data = BufWriter::new(output_data); + for i in 0..num_output_partitions { offsets[i] = output_data.stream_position()?; output_data.write_all(&output_batches[i])?; - - timer.stop(); - output_batches[i].clear(); // append partition in each spills for spill in &output_spills { let length = spill.offsets[i + 1] - spill.offsets[i]; if length > 0 { - let mut timer = self.metrics.baseline.elapsed_compute().timer(); - let mut spill_file = - BufReader::new(File::open(spill.file.path()).map_err(|e| { - DataFusionError::Execution(format!("shuffle write error: {:?}", e)) - })?); + BufReader::new(File::open(spill.file.path()).map_err(Self::to_df_err)?); spill_file.seek(SeekFrom::Start(spill.offsets[i]))?; - std::io::copy(&mut spill_file.take(length), &mut output_data).map_err(|e| { - DataFusionError::Execution(format!("shuffle write error: {:?}", e)) - })?; - - timer.stop(); + std::io::copy(&mut spill_file.take(length), &mut output_data) + .map_err(Self::to_df_err)?; } } } - let mut timer = self.metrics.baseline.elapsed_compute().timer(); output_data.flush()?; - timer.stop(); // add one extra offset at last to ease partition length computation - offsets[num_output_partitions] = output_data - .stream_position() - .map_err(|e| DataFusionError::Execution(format!("shuffle write error: {:?}", e)))?; - - let mut timer = self.metrics.baseline.elapsed_compute().timer(); + offsets[num_output_partitions] = output_data.stream_position().map_err(Self::to_df_err)?; let mut output_index = BufWriter::new(File::create(index_file).map_err(|e| { @@ -918,19 +956,27 @@ impl ShuffleRepartitioner { for offset in offsets { output_index .write_all(&(offset as i64).to_le_bytes()[..]) - .map_err(|e| DataFusionError::Execution(format!("shuffle write error: {:?}", e)))?; + .map_err(Self::to_df_err)?; } output_index.flush()?; - timer.stop(); + write_time.stop(); + let mut mempool_timer = self.metrics.mempool_time.timer(); let used = self.reservation.size(); self.reservation.shrink(used); + mempool_timer.stop(); + + elapsed_compute.stop(); // shuffle writer always has empty output Ok(Box::pin(EmptyStream::try_new(Arc::clone(&self.schema))?)) } + fn to_df_err(e: Error) -> DataFusionError { + DataFusionError::Execution(format!("shuffle write error: {:?}", e)) + } + fn used(&self) -> usize { self.reservation.size() } @@ -959,7 +1005,7 @@ impl ShuffleRepartitioner { return Ok(0); } - let mut timer = self.metrics.baseline.elapsed_compute().timer(); + let mut timer = self.metrics.write_time.timer(); let spillfile = self .runtime @@ -969,6 +1015,7 @@ impl ShuffleRepartitioner { &mut self.buffered_partitions, spillfile.path(), self.num_output_partitions, + &self.metrics.ipc_time, )?; timer.stop(); @@ -995,12 +1042,10 @@ impl ShuffleRepartitioner { let output = &mut self.buffered_partitions[partition_id]; - let time_metric = self.metrics.baseline.elapsed_compute(); - // If the range of indices is not big enough, just appending the rows into // active array builders instead of directly adding them as a record batch. let mut start_index: usize = 0; - let mut output_ret = output.append_rows(columns, indices, start_index, time_metric); + let mut output_ret = output.append_rows(columns, indices, start_index, &self.metrics); loop { match output_ret { @@ -1012,15 +1057,15 @@ impl ShuffleRepartitioner { // Cannot allocate enough memory for the array builders in the partition, // spill partitions and retry. self.spill().await?; - self.reservation.free(); + let mut mempool_timer = self.metrics.mempool_time.timer(); + self.reservation.free(); let output = &mut self.buffered_partitions[partition_id]; output.reservation.free(); - - let time_metric = self.metrics.baseline.elapsed_compute(); + mempool_timer.stop(); start_index = new_start; - output_ret = output.append_rows(columns, indices, start_index, time_metric); + output_ret = output.append_rows(columns, indices, start_index, &self.metrics); if let AppendRowStatus::StartIndex(new_start) = output_ret { if new_start == start_index { @@ -1045,11 +1090,12 @@ fn spill_into( buffered_partitions: &mut [PartitionBuffer], path: &Path, num_output_partitions: usize, + ipc_time: &Time, ) -> Result> { let mut output_batches: Vec> = vec![vec![]; num_output_partitions]; for i in 0..num_output_partitions { - buffered_partitions[i].flush()?; + buffered_partitions[i].flush(ipc_time)?; output_batches[i] = std::mem::take(&mut buffered_partitions[i].frozen); } let path = path.to_owned(); @@ -1485,10 +1531,13 @@ impl Checksum { pub(crate) fn write_ipc_compressed( batch: &RecordBatch, output: &mut W, + ipc_time: &Time, ) -> Result { if batch.num_rows() == 0 { return Ok(0); } + + let mut timer = ipc_time.timer(); let start_pos = output.stream_position()?; // write ipc_length placeholder @@ -1508,8 +1557,10 @@ pub(crate) fn write_ipc_compressed( // fill ipc length output.seek(SeekFrom::Start(start_pos))?; output.write_all(&ipc_length.to_le_bytes()[..])?; - output.seek(SeekFrom::Start(end_pos))?; + + timer.stop(); + Ok((end_pos - start_pos) as usize) } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala index 8ea0b17654..f75af5076b 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala @@ -57,7 +57,8 @@ case class CometCollectLimitExec( "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"), "numPartitions" -> SQLMetrics.createMetric( sparkContext, - "number of partitions")) ++ readMetrics ++ writeMetrics + "number of partitions")) ++ readMetrics ++ writeMetrics ++ CometMetricNode.shuffleMetrics( + sparkContext) private lazy val serializer: Serializer = new UnsafeRowSerializer(child.output.size, longMetric("dataSize")) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometMetricNode.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometMetricNode.scala index 47c89d9433..a26fa28c8b 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometMetricNode.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometMetricNode.scala @@ -130,6 +130,17 @@ object CometMetricNode { "spilled_rows" -> SQLMetrics.createMetric(sc, "Total spilled rows")) } + def shuffleMetrics(sc: SparkContext): Map[String, SQLMetric] = { + Map( + "elapsed_compute" -> SQLMetrics.createNanoTimingMetric(sc, "native shuffle time"), + "mempool_time" -> SQLMetrics.createNanoTimingMetric(sc, "memory pool time"), + "repart_time" -> SQLMetrics.createNanoTimingMetric(sc, "repartition time"), + "ipc_time" -> SQLMetrics.createNanoTimingMetric(sc, "encoding and compression time"), + "spill_count" -> SQLMetrics.createMetric(sc, "number of spills"), + "spilled_bytes" -> SQLMetrics.createMetric(sc, "spilled bytes"), + "input_batches" -> SQLMetrics.createMetric(sc, "number of input batches")) + } + /** * Creates a [[CometMetricNode]] from a [[CometPlan]]. */ diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala index 5582f4d687..19586628a7 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala @@ -57,7 +57,8 @@ case class CometTakeOrderedAndProjectExec( "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"), "numPartitions" -> SQLMetrics.createMetric( sparkContext, - "number of partitions")) ++ readMetrics ++ writeMetrics + "number of partitions")) ++ readMetrics ++ writeMetrics ++ CometMetricNode.shuffleMetrics( + sparkContext) private lazy val serializer: Serializer = new UnsafeRowSerializer(child.output.size, longMetric("dataSize")) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala index b1dd9ac836..0cd8a9ce67 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala @@ -79,7 +79,8 @@ case class CometShuffleExchangeExec( "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"), "numPartitions" -> SQLMetrics.createMetric( sparkContext, - "number of partitions")) ++ readMetrics ++ writeMetrics + "number of partitions")) ++ readMetrics ++ writeMetrics ++ CometMetricNode.shuffleMetrics( + sparkContext) override def nodeName: String = if (shuffleType == CometNativeShuffle) { "CometExchange" @@ -477,11 +478,21 @@ class CometShuffleWriteProcessor( // Call native shuffle write val nativePlan = getNativePlan(tempDataFilename, tempIndexFilename) + val detailedMetrics = Seq( + "elapsed_compute", + "ipc_time", + "repart_time", + "mempool_time", + "input_batches", + "spill_count", + "spilled_bytes") + // Maps native metrics to SQL metrics val nativeSQLMetrics = Map( "output_rows" -> metrics(SQLShuffleWriteMetricsReporter.SHUFFLE_RECORDS_WRITTEN), "data_size" -> metrics("dataSize"), - "elapsed_compute" -> metrics(SQLShuffleWriteMetricsReporter.SHUFFLE_WRITE_TIME)) + "write_time" -> metrics(SQLShuffleWriteMetricsReporter.SHUFFLE_WRITE_TIME)) ++ + metrics.filterKeys(detailedMetrics.contains) val nativeMetrics = CometMetricNode(nativeSQLMetrics) // Getting rid of the fake partitionId From 8f4a8a54a292d86a30fb66d3dac442bff46a1039 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 19 Dec 2024 13:06:56 -0800 Subject: [PATCH 40/83] fix: stddev_pop should not directly return 0.0 when count is 1.0 (#1184) * add test * fix * fix * fix --- native/spark-expr/src/variance.rs | 6 ++---- .../apache/comet/exec/CometAggregateSuite.scala | 17 +++++++++++++++++ 2 files changed, 19 insertions(+), 4 deletions(-) diff --git a/native/spark-expr/src/variance.rs b/native/spark-expr/src/variance.rs index 2f4d8091c2..4370d89ff4 100644 --- a/native/spark-expr/src/variance.rs +++ b/native/spark-expr/src/variance.rs @@ -245,10 +245,8 @@ impl Accumulator for VarianceAccumulator { Ok(ScalarValue::Float64(match self.count { count if count == 0.0 => None, - count if count == 1.0 => { - if let StatsType::Population = self.stats_type { - Some(0.0) - } else if self.null_on_divide_by_zero { + count if count == 1.0 && StatsType::Sample == self.stats_type => { + if self.null_on_divide_by_zero { None } else { Some(f64::NAN) diff --git a/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala index e337d36c48..9a642f12f7 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala @@ -38,6 +38,23 @@ import org.apache.comet.CometSparkSessionExtensions.isSpark34Plus class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { import testImplicits._ + test("stddev_pop should return NaN for some cases") { + withSQLConf( + CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", + CometConf.COMET_EXPR_STDDEV_ENABLED.key -> "true") { + Seq(true, false).foreach { nullOnDivideByZero => + withSQLConf("spark.sql.legacy.statisticalAggregate" -> nullOnDivideByZero.toString) { + + val data: Seq[(Float, Int)] = Seq((Float.PositiveInfinity, 1)) + withParquetTable(data, "tbl", false) { + val df = sql("SELECT stddev_pop(_1), stddev_pop(_2) FROM tbl") + checkSparkAnswer(df) + } + } + } + } + } + test("count with aggregation filter") { withSQLConf( CometConf.COMET_ENABLED.key -> "true", From ea6d20511e813a2698c47a964b3a0739e9543add Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 20 Dec 2024 11:11:14 -0700 Subject: [PATCH 41/83] feat: Make native shuffle compression configurable and respect `spark.shuffle.compress` (#1185) * Make shuffle compression codec and level configurable * remove lz4 references * docs * update comment * clippy * fix benches * clippy * clippy * disable test for miri * remove lz4 reference from proto --- .../scala/org/apache/comet/CometConf.scala | 14 ++- .../shuffle/IpcInputStreamIterator.scala | 6 +- .../execution/shuffle/ShuffleUtils.scala | 31 ++++--- docs/source/user-guide/configs.md | 3 +- docs/source/user-guide/tuning.md | 6 ++ native/core/benches/shuffle_writer.rs | 87 +++++++++++++----- native/core/src/execution/planner.rs | 17 +++- native/core/src/execution/shuffle/mod.rs | 2 +- native/core/src/execution/shuffle/row.rs | 5 +- .../src/execution/shuffle/shuffle_writer.rs | 90 +++++++++++++++---- native/proto/src/proto/operator.proto | 7 ++ .../shuffle/CometShuffleExchangeExec.scala | 14 ++- .../shuffle/CometShuffleManager.scala | 2 +- 13 files changed, 221 insertions(+), 63 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index b602d7cf1c..8815ac4eb8 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -272,13 +272,21 @@ object CometConf extends ShimCometConf { .booleanConf .createWithDefault(false) - val COMET_EXEC_SHUFFLE_CODEC: ConfigEntry[String] = conf( - s"$COMET_EXEC_CONFIG_PREFIX.shuffle.codec") + val COMET_EXEC_SHUFFLE_COMPRESSION_CODEC: ConfigEntry[String] = conf( + s"$COMET_EXEC_CONFIG_PREFIX.shuffle.compression.codec") .doc( - "The codec of Comet native shuffle used to compress shuffle data. Only zstd is supported.") + "The codec of Comet native shuffle used to compress shuffle data. Only zstd is supported. " + + "Compression can be disabled by setting spark.shuffle.compress=false.") .stringConf + .checkValues(Set("zstd")) .createWithDefault("zstd") + val COMET_EXEC_SHUFFLE_COMPRESSION_LEVEL: ConfigEntry[Int] = + conf(s"$COMET_EXEC_CONFIG_PREFIX.shuffle.compression.level") + .doc("The compression level to use when compression shuffle files.") + .intConf + .createWithDefault(1) + val COMET_COLUMNAR_SHUFFLE_ASYNC_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.columnar.shuffle.async.enabled") .doc("Whether to enable asynchronous shuffle for Arrow-based shuffle.") diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala b/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala index 281c481083..d1d5af350f 100644 --- a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala +++ b/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala @@ -110,8 +110,10 @@ case class IpcInputStreamIterator( currentLimitedInputStream = is if (decompressingNeeded) { - val zs = ShuffleUtils.compressionCodecForShuffling.compressedInputStream(is) - Channels.newChannel(zs) + ShuffleUtils.compressionCodecForShuffling match { + case Some(codec) => Channels.newChannel(codec.compressedInputStream(is)) + case _ => Channels.newChannel(is) + } } else { Channels.newChannel(is) } diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ShuffleUtils.scala b/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ShuffleUtils.scala index eea134ab5e..23b4a5ec23 100644 --- a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ShuffleUtils.scala +++ b/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ShuffleUtils.scala @@ -21,22 +21,33 @@ package org.apache.spark.sql.comet.execution.shuffle import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.IO_COMPRESSION_CODEC +import org.apache.spark.internal.config.{IO_COMPRESSION_CODEC, SHUFFLE_COMPRESS} import org.apache.spark.io.CompressionCodec -import org.apache.spark.sql.internal.SQLConf import org.apache.comet.CometConf private[spark] object ShuffleUtils extends Logging { - lazy val compressionCodecForShuffling: CompressionCodec = { + // optional compression codec to use when compressing shuffle files + lazy val compressionCodecForShuffling: Option[CompressionCodec] = { val sparkConf = SparkEnv.get.conf - val codecName = CometConf.COMET_EXEC_SHUFFLE_CODEC.get(SQLConf.get) - - // only zstd compression is supported at the moment - if (codecName != "zstd") { - logWarning( - s"Overriding config ${IO_COMPRESSION_CODEC}=${codecName} in shuffling, force using zstd") + val shuffleCompressionEnabled = sparkConf.getBoolean(SHUFFLE_COMPRESS.key, true) + val sparkShuffleCodec = sparkConf.get(IO_COMPRESSION_CODEC.key, "lz4") + val cometShuffleCodec = CometConf.COMET_EXEC_SHUFFLE_COMPRESSION_CODEC.get() + if (shuffleCompressionEnabled) { + if (sparkShuffleCodec != cometShuffleCodec) { + logWarning( + s"Overriding config $IO_COMPRESSION_CODEC=$sparkShuffleCodec in shuffling, " + + s"force using $cometShuffleCodec") + } + cometShuffleCodec match { + case "zstd" => + Some(CompressionCodec.createCodec(sparkConf, "zstd")) + case other => + throw new UnsupportedOperationException( + s"Unsupported shuffle compression codec: $other") + } + } else { + None } - CompressionCodec.createCodec(sparkConf, "zstd") } } diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 69da792223..7881f07632 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -50,7 +50,8 @@ Comet provides the following configuration settings. | spark.comet.exec.memoryFraction | The fraction of memory from Comet memory overhead that the native memory manager can use for execution. The purpose of this config is to set aside memory for untracked data structures, as well as imprecise size estimation during memory acquisition. | 0.7 | | spark.comet.exec.project.enabled | Whether to enable project by default. | true | | spark.comet.exec.replaceSortMergeJoin | Experimental feature to force Spark to replace SortMergeJoin with ShuffledHashJoin for improved performance. This feature is not stable yet. For more information, refer to the Comet Tuning Guide (https://datafusion.apache.org/comet/user-guide/tuning.html). | false | -| spark.comet.exec.shuffle.codec | The codec of Comet native shuffle used to compress shuffle data. Only zstd is supported. | zstd | +| spark.comet.exec.shuffle.compression.codec | The codec of Comet native shuffle used to compress shuffle data. Only zstd is supported. Compression can be disabled by setting spark.shuffle.compress=false. | zstd | +| spark.comet.exec.shuffle.compression.level | The compression level to use when compression shuffle files. | 1 | | spark.comet.exec.shuffle.enabled | Whether to enable Comet native shuffle. Note that this requires setting 'spark.shuffle.manager' to 'org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager'. 'spark.shuffle.manager' must be set before starting the Spark application and cannot be changed during the application. | true | | spark.comet.exec.sort.enabled | Whether to enable sort by default. | true | | spark.comet.exec.sortMergeJoin.enabled | Whether to enable sortMergeJoin by default. | true | diff --git a/docs/source/user-guide/tuning.md b/docs/source/user-guide/tuning.md index d68481d172..e04e750b47 100644 --- a/docs/source/user-guide/tuning.md +++ b/docs/source/user-guide/tuning.md @@ -103,6 +103,12 @@ native shuffle currently only supports `HashPartitioning` and `SinglePartitionin To enable native shuffle, set `spark.comet.exec.shuffle.mode` to `native`. If this mode is explicitly set, then any shuffle operations that cannot be supported in this mode will fall back to Spark. +### Shuffle Compression + +By default, Spark compresses shuffle files using LZ4 compression. Comet overrides this behavior with ZSTD compression. +Compression can be disabled by setting `spark.shuffle.compress=false`, which may result in faster shuffle times in +certain environments, such as single-node setups with fast NVMe drives, at the expense of increased disk space usage. + ## Explain Plan ### Extended Explain With Spark 4.0.0 and newer, Comet can provide extended explain plan information in the Spark UI. Currently this lists diff --git a/native/core/benches/shuffle_writer.rs b/native/core/benches/shuffle_writer.rs index 272887238e..865ca73b4a 100644 --- a/native/core/benches/shuffle_writer.rs +++ b/native/core/benches/shuffle_writer.rs @@ -15,36 +15,47 @@ // specific language governing permissions and limitations // under the License. +use arrow_array::builder::Int32Builder; use arrow_array::{builder::StringBuilder, RecordBatch}; use arrow_schema::{DataType, Field, Schema}; -use comet::execution::shuffle::ShuffleWriterExec; +use comet::execution::shuffle::{write_ipc_compressed, CompressionCodec, ShuffleWriterExec}; use criterion::{criterion_group, criterion_main, Criterion}; +use datafusion::physical_plan::metrics::Time; use datafusion::{ physical_plan::{common::collect, memory::MemoryExec, ExecutionPlan}, prelude::SessionContext, }; use datafusion_physical_expr::{expressions::Column, Partitioning}; +use std::io::Cursor; use std::sync::Arc; use tokio::runtime::Runtime; fn criterion_benchmark(c: &mut Criterion) { - let batch = create_batch(); - let mut batches = Vec::new(); - for _ in 0..10 { - batches.push(batch.clone()); - } - let partitions = &[batches]; - let exec = ShuffleWriterExec::try_new( - Arc::new(MemoryExec::try_new(partitions, batch.schema(), None).unwrap()), - Partitioning::Hash(vec![Arc::new(Column::new("a", 0))], 16), - "/tmp/data.out".to_string(), - "/tmp/index.out".to_string(), - ) - .unwrap(); - let mut group = c.benchmark_group("shuffle_writer"); - group.bench_function("shuffle_writer", |b| { + group.bench_function("shuffle_writer: encode (no compression))", |b| { + let batch = create_batch(8192, true); + let mut buffer = vec![]; + let mut cursor = Cursor::new(&mut buffer); + let ipc_time = Time::default(); + b.iter(|| write_ipc_compressed(&batch, &mut cursor, &CompressionCodec::None, &ipc_time)); + }); + group.bench_function("shuffle_writer: encode and compress (zstd level 1)", |b| { + let batch = create_batch(8192, true); + let mut buffer = vec![]; + let mut cursor = Cursor::new(&mut buffer); + let ipc_time = Time::default(); + b.iter(|| write_ipc_compressed(&batch, &mut cursor, &CompressionCodec::Zstd(1), &ipc_time)); + }); + group.bench_function("shuffle_writer: encode and compress (zstd level 6)", |b| { + let batch = create_batch(8192, true); + let mut buffer = vec![]; + let mut cursor = Cursor::new(&mut buffer); + let ipc_time = Time::default(); + b.iter(|| write_ipc_compressed(&batch, &mut cursor, &CompressionCodec::Zstd(6), &ipc_time)); + }); + group.bench_function("shuffle_writer: end to end", |b| { let ctx = SessionContext::new(); + let exec = create_shuffle_writer_exec(CompressionCodec::Zstd(1)); b.iter(|| { let task_ctx = ctx.task_ctx(); let stream = exec.execute(0, task_ctx).unwrap(); @@ -54,19 +65,47 @@ fn criterion_benchmark(c: &mut Criterion) { }); } -fn create_batch() -> RecordBatch { - let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, true)])); +fn create_shuffle_writer_exec(compression_codec: CompressionCodec) -> ShuffleWriterExec { + let batches = create_batches(8192, 10); + let schema = batches[0].schema(); + let partitions = &[batches]; + ShuffleWriterExec::try_new( + Arc::new(MemoryExec::try_new(partitions, schema, None).unwrap()), + Partitioning::Hash(vec![Arc::new(Column::new("a", 0))], 16), + compression_codec, + "/tmp/data.out".to_string(), + "/tmp/index.out".to_string(), + ) + .unwrap() +} + +fn create_batches(size: usize, count: usize) -> Vec { + let batch = create_batch(size, true); + let mut batches = Vec::new(); + for _ in 0..count { + batches.push(batch.clone()); + } + batches +} + +fn create_batch(num_rows: usize, allow_nulls: bool) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![ + Field::new("c0", DataType::Int32, true), + Field::new("c1", DataType::Utf8, true), + ])); + let mut a = Int32Builder::new(); let mut b = StringBuilder::new(); - for i in 0..8192 { - if i % 10 == 0 { + for i in 0..num_rows { + a.append_value(i as i32); + if allow_nulls && i % 10 == 0 { b.append_null(); } else { - b.append_value(format!("{i}")); + b.append_value(format!("this is string number {i}")); } } - let array = b.finish(); - - RecordBatch::try_new(schema.clone(), vec![Arc::new(array)]).unwrap() + let a = a.finish(); + let b = b.finish(); + RecordBatch::try_new(schema.clone(), vec![Arc::new(a), Arc::new(b)]).unwrap() } fn config() -> Criterion { diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 3ac830c04e..0a74933547 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -68,6 +68,7 @@ use datafusion_comet_spark_expr::{create_comet_physical_fun, create_negate_expr} use datafusion_functions_nested::concat::ArrayAppend; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; +use crate::execution::shuffle::CompressionCodec; use crate::execution::spark_plan::SparkPlan; use datafusion_comet_proto::{ spark_expression::{ @@ -76,8 +77,8 @@ use datafusion_comet_proto::{ }, spark_operator::{ self, lower_window_frame_bound::LowerFrameBoundStruct, operator::OpStruct, - upper_window_frame_bound::UpperFrameBoundStruct, BuildSide, JoinType, Operator, - WindowFrameType, + upper_window_frame_bound::UpperFrameBoundStruct, BuildSide, + CompressionCodec as SparkCompressionCodec, JoinType, Operator, WindowFrameType, }, spark_partitioning::{partitioning::PartitioningStruct, Partitioning as SparkPartitioning}, }; @@ -1049,9 +1050,21 @@ impl PhysicalPlanner { let partitioning = self .create_partitioning(writer.partitioning.as_ref().unwrap(), child.schema())?; + let codec = match writer.codec.try_into() { + Ok(SparkCompressionCodec::None) => Ok(CompressionCodec::None), + Ok(SparkCompressionCodec::Zstd) => { + Ok(CompressionCodec::Zstd(writer.compression_level)) + } + _ => Err(ExecutionError::GeneralError(format!( + "Unsupported shuffle compression codec: {:?}", + writer.codec + ))), + }?; + let shuffle_writer = Arc::new(ShuffleWriterExec::try_new( Arc::clone(&child.native_plan), partitioning, + codec, writer.output_data_file.clone(), writer.output_index_file.clone(), )?); diff --git a/native/core/src/execution/shuffle/mod.rs b/native/core/src/execution/shuffle/mod.rs index 8721ead74b..8111f5eede 100644 --- a/native/core/src/execution/shuffle/mod.rs +++ b/native/core/src/execution/shuffle/mod.rs @@ -19,4 +19,4 @@ mod list; mod map; pub mod row; mod shuffle_writer; -pub use shuffle_writer::ShuffleWriterExec; +pub use shuffle_writer::{write_ipc_compressed, CompressionCodec, ShuffleWriterExec}; diff --git a/native/core/src/execution/shuffle/row.rs b/native/core/src/execution/shuffle/row.rs index ecab77d966..405f642163 100644 --- a/native/core/src/execution/shuffle/row.rs +++ b/native/core/src/execution/shuffle/row.rs @@ -292,6 +292,7 @@ macro_rules! downcast_builder_ref { } // Expose the macro for other modules. +use crate::execution::shuffle::shuffle_writer::CompressionCodec; pub(crate) use downcast_builder_ref; /// Appends field of row to the given struct builder. `dt` is the data type of the field. @@ -3358,7 +3359,9 @@ pub fn process_sorted_row_partition( // we do not collect metrics in Native_writeSortedFileNative let ipc_time = Time::default(); - written += write_ipc_compressed(&batch, &mut cursor, &ipc_time)?; + // compression codec is not configurable for CometBypassMergeSortShuffleWriter + let codec = CompressionCodec::Zstd(1); + written += write_ipc_compressed(&batch, &mut cursor, &codec, &ipc_time)?; if let Some(checksum) = &mut current_checksum { checksum.update(&mut cursor)?; diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index fcc8c51f60..01117199eb 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -90,6 +90,7 @@ pub struct ShuffleWriterExec { /// Metrics metrics: ExecutionPlanMetricsSet, cache: PlanProperties, + codec: CompressionCodec, } impl DisplayAs for ShuffleWriterExec { @@ -126,6 +127,7 @@ impl ExecutionPlan for ShuffleWriterExec { 1 => Ok(Arc::new(ShuffleWriterExec::try_new( Arc::clone(&children[0]), self.partitioning.clone(), + self.codec.clone(), self.output_data_file.clone(), self.output_index_file.clone(), )?)), @@ -152,6 +154,7 @@ impl ExecutionPlan for ShuffleWriterExec { self.partitioning.clone(), metrics, context, + self.codec.clone(), ) .map_err(|e| ArrowError::ExternalError(Box::new(e))), ) @@ -181,6 +184,7 @@ impl ShuffleWriterExec { pub fn try_new( input: Arc, partitioning: Partitioning, + codec: CompressionCodec, output_data_file: String, output_index_file: String, ) -> Result { @@ -197,6 +201,7 @@ impl ShuffleWriterExec { output_data_file, output_index_file, cache, + codec, }) } } @@ -217,6 +222,7 @@ struct PartitionBuffer { batch_size: usize, /// Memory reservation for this partition buffer. reservation: MemoryReservation, + codec: CompressionCodec, } impl PartitionBuffer { @@ -225,6 +231,7 @@ impl PartitionBuffer { batch_size: usize, partition_id: usize, runtime: &Arc, + codec: CompressionCodec, ) -> Self { let reservation = MemoryConsumer::new(format!("PartitionBuffer[{}]", partition_id)) .with_can_spill(true) @@ -238,6 +245,7 @@ impl PartitionBuffer { num_active_rows: 0, batch_size, reservation, + codec, } } @@ -337,7 +345,7 @@ impl PartitionBuffer { let frozen_capacity_old = self.frozen.capacity(); let mut cursor = Cursor::new(&mut self.frozen); cursor.seek(SeekFrom::End(0))?; - write_ipc_compressed(&frozen_batch, &mut cursor, ipc_time)?; + write_ipc_compressed(&frozen_batch, &mut cursor, &self.codec, ipc_time)?; mem_diff += (self.frozen.capacity() - frozen_capacity_old) as isize; Ok(mem_diff) @@ -687,6 +695,7 @@ impl ShuffleRepartitioner { metrics: ShuffleRepartitionerMetrics, runtime: Arc, batch_size: usize, + codec: CompressionCodec, ) -> Self { let num_output_partitions = partitioning.partition_count(); let reservation = MemoryConsumer::new(format!("ShuffleRepartitioner[{}]", partition_id)) @@ -709,7 +718,13 @@ impl ShuffleRepartitioner { schema: Arc::clone(&schema), buffered_partitions: (0..num_output_partitions) .map(|partition_id| { - PartitionBuffer::new(Arc::clone(&schema), batch_size, partition_id, &runtime) + PartitionBuffer::new( + Arc::clone(&schema), + batch_size, + partition_id, + &runtime, + codec.clone(), + ) }) .collect::>(), spills: Mutex::new(vec![]), @@ -1129,6 +1144,7 @@ impl Debug for ShuffleRepartitioner { } } +#[allow(clippy::too_many_arguments)] async fn external_shuffle( mut input: SendableRecordBatchStream, partition_id: usize, @@ -1137,6 +1153,7 @@ async fn external_shuffle( partitioning: Partitioning, metrics: ShuffleRepartitionerMetrics, context: Arc, + codec: CompressionCodec, ) -> Result { let schema = input.schema(); let mut repartitioner = ShuffleRepartitioner::new( @@ -1148,6 +1165,7 @@ async fn external_shuffle( metrics, context.runtime_env(), context.session_config().batch_size(), + codec, ); while let Some(batch) = input.next().await { @@ -1526,11 +1544,18 @@ impl Checksum { } } +#[derive(Debug, Clone)] +pub enum CompressionCodec { + None, + Zstd(i32), +} + /// Writes given record batch as Arrow IPC bytes into given writer. /// Returns number of bytes written. -pub(crate) fn write_ipc_compressed( +pub fn write_ipc_compressed( batch: &RecordBatch, output: &mut W, + codec: &CompressionCodec, ipc_time: &Time, ) -> Result { if batch.num_rows() == 0 { @@ -1543,14 +1568,24 @@ pub(crate) fn write_ipc_compressed( // write ipc_length placeholder output.write_all(&[0u8; 8])?; - // write ipc data - // TODO: make compression level configurable - let mut arrow_writer = StreamWriter::try_new(zstd::Encoder::new(output, 1)?, &batch.schema())?; - arrow_writer.write(batch)?; - arrow_writer.finish()?; + let output = match codec { + CompressionCodec::None => { + let mut arrow_writer = StreamWriter::try_new(output, &batch.schema())?; + arrow_writer.write(batch)?; + arrow_writer.finish()?; + arrow_writer.into_inner()? + } + CompressionCodec::Zstd(level) => { + let encoder = zstd::Encoder::new(output, *level)?; + let mut arrow_writer = StreamWriter::try_new(encoder, &batch.schema())?; + arrow_writer.write(batch)?; + arrow_writer.finish()?; + let zstd_encoder = arrow_writer.into_inner()?; + zstd_encoder.finish()? + } + }; - let zwriter = arrow_writer.into_inner()?; - let output = zwriter.finish()?; + // fill ipc length let end_pos = output.stream_position()?; let ipc_length = end_pos - start_pos - 8; @@ -1611,6 +1646,22 @@ mod test { use datafusion_physical_expr::expressions::Column; use tokio::runtime::Runtime; + #[test] + #[cfg_attr(miri, ignore)] // miri can't call foreign function `ZSTD_createCCtx` + fn write_ipc_zstd() { + let batch = create_batch(8192); + let mut output = vec![]; + let mut cursor = Cursor::new(&mut output); + write_ipc_compressed( + &batch, + &mut cursor, + &CompressionCodec::Zstd(1), + &Time::default(), + ) + .unwrap(); + assert_eq!(40218, output.len()); + } + #[test] fn test_slot_size() { let batch_size = 1usize; @@ -1673,13 +1724,7 @@ mod test { num_partitions: usize, memory_limit: Option, ) { - let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, true)])); - let mut b = StringBuilder::new(); - for i in 0..batch_size { - b.append_value(format!("{i}")); - } - let array = b.finish(); - let batch = RecordBatch::try_new(Arc::clone(&schema), vec![Arc::new(array)]).unwrap(); + let batch = create_batch(batch_size); let batches = (0..num_batches).map(|_| batch.clone()).collect::>(); @@ -1687,6 +1732,7 @@ mod test { let exec = ShuffleWriterExec::try_new( Arc::new(MemoryExec::try_new(partitions, batch.schema(), None).unwrap()), Partitioning::Hash(vec![Arc::new(Column::new("a", 0))], num_partitions), + CompressionCodec::Zstd(1), "/tmp/data.out".to_string(), "/tmp/index.out".to_string(), ) @@ -1707,6 +1753,16 @@ mod test { rt.block_on(collect(stream)).unwrap(); } + fn create_batch(batch_size: usize) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, true)])); + let mut b = StringBuilder::new(); + for i in 0..batch_size { + b.append_value(format!("{i}")); + } + let array = b.finish(); + RecordBatch::try_new(Arc::clone(&schema), vec![Arc::new(array)]).unwrap() + } + #[test] fn test_pmod() { let i: Vec = vec![0x99f0149d, 0x9c67b85d, 0xc8008529, 0xa05b5d7b, 0xcd1e64fb]; diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 74ec80cb54..5cb2802da8 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -82,10 +82,17 @@ message Limit { int32 offset = 2; } +enum CompressionCodec { + None = 0; + Zstd = 1; +} + message ShuffleWriter { spark.spark_partitioning.Partitioning partitioning = 1; string output_data_file = 3; string output_index_file = 4; + CompressionCodec codec = 5; + int32 compression_level = 6; } enum AggregateMode { diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala index 0cd8a9ce67..3a11b8b28c 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala @@ -52,8 +52,9 @@ import org.apache.spark.util.random.XORShiftRandom import com.google.common.base.Objects +import org.apache.comet.CometConf import org.apache.comet.serde.{OperatorOuterClass, PartitioningOuterClass, QueryPlanSerde} -import org.apache.comet.serde.OperatorOuterClass.Operator +import org.apache.comet.serde.OperatorOuterClass.{CompressionCodec, Operator} import org.apache.comet.serde.QueryPlanSerde.serializeDataType import org.apache.comet.shims.ShimCometShuffleExchangeExec @@ -553,6 +554,17 @@ class CometShuffleWriteProcessor( shuffleWriterBuilder.setOutputDataFile(dataFile) shuffleWriterBuilder.setOutputIndexFile(indexFile) + if (SparkEnv.get.conf.getBoolean("spark.shuffle.compress", true)) { + val codec = CometConf.COMET_EXEC_SHUFFLE_COMPRESSION_CODEC.get() match { + case "zstd" => CompressionCodec.Zstd + case other => throw new UnsupportedOperationException(s"invalid codec: $other") + } + shuffleWriterBuilder.setCodec(codec) + } else { + shuffleWriterBuilder.setCodec(CompressionCodec.None) + } + shuffleWriterBuilder.setCompressionLevel(CometConf.COMET_EXEC_SHUFFLE_COMPRESSION_LEVEL.get) + outputPartitioning match { case _: HashPartitioning => val hashPartitioning = outputPartitioning.asInstanceOf[HashPartitioning] diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleManager.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleManager.scala index ef67167c4b..b2cc2c2bad 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleManager.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleManager.scala @@ -243,7 +243,7 @@ object CometShuffleManager extends Logging { lazy val compressionCodecForShuffling: CompressionCodec = { val sparkConf = SparkEnv.get.conf - val codecName = CometConf.COMET_EXEC_SHUFFLE_CODEC.get(SQLConf.get) + val codecName = CometConf.COMET_EXEC_SHUFFLE_COMPRESSION_CODEC.get(SQLConf.get) // only zstd compression is supported at the moment if (codecName != "zstd") { From 053b7ccc18c8397c7035a79b4df092beddf28289 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 21 Dec 2024 17:59:34 -0700 Subject: [PATCH 42/83] minor: move shuffle classes from common to spark (#1193) --- .../execution/shuffle/ArrowReaderIterator.scala | 0 .../shuffle/CometBlockStoreShuffleReader.scala | 17 ++++------------- .../shuffle/CometShuffleDependency.scala | 0 .../execution/shuffle/CometShuffledRowRDD.scala | 4 ++-- .../shuffle/IpcInputStreamIterator.scala | 9 +++------ .../comet/execution/shuffle/ShuffleUtils.scala | 0 6 files changed, 9 insertions(+), 21 deletions(-) rename {common => spark}/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ArrowReaderIterator.scala (100%) rename {common => spark}/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala (92%) rename {common => spark}/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleDependency.scala (100%) rename {common => spark}/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala (95%) rename {common => spark}/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala (95%) rename {common => spark}/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ShuffleUtils.scala (100%) diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ArrowReaderIterator.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ArrowReaderIterator.scala similarity index 100% rename from common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ArrowReaderIterator.scala rename to spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ArrowReaderIterator.scala diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala similarity index 92% rename from common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala rename to spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala index e026cbeb1a..74c6559504 100644 --- a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala @@ -21,21 +21,12 @@ package org.apache.spark.sql.comet.execution.shuffle import java.io.InputStream -import org.apache.spark.InterruptibleIterator -import org.apache.spark.MapOutputTracker -import org.apache.spark.SparkEnv -import org.apache.spark.TaskContext -import org.apache.spark.internal.Logging -import org.apache.spark.internal.config +import org.apache.spark.{InterruptibleIterator, MapOutputTracker, SparkEnv, TaskContext} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.io.CompressionCodec import org.apache.spark.serializer.SerializerManager -import org.apache.spark.shuffle.BaseShuffleHandle -import org.apache.spark.shuffle.ShuffleReader -import org.apache.spark.shuffle.ShuffleReadMetricsReporter -import org.apache.spark.storage.BlockId -import org.apache.spark.storage.BlockManager -import org.apache.spark.storage.BlockManagerId -import org.apache.spark.storage.ShuffleBlockFetcherIterator +import org.apache.spark.shuffle.{BaseShuffleHandle, ShuffleReader, ShuffleReadMetricsReporter} +import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerId, ShuffleBlockFetcherIterator} import org.apache.spark.util.CompletionIterator /** diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleDependency.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleDependency.scala similarity index 100% rename from common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleDependency.scala rename to spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleDependency.scala diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala similarity index 95% rename from common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala rename to spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala index af78ed2905..ba6fc588e2 100644 --- a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala @@ -19,10 +19,10 @@ package org.apache.spark.sql.comet.execution.shuffle -import org.apache.spark.{Dependency, MapOutputTrackerMaster, Partition, Partitioner, ShuffleDependency, SparkEnv, TaskContext} +import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.shuffle.sort.SortShuffleManager -import org.apache.spark.sql.execution.{CoalescedMapperPartitionSpec, CoalescedPartitioner, CoalescedPartitionSpec, PartialMapperPartitionSpec, PartialReducerPartitionSpec, ShufflePartitionSpec} +import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLShuffleReadMetricsReporter} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.vectorized.ColumnarBatch diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala similarity index 95% rename from common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala rename to spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala index d1d5af350f..aa40550488 100644 --- a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala @@ -19,12 +19,9 @@ package org.apache.spark.sql.comet.execution.shuffle -import java.io.EOFException -import java.io.InputStream -import java.nio.ByteBuffer -import java.nio.ByteOrder -import java.nio.channels.Channels -import java.nio.channels.ReadableByteChannel +import java.io.{EOFException, InputStream} +import java.nio.{ByteBuffer, ByteOrder} +import java.nio.channels.{Channels, ReadableByteChannel} import org.apache.spark.TaskContext import org.apache.spark.internal.Logging diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ShuffleUtils.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ShuffleUtils.scala similarity index 100% rename from common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ShuffleUtils.scala rename to spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ShuffleUtils.scala From 639fa2fb450a760728bc6921ca58da64df0c64a7 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 22 Dec 2024 12:25:21 -0700 Subject: [PATCH 43/83] minor: refactor decodeBatches to make private in broadcast exchange (#1195) --- .../comet/CometBroadcastExchangeExec.scala | 24 +++++++++++-- .../apache/spark/sql/comet/operators.scala | 35 ++----------------- .../apache/comet/exec/CometExecSuite.scala | 33 ----------------- 3 files changed, 25 insertions(+), 67 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala index ccf218cf6c..6bc519ab95 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.comet +import java.io.DataInputStream +import java.nio.channels.Channels import java.util.UUID import java.util.concurrent.{Future, TimeoutException, TimeUnit} @@ -26,13 +28,15 @@ import scala.concurrent.{ExecutionContext, Promise} import scala.concurrent.duration.NANOSECONDS import scala.util.control.NonFatal -import org.apache.spark.{broadcast, Partition, SparkContext, TaskContext} +import org.apache.spark.{broadcast, Partition, SparkContext, SparkEnv, TaskContext} import org.apache.spark.comet.shims.ShimCometBroadcastExchangeExec +import org.apache.spark.io.CompressionCodec import org.apache.spark.launcher.SparkLauncher import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.Statistics +import org.apache.spark.sql.comet.execution.shuffle.ArrowReaderIterator import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, SQLExecution} import org.apache.spark.sql.execution.adaptive.{AQEShuffleReadExec, ShuffleQueryStageExec} @@ -299,7 +303,23 @@ class CometBatchRDD( override def compute(split: Partition, context: TaskContext): Iterator[ColumnarBatch] = { val partition = split.asInstanceOf[CometBatchPartition] partition.value.value.toIterator - .flatMap(CometExec.decodeBatches(_, this.getClass.getSimpleName)) + .flatMap(decodeBatches(_, this.getClass.getSimpleName)) + } + + /** + * Decodes the byte arrays back to ColumnarBatchs and put them into buffer. + */ + private def decodeBatches(bytes: ChunkedByteBuffer, source: String): Iterator[ColumnarBatch] = { + if (bytes.size == 0) { + return Iterator.empty + } + + // use Spark's compression codec (LZ4 by default) and not Comet's compression + val codec = CompressionCodec.createCodec(SparkEnv.get.conf) + val cbbis = bytes.toInputStream() + val ins = new DataInputStream(codec.compressedInputStream(cbbis)) + // batches are in Arrow IPC format + new ArrowReaderIterator(Channels.newChannel(ins), source) } } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala index 77188312e4..c70f7464ec 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala @@ -19,14 +19,12 @@ package org.apache.spark.sql.comet -import java.io.{ByteArrayOutputStream, DataInputStream} -import java.nio.channels.Channels +import java.io.ByteArrayOutputStream import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{SparkEnv, TaskContext} -import org.apache.spark.io.CompressionCodec +import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression, NamedExpression, SortOrder} @@ -34,7 +32,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning} -import org.apache.spark.sql.comet.execution.shuffle.{ArrowReaderIterator, CometShuffleExchangeExec} +import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec import org.apache.spark.sql.comet.plans.PartitioningPreservingUnaryExecNode import org.apache.spark.sql.comet.util.Utils import org.apache.spark.sql.execution.{BinaryExecNode, ColumnarToRowExec, ExecSubqueryExpression, ExplainUtils, LeafExecNode, ScalarSubquery, SparkPlan, UnaryExecNode} @@ -78,18 +76,6 @@ abstract class CometExec extends CometPlan { // outputPartitioning of SparkPlan, e.g., AQEShuffleReadExec. override def outputPartitioning: Partitioning = originalPlan.outputPartitioning - /** - * Executes the Comet operator and returns the result as an iterator of ColumnarBatch. - */ - def executeColumnarCollectIterator(): (Long, Iterator[ColumnarBatch]) = { - val countsAndBytes = CometExec.getByteArrayRdd(this).collect() - val total = countsAndBytes.map(_._1).sum - val rows = countsAndBytes.iterator - .flatMap(countAndBytes => - CometExec.decodeBatches(countAndBytes._2, this.getClass.getSimpleName)) - (total, rows) - } - protected def setSubqueries(planId: Long, sparkPlan: SparkPlan): Unit = { sparkPlan.children.foreach(setSubqueries(planId, _)) @@ -161,21 +147,6 @@ object CometExec { Utils.serializeBatches(iter) } } - - /** - * Decodes the byte arrays back to ColumnarBatchs and put them into buffer. - */ - def decodeBatches(bytes: ChunkedByteBuffer, source: String): Iterator[ColumnarBatch] = { - if (bytes.size == 0) { - return Iterator.empty - } - - val codec = CompressionCodec.createCodec(SparkEnv.get.conf) - val cbbis = bytes.toInputStream() - val ins = new DataInputStream(codec.compressedInputStream(cbbis)) - - new ArrowReaderIterator(Channels.newChannel(ins), source) - } } /** diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index 1027695372..90c3221e5d 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -22,8 +22,6 @@ package org.apache.comet.exec import java.sql.Date import java.time.{Duration, Period} -import scala.collection.JavaConverters._ -import scala.collection.mutable import scala.util.Random import org.scalactic.source.Position @@ -462,37 +460,6 @@ class CometExecSuite extends CometTestBase { } } - test("CometExec.executeColumnarCollectIterator can collect ColumnarBatch results") { - assume(isSpark34Plus, "ChunkedByteBuffer is not serializable before Spark 3.4+") - withSQLConf(CometConf.COMET_EXEC_ENABLED.key -> "true") { - withParquetTable((0 until 50).map(i => (i, i + 1)), "tbl") { - val df = sql("SELECT _1 + 1, _2 + 2 FROM tbl WHERE _1 > 3") - - val nativeProject = find(df.queryExecution.executedPlan) { - case _: CometProjectExec => true - case _ => false - }.get.asInstanceOf[CometProjectExec] - - val (rows, batches) = nativeProject.executeColumnarCollectIterator() - assert(rows == 46) - - val column1 = mutable.ArrayBuffer.empty[Int] - val column2 = mutable.ArrayBuffer.empty[Int] - - batches.foreach(batch => { - batch.rowIterator().asScala.foreach { row => - assert(row.numFields == 2) - column1 += row.getInt(0) - column2 += row.getInt(1) - } - }) - - assert(column1.toArray.sorted === (4 until 50).map(_ + 1).toArray) - assert(column2.toArray.sorted === (5 until 51).map(_ + 2).toArray) - } - } - } - test("scalar subquery") { val dataTypes = Seq( From 58dee739b6b8c3e7c6057e01c72307cdcff56ada Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 22 Dec 2024 12:25:35 -0700 Subject: [PATCH 44/83] minor: refactor prepare_output so that it does not require an ExecutionContext (#1194) --- native/core/src/execution/jni_api.rs | 20 +++++++------------- 1 file changed, 7 insertions(+), 13 deletions(-) diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index eb73675b5b..2c1a55f48f 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -231,7 +231,7 @@ fn prepare_output( array_addrs: jlongArray, schema_addrs: jlongArray, output_batch: RecordBatch, - exec_context: &mut ExecutionContext, + validate: bool, ) -> CometResult { let array_address_array = unsafe { JLongArray::from_raw(array_addrs) }; let num_cols = env.get_array_length(&array_address_array)? as usize; @@ -255,7 +255,7 @@ fn prepare_output( ))); } - if exec_context.debug_native { + if validate { // Validate the output arrays. for array in results.iter() { let array_data = array.to_data(); @@ -275,9 +275,6 @@ fn prepare_output( i += 1; } - // Update metrics - update_metrics(env, exec_context)?; - Ok(num_rows as jlong) } @@ -356,22 +353,22 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlan( let next_item = exec_context.stream.as_mut().unwrap().next(); let poll_output = exec_context.runtime.block_on(async { poll!(next_item) }); + // Update metrics + update_metrics(&mut env, exec_context)?; + match poll_output { Poll::Ready(Some(output)) => { + // prepare output for FFI transfer return prepare_output( &mut env, array_addrs, schema_addrs, output?, - exec_context, + exec_context.debug_native, ); } Poll::Ready(None) => { // Reaches EOF of output. - - // Update metrics - update_metrics(&mut env, exec_context)?; - if exec_context.explain_native { if let Some(plan) = &exec_context.root_op { let formatted_plan_str = @@ -391,9 +388,6 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlan( // A poll pending means there are more than one blocking operators, // we don't need go back-forth between JVM/Native. Just keeping polling. Poll::Pending => { - // Update metrics - update_metrics(&mut env, exec_context)?; - // Pull input batches pull_input_batches(exec_context)?; From 5432e03e4c441257056638dca70f576e1e4c186b Mon Sep 17 00:00:00 2001 From: Raz Luvaton <16746759+rluvaton@users.noreply.github.com> Date: Fri, 27 Dec 2024 08:43:53 +0200 Subject: [PATCH 45/83] fix: fix missing explanation for then branch in case when (#1200) --- .../src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index b33f6b5a6b..518fa06858 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -1745,7 +1745,7 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim exprToProtoInternal(elements._1, inputs) }) val thenSeq = branches.map(elements => { - allBranches = allBranches :+ elements._1 + allBranches = allBranches :+ elements._2 exprToProtoInternal(elements._2, inputs) }) assert(whenSeq.length == thenSeq.length) From 103f82f5491a48d8f0dbcb697741270d7fe268b8 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 27 Dec 2024 22:54:17 -0700 Subject: [PATCH 46/83] minor: remove unused source files (#1202) --- native/Cargo.lock | 48 +--- native/core/Cargo.toml | 4 - native/core/src/parquet/compression.rs | 319 --------------------- native/core/src/parquet/util/jni_buffer.rs | 98 ------- 4 files changed, 2 insertions(+), 467 deletions(-) delete mode 100644 native/core/src/parquet/compression.rs delete mode 100644 native/core/src/parquet/util/jni_buffer.rs diff --git a/native/Cargo.lock b/native/Cargo.lock index 538c40ee23..ad572acb9e 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -428,17 +428,6 @@ dependencies = [ "generic-array", ] -[[package]] -name = "brotli" -version = "3.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d640d25bc63c50fb1f0b545ffd80207d2e10a4c965530809b40ba3386825c391" -dependencies = [ - "alloc-no-stdlib", - "alloc-stdlib", - "brotli-decompressor 2.5.1", -] - [[package]] name = "brotli" version = "7.0.0" @@ -447,17 +436,7 @@ checksum = "cc97b8f16f944bba54f0433f07e30be199b6dc2bd25937444bbad560bcea29bd" dependencies = [ "alloc-no-stdlib", "alloc-stdlib", - "brotli-decompressor 4.0.1", -] - -[[package]] -name = "brotli-decompressor" -version = "2.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4e2e4afe60d7dd600fdd3de8d0f08c2b7ec039712e3b6137ff98b7004e82de4f" -dependencies = [ - "alloc-no-stdlib", - "alloc-stdlib", + "brotli-decompressor", ] [[package]] @@ -900,7 +879,6 @@ dependencies = [ "arrow-schema", "assertables", "async-trait", - "brotli 3.5.0", "bytes", "crc32fast", "criterion", @@ -912,7 +890,6 @@ dependencies = [ "datafusion-expr", "datafusion-functions-nested", "datafusion-physical-expr", - "flate2", "futures", "hex", "itertools 0.11.0", @@ -920,7 +897,6 @@ dependencies = [ "lazy_static", "log", "log4rs", - "lz4", "mimalloc", "num", "once_cell", @@ -932,7 +908,6 @@ dependencies = [ "regex", "serde", "simd-adler32", - "snap", "tempfile", "thiserror", "tokio", @@ -2111,25 +2086,6 @@ dependencies = [ "winapi", ] -[[package]] -name = "lz4" -version = "1.28.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d1febb2b4a79ddd1980eede06a8f7902197960aa0383ffcfdd62fe723036725" -dependencies = [ - "lz4-sys", -] - -[[package]] -name = "lz4-sys" -version = "1.11.1+lz4-1.10.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6bd8c0d6c6ed0cd30b3652886bb8711dc4bb01d637a68105a3d5158039b418e6" -dependencies = [ - "cc", - "libc", -] - [[package]] name = "lz4_flex" version = "0.11.3" @@ -2382,7 +2338,7 @@ dependencies = [ "arrow-schema", "arrow-select", "base64", - "brotli 7.0.0", + "brotli", "bytes", "chrono", "flate2", diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 489da46d47..5089e67a03 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -52,10 +52,6 @@ serde = { version = "1", features = ["derive"] } lazy_static = "1.4.0" prost = "0.12.1" jni = "0.21" -snap = "1.1" -brotli = "3.3" -flate2 = "1.0" -lz4 = "1.24" zstd = "0.11" rand = { workspace = true} num = { workspace = true } diff --git a/native/core/src/parquet/compression.rs b/native/core/src/parquet/compression.rs deleted file mode 100644 index 37b857f4a2..0000000000 --- a/native/core/src/parquet/compression.rs +++ /dev/null @@ -1,319 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Contains codec interface and supported codec implementations. -//! -//! See [`Compression`](crate::basic::Compression) enum for all available compression -//! algorithms. -//! -//! # Example -//! -//! ```no_run -//! use comet::parquet::{basic::Compression, compression::create_codec}; -//! -//! let mut codec = match create_codec(Compression::SNAPPY) { -//! Ok(Some(codec)) => codec, -//! _ => panic!(), -//! }; -//! -//! let data = vec![b'p', b'a', b'r', b'q', b'u', b'e', b't']; -//! let mut compressed = vec![]; -//! codec.compress(&data[..], &mut compressed).unwrap(); -//! -//! let mut output = vec![]; -//! codec.decompress(&compressed[..], &mut output).unwrap(); -//! -//! assert_eq!(output, data); -//! ``` - -use super::basic::Compression as CodecType; -use crate::errors::{ParquetError, ParquetResult as Result}; - -use brotli::Decompressor; -use flate2::{read, write, Compression}; -use snap::raw::{decompress_len, max_compress_len, Decoder, Encoder}; -use std::io::{copy, Read, Write}; - -/// Parquet compression codec interface. -#[allow(clippy::ptr_arg)] -pub trait Codec { - /// Compresses data stored in slice `input_buf` and writes the compressed result - /// to `output_buf`. - /// Note that you'll need to call `clear()` before reusing the same `output_buf` - /// across different `compress` calls. - fn compress(&mut self, input_buf: &[u8], output_buf: &mut Vec) -> Result<()>; - - /// Decompresses data stored in slice `input_buf` and writes output to `output_buf`. - /// Returns the total number of bytes written. - fn decompress(&mut self, input_buf: &[u8], output_buf: &mut Vec) -> Result; -} - -/// Given the compression type `codec`, returns a codec used to compress and decompress -/// bytes for the compression type. -/// This returns `None` if the codec type is `UNCOMPRESSED`. -pub fn create_codec(codec: CodecType) -> Result>> { - match codec { - CodecType::BROTLI => Ok(Some(Box::new(BrotliCodec::new()))), - CodecType::GZIP => Ok(Some(Box::new(GZipCodec::new()))), - CodecType::SNAPPY => Ok(Some(Box::new(SnappyCodec::new()))), - CodecType::LZ4 => Ok(Some(Box::new(LZ4Codec::new()))), - CodecType::ZSTD => Ok(Some(Box::new(ZSTDCodec::new()))), - CodecType::UNCOMPRESSED => Ok(None), - _ => Err(nyi_err!("The codec type {} is not supported yet", codec)), - } -} - -/// Codec for Snappy compression format. -pub struct SnappyCodec { - decoder: Decoder, - encoder: Encoder, -} - -impl SnappyCodec { - /// Creates new Snappy compression codec. - pub(crate) fn new() -> Self { - Self { - decoder: Decoder::new(), - encoder: Encoder::new(), - } - } -} - -impl Codec for SnappyCodec { - fn decompress(&mut self, input_buf: &[u8], output_buf: &mut Vec) -> Result { - let len = decompress_len(input_buf)?; - output_buf.resize(len, 0); - self.decoder - .decompress(input_buf, output_buf) - .map_err(|e| e.into()) - } - - fn compress(&mut self, input_buf: &[u8], output_buf: &mut Vec) -> Result<()> { - let output_buf_len = output_buf.len(); - let required_len = max_compress_len(input_buf.len()); - output_buf.resize(output_buf_len + required_len, 0); - let n = self - .encoder - .compress(input_buf, &mut output_buf[output_buf_len..])?; - output_buf.truncate(output_buf_len + n); - Ok(()) - } -} - -/// Codec for GZIP compression algorithm. -pub struct GZipCodec {} - -impl GZipCodec { - /// Creates new GZIP compression codec. - pub(crate) fn new() -> Self { - Self {} - } -} - -impl Codec for GZipCodec { - fn decompress(&mut self, input_buf: &[u8], output_buf: &mut Vec) -> Result { - let mut decoder = read::GzDecoder::new(input_buf); - decoder.read_to_end(output_buf).map_err(|e| e.into()) - } - - fn compress(&mut self, input_buf: &[u8], output_buf: &mut Vec) -> Result<()> { - let mut encoder = write::GzEncoder::new(output_buf, Compression::default()); - encoder.write_all(input_buf)?; - encoder.try_finish().map_err(|e| e.into()) - } -} - -const BROTLI_DEFAULT_BUFFER_SIZE: usize = 4096; -const BROTLI_DEFAULT_COMPRESSION_QUALITY: u32 = 1; // supported levels 0-9 -const BROTLI_DEFAULT_LG_WINDOW_SIZE: u32 = 22; // recommended between 20-22 - -/// Codec for Brotli compression algorithm. -pub struct BrotliCodec {} - -impl BrotliCodec { - /// Creates new Brotli compression codec. - pub(crate) fn new() -> Self { - Self {} - } -} - -impl Codec for BrotliCodec { - fn decompress(&mut self, input_buf: &[u8], output_buf: &mut Vec) -> Result { - Decompressor::new(input_buf, BROTLI_DEFAULT_BUFFER_SIZE) - .read_to_end(output_buf) - .map_err(|e| e.into()) - } - - fn compress(&mut self, input_buf: &[u8], output_buf: &mut Vec) -> Result<()> { - let mut encoder = brotli::CompressorWriter::new( - output_buf, - BROTLI_DEFAULT_BUFFER_SIZE, - BROTLI_DEFAULT_COMPRESSION_QUALITY, - BROTLI_DEFAULT_LG_WINDOW_SIZE, - ); - encoder.write_all(input_buf)?; - encoder.flush().map_err(|e| e.into()) - } -} - -const LZ4_BUFFER_SIZE: usize = 4096; - -/// Codec for LZ4 compression algorithm. -pub struct LZ4Codec {} - -impl LZ4Codec { - /// Creates new LZ4 compression codec. - pub(crate) fn new() -> Self { - Self {} - } -} - -impl Codec for LZ4Codec { - fn decompress(&mut self, input_buf: &[u8], output_buf: &mut Vec) -> Result { - let mut decoder = lz4::Decoder::new(input_buf)?; - let mut buffer: [u8; LZ4_BUFFER_SIZE] = [0; LZ4_BUFFER_SIZE]; - let mut total_len = 0; - loop { - let len = decoder.read(&mut buffer)?; - if len == 0 { - break; - } - total_len += len; - output_buf.write_all(&buffer[0..len])?; - } - Ok(total_len) - } - - fn compress(&mut self, input_buf: &[u8], output_buf: &mut Vec) -> Result<()> { - let mut encoder = lz4::EncoderBuilder::new().build(output_buf)?; - let mut from = 0; - loop { - let to = std::cmp::min(from + LZ4_BUFFER_SIZE, input_buf.len()); - encoder.write_all(&input_buf[from..to])?; - from += LZ4_BUFFER_SIZE; - if from >= input_buf.len() { - break; - } - } - encoder.finish().1.map_err(|e| e.into()) - } -} - -/// Codec for Zstandard compression algorithm. -pub struct ZSTDCodec {} - -impl ZSTDCodec { - /// Creates new Zstandard compression codec. - pub(crate) fn new() -> Self { - Self {} - } -} - -/// Compression level (1-21) for ZSTD. Choose 1 here for better compression speed. -const ZSTD_COMPRESSION_LEVEL: i32 = 1; - -impl Codec for ZSTDCodec { - fn decompress(&mut self, input_buf: &[u8], output_buf: &mut Vec) -> Result { - let mut decoder = zstd::Decoder::new(input_buf)?; - match copy(&mut decoder, output_buf) { - Ok(n) => Ok(n as usize), - Err(e) => Err(e.into()), - } - } - - fn compress(&mut self, input_buf: &[u8], output_buf: &mut Vec) -> Result<()> { - let mut encoder = zstd::Encoder::new(output_buf, ZSTD_COMPRESSION_LEVEL)?; - encoder.write_all(input_buf)?; - match encoder.finish() { - Ok(_) => Ok(()), - Err(e) => Err(e.into()), - } - } -} - -#[cfg(test)] -mod tests { - use super::*; - - use crate::parquet::util::test_common::*; - - fn test_roundtrip(c: CodecType, data: &[u8]) { - let mut c1 = create_codec(c).unwrap().unwrap(); - let mut c2 = create_codec(c).unwrap().unwrap(); - - // Compress with c1 - let mut compressed = Vec::new(); - let mut decompressed = Vec::new(); - c1.compress(data, &mut compressed) - .expect("Error when compressing"); - - // Decompress with c2 - let mut decompressed_size = c2 - .decompress(compressed.as_slice(), &mut decompressed) - .expect("Error when decompressing"); - assert_eq!(data.len(), decompressed_size); - decompressed.truncate(decompressed_size); - assert_eq!(data, decompressed.as_slice()); - - compressed.clear(); - - // Compress with c2 - c2.compress(data, &mut compressed) - .expect("Error when compressing"); - - // Decompress with c1 - decompressed_size = c1 - .decompress(compressed.as_slice(), &mut decompressed) - .expect("Error when decompressing"); - assert_eq!(data.len(), decompressed_size); - decompressed.truncate(decompressed_size); - assert_eq!(data, decompressed.as_slice()); - } - - fn test_codec(c: CodecType) { - let sizes = vec![100, 10000, 100000]; - for size in sizes { - let data = random_bytes(size); - test_roundtrip(c, &data); - } - } - - #[test] - fn test_codec_snappy() { - test_codec(CodecType::SNAPPY); - } - - #[test] - fn test_codec_gzip() { - test_codec(CodecType::GZIP); - } - - #[test] - fn test_codec_brotli() { - test_codec(CodecType::BROTLI); - } - - #[test] - fn test_codec_lz4() { - test_codec(CodecType::LZ4); - } - - #[test] - fn test_codec_zstd() { - test_codec(CodecType::ZSTD); - } -} diff --git a/native/core/src/parquet/util/jni_buffer.rs b/native/core/src/parquet/util/jni_buffer.rs deleted file mode 100644 index 33f36ed9dd..0000000000 --- a/native/core/src/parquet/util/jni_buffer.rs +++ /dev/null @@ -1,98 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use core::slice; -use std::ptr::NonNull; - -use jni::{ - objects::{ReleaseMode, TypeArray}, - sys::{jbyte, jbyteArray, JNI_TRUE}, - JavaVM, -}; - -use crate::errors::{CometError, CometResult as Result}; - -use super::Buffer; - -/// An immutable byte buffer wrapping a JNI byte array allocated on heap. -/// -/// Unlike `AutoArray`, this doesn't have a lifetime and can be used across different JNI calls. -pub struct JniBuffer { - /// A pointer for the JVM instance, used to obtain byte array elements (via - /// `GetByteArrayElements`) and release byte array elements (via `ReleaseByteArrayElements`). - jvm: JavaVM, - /// The original JNI byte array that backs this buffer - inner: jbyteArray, - /// The raw pointer from the JNI byte array - ptr: NonNull, - /// Total number of bytes in the original array (i.e., `inner`). - len: usize, - /// Whether the JNI byte array is copied or not. - is_copy: bool, -} - -impl JniBuffer { - pub fn try_new(jvm: JavaVM, array: jbyteArray, len: usize) -> Result { - let env = jvm.get_env()?; - let mut is_copy = 0xff; - let ptr = jbyte::get(&env, array.into(), &mut is_copy)?; - let res = Self { - jvm, - inner: array, - ptr: NonNull::new(ptr) - .ok_or_else(|| CometError::NullPointer("null byte array pointer".to_string()))?, - len, - is_copy: is_copy == JNI_TRUE, - }; - Ok(res) - } - - /// Whether the JNI byte array is copied or not, i.e., whether the JVM pinned down the original - /// Java byte array, or made a new copy of it. - pub fn is_copy(&self) -> bool { - self.is_copy - } -} - -impl Buffer for JniBuffer { - fn len(&self) -> usize { - self.len - } - - fn data(&self) -> &[u8] { - self.as_ref() - } -} - -impl AsRef<[u8]> for JniBuffer { - fn as_ref(&self) -> &[u8] { - unsafe { slice::from_raw_parts(self.ptr.as_ptr() as *mut u8 as *const u8, self.len) } - } -} - -impl Drop for JniBuffer { - fn drop(&mut self) { - let env = self.jvm.get_env().unwrap(); // TODO: log error here - jbyte::release( - &env, - self.inner.into(), - self.ptr, - ReleaseMode::NoCopyBack as i32, // don't copy back since it's read-only here - ) - .unwrap(); - } -} From 5d2c909f59caf90348096511d4a26e48a0ae8e3d Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 28 Dec 2024 15:14:11 -0700 Subject: [PATCH 47/83] chore: Upgrade to DataFusion 44.0.0-rc2 (#1154) * move aggregate expressions to spark-expr crate * move more expressions * move benchmark * normalize_nan * bitwise not * comet scalar funcs * update bench imports * save * save * save * remove unused imports * clippy * implement more hashers * implement Hash and PartialEq * implement Hash and PartialEq * implement Hash and PartialEq * benches * fix ScalarUDFImpl.return_type failure * exclude test from miri * ignore correct test * ignore another test * remove miri checks * use return_type_from_exprs * Revert "use return_type_from_exprs" This reverts commit febc1f1ec1301f9b359fc23ad6a117224fce35b7. * use DF main branch * hacky workaround for regression in ScalarUDFImpl.return_type * fix repo url * pin to revision * bump to latest rev * bump to latest DF rev * bump DF to rev 9f530dd * add Cargo.lock * bump DF version * no default features * Revert "remove miri checks" This reverts commit 4638fe3aa5501966cd5d8b53acf26c698b10b3c9. * Update pin to DataFusion e99e02b9b9093ceb0c13a2dd32a2a89beba47930 * update pin * Update Cargo.toml Bump to 44.0.0-rc2 * update cargo lock * revert miri change --------- Co-authored-by: Andrew Lamb --- native/Cargo.lock | 598 +++++++++--------- native/Cargo.toml | 29 +- .../expressions/bloom_filter_might_contain.rs | 46 +- .../src/execution/expressions/subquery.rs | 23 +- native/core/src/execution/jni_api.rs | 9 +- native/core/src/execution/operators/copy.rs | 4 +- native/core/src/execution/operators/expand.rs | 6 +- native/core/src/execution/operators/filter.rs | 3 +- native/core/src/execution/operators/scan.rs | 4 +- native/core/src/execution/planner.rs | 55 +- .../src/execution/shuffle/shuffle_writer.rs | 6 +- .../src/execution/util/spark_bit_array.rs | 2 +- .../src/execution/util/spark_bloom_filter.rs | 2 +- native/spark-expr/Cargo.toml | 1 + native/spark-expr/benches/aggregate.rs | 3 +- native/spark-expr/src/avg.rs | 22 +- native/spark-expr/src/avg_decimal.rs | 20 +- native/spark-expr/src/bitwise_not.rs | 38 +- native/spark-expr/src/cast.rs | 42 +- native/spark-expr/src/checkoverflow.rs | 53 +- native/spark-expr/src/correlation.rs | 29 +- native/spark-expr/src/covariance.rs | 26 +- native/spark-expr/src/if_expr.rs | 49 +- native/spark-expr/src/list.rs | 127 ++-- native/spark-expr/src/negative.rs | 37 +- native/spark-expr/src/normalize_nan.rs | 45 +- native/spark-expr/src/regexp.rs | 27 +- native/spark-expr/src/stddev.rs | 30 +- native/spark-expr/src/strings.rs | 89 ++- native/spark-expr/src/structs.rs | 59 +- native/spark-expr/src/sum_decimal.rs | 28 +- native/spark-expr/src/temporal.rs | 175 +++-- native/spark-expr/src/to_json.rs | 23 +- native/spark-expr/src/unbound.rs | 21 +- native/spark-expr/src/variance.rs | 19 +- 35 files changed, 715 insertions(+), 1035 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index ad572acb9e..bbc0ff97a9 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -1,6 +1,6 @@ # This file is automatically @generated by Cargo. # It is not intended for manual editing. -version = 3 +version = 4 [[package]] name = "addr2line" @@ -57,9 +57,9 @@ dependencies = [ [[package]] name = "allocator-api2" -version = "0.2.20" +version = "0.2.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "45862d1c77f2228b9e10bc609d5bc203d86ebc9b87ad8d5d5167a6c9abf739d9" +checksum = "683d7910e743518b0e34f1186f92494becacb047c7b6bf616c96772180fef923" [[package]] name = "android-tzdata" @@ -90,9 +90,9 @@ checksum = "55cc3b69f167a1ef2e161439aa98aed94e6028e5f9a59be9a6ffb47aef1651f9" [[package]] name = "anyhow" -version = "1.0.93" +version = "1.0.95" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c95c10ba0b00a02636238b814946408b1322d5ac4760326e6fb8ec956d85775" +checksum = "34ac096ce696dc2fcabef30516bb13c0a68a11d30131d3df6f04711467681b04" [[package]] name = "arc-swap" @@ -114,9 +114,9 @@ checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4caf25cdc4a985f91df42ed9e9308e1adbcd341a31a72605c697033fcef163e3" +checksum = "c91839b07e474b3995035fd8ac33ee54f9c9ccbbb1ea33d9909c71bffdf1259d" dependencies = [ "arrow-arith", "arrow-array", @@ -135,9 +135,9 @@ dependencies = [ [[package]] name = "arrow-arith" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "91f2dfd1a7ec0aca967dfaa616096aec49779adc8eccec005e2f5e4111b1192a" +checksum = "855c57c4efd26722b044dcd3e348252560e3e0333087fb9f6479dc0bf744054f" dependencies = [ "arrow-array", "arrow-buffer", @@ -150,9 +150,9 @@ dependencies = [ [[package]] name = "arrow-array" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d39387ca628be747394890a6e47f138ceac1aa912eab64f02519fed24b637af8" +checksum = "bd03279cea46569acf9295f6224fbc370c5df184b4d2ecfe97ccb131d5615a7f" dependencies = [ "ahash", "arrow-buffer", @@ -161,15 +161,15 @@ dependencies = [ "chrono", "chrono-tz 0.10.0", "half", - "hashbrown 0.14.5", + "hashbrown 0.15.2", "num", ] [[package]] name = "arrow-buffer" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e51e05228852ffe3eb391ce7178a0f97d2cf80cc6ef91d3c4a6b3cb688049ec" +checksum = "9e4a9b9b1d6d7117f6138e13bc4dd5daa7f94e671b70e8c9c4dc37b4f5ecfc16" dependencies = [ "bytes", "half", @@ -178,9 +178,9 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d09aea56ec9fa267f3f3f6cdab67d8a9974cbba90b3aa38c8fe9d0bb071bd8c1" +checksum = "bc70e39916e60c5b7af7a8e2719e3ae589326039e1e863675a008bee5ffe90fd" dependencies = [ "arrow-array", "arrow-buffer", @@ -199,9 +199,9 @@ dependencies = [ [[package]] name = "arrow-csv" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c07b5232be87d115fde73e32f2ca7f1b353bff1b44ac422d3c6fc6ae38f11f0d" +checksum = "789b2af43c1049b03a8d088ff6b2257cdcea1756cd76b174b1f2600356771b97" dependencies = [ "arrow-array", "arrow-buffer", @@ -218,9 +218,9 @@ dependencies = [ [[package]] name = "arrow-data" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b98ae0af50890b494cebd7d6b04b35e896205c1d1df7b29a6272c5d0d0249ef5" +checksum = "e4e75edf21ffd53744a9b8e3ed11101f610e7ceb1a29860432824f1834a1f623" dependencies = [ "arrow-buffer", "arrow-schema", @@ -230,9 +230,9 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ed91bdeaff5a1c00d28d8f73466bcb64d32bbd7093b5a30156b4b9f4dba3eee" +checksum = "d186a909dece9160bf8312f5124d797884f608ef5435a36d9d608e0b2a9bcbf8" dependencies = [ "arrow-array", "arrow-buffer", @@ -245,9 +245,9 @@ dependencies = [ [[package]] name = "arrow-json" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0471f51260a5309307e5d409c9dc70aede1cd9cf1d4ff0f0a1e8e1a2dd0e0d3c" +checksum = "b66ff2fedc1222942d0bd2fd391cb14a85baa3857be95c9373179bd616753b85" dependencies = [ "arrow-array", "arrow-buffer", @@ -265,9 +265,9 @@ dependencies = [ [[package]] name = "arrow-ord" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2883d7035e0b600fb4c30ce1e50e66e53d8656aa729f2bfa4b51d359cf3ded52" +checksum = "ece7b5bc1180e6d82d1a60e1688c199829e8842e38497563c3ab6ea813e527fd" dependencies = [ "arrow-array", "arrow-buffer", @@ -280,9 +280,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "552907e8e587a6fde4f8843fd7a27a576a260f65dab6c065741ea79f633fc5be" +checksum = "745c114c8f0e8ce211c83389270de6fbe96a9088a7b32c2a041258a443fe83ff" dependencies = [ "ahash", "arrow-array", @@ -294,18 +294,18 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "539ada65246b949bd99ffa0881a9a15a4a529448af1a07a9838dd78617dafab1" +checksum = "b95513080e728e4cec37f1ff5af4f12c9688d47795d17cda80b6ec2cf74d4678" dependencies = [ "bitflags 2.6.0", ] [[package]] name = "arrow-select" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6259e566b752da6dceab91766ed8b2e67bf6270eb9ad8a6e07a33c1bede2b125" +checksum = "8e415279094ea70323c032c6e739c48ad8d80e78a09bef7117b8718ad5bf3722" dependencies = [ "ahash", "arrow-array", @@ -317,9 +317,9 @@ dependencies = [ [[package]] name = "arrow-string" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3179ccbd18ebf04277a095ba7321b93fd1f774f18816bd5f6b3ce2f594edb6c" +checksum = "11d956cae7002eb8d83a27dbd34daaea1cf5b75852f0b84deb4d93a276e92bbf" dependencies = [ "arrow-array", "arrow-buffer", @@ -346,7 +346,7 @@ checksum = "721cae7de5c34fbb2acd27e21e6d2cf7b886dce0c27388d46c4e6c47ea4318dd" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] @@ -385,6 +385,19 @@ version = "0.22.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "72b3254f16251a8381aa12e40e3c4d2f0199f8c6508fbecb9d91f575e0fbb8c6" +[[package]] +name = "bigdecimal" +version = "0.4.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f31f3af01c5c65a07985c804d3366560e6fa7883d640a122819b14ec327482c" +dependencies = [ + "autocfg", + "libm", + "num-bigint", + "num-integer", + "num-traits", +] + [[package]] name = "bitflags" version = "1.3.2" @@ -408,9 +421,9 @@ dependencies = [ [[package]] name = "blake3" -version = "1.5.4" +version = "1.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d82033247fd8e890df8f740e407ad4d038debb9eb1f40533fffb32e7d17dc6f7" +checksum = "b8ee0c1824c4dea5b5f81736aff91bae041d2c07ee1192bec91054e10e3e601e" dependencies = [ "arrayref", "arrayvec", @@ -457,9 +470,9 @@ checksum = "79296716171880943b8470b5f8d03aa55eb2e645a4874bdbb28adb49162e012c" [[package]] name = "bytemuck" -version = "1.19.0" +version = "1.21.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8334215b81e418a0a7bdb8ef0849474f40bb10c8b71f1c4ed315cff49f32494d" +checksum = "ef657dfab802224e671f5818e9a4935f9b1957ed18e58292690cc39e7a4092a3" [[package]] name = "byteorder" @@ -469,9 +482,9 @@ checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" [[package]] name = "bytes" -version = "1.8.0" +version = "1.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ac0150caa2ae65ca5bd83f25c7de183dea78d4d366469f148435e2acfbad0da" +checksum = "325918d6fe32f23b19878fe4b34794ae41fc19ddbe53b10571a4874d44ffd39b" [[package]] name = "cast" @@ -481,9 +494,9 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "cc" -version = "1.2.1" +version = "1.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd9de9f2205d5ef3fd67e685b0df337994ddd4495e2a28d185500d0e1edfea47" +checksum = "8d6dbb628b8f8555f86d0323c2eb39e3ec81901f4b83e091db8a6a76d316a333" dependencies = [ "jobserver", "libc", @@ -504,9 +517,9 @@ checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" [[package]] name = "chrono" -version = "0.4.38" +version = "0.4.39" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a21f936df1771bf62b77f047b726c4625ff2e8aa607c01ec06e5a05bd8463401" +checksum = "7e36cc9d416881d2e24f9a963be5fb1cd90966419ac844274161d10488b3e825" dependencies = [ "android-tzdata", "iana-time-zone", @@ -586,18 +599,18 @@ dependencies = [ [[package]] name = "clap" -version = "4.5.21" +version = "4.5.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb3b4b9e5a7c7514dfa52869339ee98b3156b0bfb4e8a77c4ff4babb64b1604f" +checksum = "3135e7ec2ef7b10c6ed8950f0f792ed96ee093fa088608f1c76e569722700c84" dependencies = [ "clap_builder", ] [[package]] name = "clap_builder" -version = "4.5.21" +version = "4.5.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b17a95aa67cc7b5ebd32aa5370189aa0d79069ef1c64ce893bd30fb24bff20ec" +checksum = "30582fc632330df2bd26877bde0c1f4470d57c582bbc070376afcd04d8cb4838" dependencies = [ "anstyle", "clap_lex", @@ -605,9 +618,9 @@ dependencies = [ [[package]] name = "clap_lex" -version = "0.7.3" +version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "afb84c814227b90d6895e01398aee0d8033c00e7466aca416fb6a8e0eb19d8a7" +checksum = "f46ad14479a25103f283c0f10005961cf086d8dc42205bb44c46ac563475dca6" [[package]] name = "combine" @@ -673,9 +686,9 @@ dependencies = [ [[package]] name = "cpufeatures" -version = "0.2.15" +version = "0.2.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ca741a962e1b0bff6d724a1a0958b686406e853bb14061f218562e1896f95e6" +checksum = "16b80225097f2e5ae4e7179dd2266824648f3e2f49d9134d584b76389d31c4c3" dependencies = [ "libc", ] @@ -729,9 +742,9 @@ dependencies = [ [[package]] name = "crossbeam-deque" -version = "0.8.5" +version = "0.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "613f8cc01fe9cf1a3eb3d7f488fd2fa8388403e97039e2f73692932e291a770d" +checksum = "9dd111b7b7f7d55b72c0a6ae361660ee5853c9af73f70c3c2ef6858b950e2e51" dependencies = [ "crossbeam-epoch", "crossbeam-utils", @@ -748,9 +761,9 @@ dependencies = [ [[package]] name = "crossbeam-utils" -version = "0.8.20" +version = "0.8.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22ec99545bb0ed0ea7bb9b8e1e9122ea386ff8a48c0922e43f36d45ab09e0e80" +checksum = "d0a5c400df2834b80a4c3327b3aad3a4c4cd4de0629063962b03235697506a28" [[package]] name = "crunchy" @@ -805,11 +818,9 @@ dependencies = [ [[package]] name = "datafusion" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cbba0799cf6913b456ed07a94f0f3b6e12c62a5d88b10809e2284a0f2b915c05" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ - "ahash", "arrow", "arrow-array", "arrow-ipc", @@ -825,6 +836,7 @@ dependencies = [ "datafusion-expr", "datafusion-functions", "datafusion-functions-aggregate", + "datafusion-functions-table", "datafusion-functions-window", "datafusion-optimizer", "datafusion-physical-expr", @@ -834,18 +846,13 @@ dependencies = [ "datafusion-sql", "futures", "glob", - "half", - "hashbrown 0.14.5", - "indexmap", "itertools 0.13.0", "log", - "num_cpus", "object_store", "parking_lot", "parquet", - "paste", - "pin-project-lite", "rand", + "regex", "sqlparser", "tempfile", "tokio", @@ -855,9 +862,8 @@ dependencies = [ [[package]] name = "datafusion-catalog" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7493c5c2d40eec435b13d92e5703554f4efc7059451fcb8d3a79580ff0e45560" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow-schema", "async-trait", @@ -937,6 +943,7 @@ dependencies = [ "datafusion", "datafusion-common", "datafusion-expr", + "datafusion-expr-common", "datafusion-physical-expr", "futures", "num", @@ -945,57 +952,56 @@ dependencies = [ "regex", "thiserror", "tokio", - "twox-hash 2.0.1", + "twox-hash 2.1.0", ] [[package]] name = "datafusion-common" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "24953049ebbd6f8964f91f60aa3514e121b5e81e068e33b60e77815ab369b25c" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "ahash", "arrow", "arrow-array", "arrow-buffer", "arrow-schema", - "chrono", "half", "hashbrown 0.14.5", "indexmap", - "instant", "libc", - "num_cpus", + "log", "object_store", "parquet", "paste", "sqlparser", "tokio", + "web-time", ] [[package]] name = "datafusion-common-runtime" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f06df4ef76872e11c924d3c814fd2a8dd09905ed2e2195f71c857d78abd19685" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "log", "tokio", ] +[[package]] +name = "datafusion-doc" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" + [[package]] name = "datafusion-execution" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6bbdcb628d690f3ce5fea7de81642b514486d58ff9779a51f180a69a4eadb361" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow", - "chrono", "dashmap", "datafusion-common", "datafusion-expr", "futures", - "hashbrown 0.14.5", "log", "object_store", "parking_lot", @@ -1006,16 +1012,13 @@ dependencies = [ [[package]] name = "datafusion-expr" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8036495980e3131f706b7d33ab00b4492d73dc714e3cb74d11b50f9602a73246" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ - "ahash", "arrow", - "arrow-array", - "arrow-buffer", "chrono", "datafusion-common", + "datafusion-doc", "datafusion-expr-common", "datafusion-functions-aggregate-common", "datafusion-functions-window-common", @@ -1024,27 +1027,22 @@ dependencies = [ "paste", "serde_json", "sqlparser", - "strum", - "strum_macros", ] [[package]] name = "datafusion-expr-common" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4da0f3cb4669f9523b403d6b5a0ec85023e0ab3bf0183afd1517475b3e64fdd2" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow", "datafusion-common", "itertools 0.13.0", - "paste", ] [[package]] name = "datafusion-functions" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f52c4012648b34853e40a2c6bcaa8772f837831019b68aca384fb38436dba162" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow", "arrow-buffer", @@ -1053,8 +1051,11 @@ dependencies = [ "blake3", "chrono", "datafusion-common", + "datafusion-doc", "datafusion-execution", "datafusion-expr", + "datafusion-expr-common", + "datafusion-macros", "hashbrown 0.14.5", "hex", "itertools 0.13.0", @@ -1069,44 +1070,41 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e5b8bb624597ba28ed7446df4a9bd7c7a7bde7c578b6b527da3f47371d5f6741" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "ahash", "arrow", "arrow-schema", "datafusion-common", + "datafusion-doc", "datafusion-execution", "datafusion-expr", "datafusion-functions-aggregate-common", + "datafusion-macros", "datafusion-physical-expr", "datafusion-physical-expr-common", "half", - "indexmap", "log", "paste", ] [[package]] name = "datafusion-functions-aggregate-common" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6fb06208fc470bc8cf1ce2d9a1159d42db591f2c7264a8c1776b53ad8f675143" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "ahash", "arrow", "datafusion-common", "datafusion-expr-common", "datafusion-physical-expr-common", - "rand", ] [[package]] name = "datafusion-functions-nested" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fca25bbb87323716d05e54114666e942172ccca23c5a507e9c7851db6e965317" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow", "arrow-array", @@ -1122,18 +1120,33 @@ dependencies = [ "itertools 0.13.0", "log", "paste", - "rand", +] + +[[package]] +name = "datafusion-functions-table" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" +dependencies = [ + "arrow", + "async-trait", + "datafusion-catalog", + "datafusion-common", + "datafusion-expr", + "datafusion-physical-plan", + "parking_lot", + "paste", ] [[package]] name = "datafusion-functions-window" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5ae23356c634e54c59f7c51acb7a5b9f6240ffb2cf997049a1a24a8a88598dbe" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "datafusion-common", + "datafusion-doc", "datafusion-expr", "datafusion-functions-window-common", + "datafusion-macros", "datafusion-physical-expr", "datafusion-physical-expr-common", "log", @@ -1142,48 +1155,49 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d4b3d6ff7794acea026de36007077a06b18b89e4f9c3fea7f2215f9f7dd9059b" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", ] +[[package]] +name = "datafusion-macros" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" +dependencies = [ + "quote", + "syn 2.0.92", +] + [[package]] name = "datafusion-optimizer" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bec6241eb80c595fa0e1a8a6b69686b5cf3bd5fdacb8319582a0943b0bd788aa" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow", - "async-trait", "chrono", "datafusion-common", "datafusion-expr", "datafusion-physical-expr", - "hashbrown 0.14.5", "indexmap", "itertools 0.13.0", "log", - "paste", + "regex", "regex-syntax", ] [[package]] name = "datafusion-physical-expr" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3370357b8fc75ec38577700644e5d1b0bc78f38babab99c0b8bd26bafb3e4335" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "ahash", "arrow", "arrow-array", "arrow-buffer", - "arrow-ord", "arrow-schema", - "arrow-string", - "chrono", "datafusion-common", "datafusion-expr", "datafusion-expr-common", @@ -1200,39 +1214,36 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b8b7734d94bf2fa6f6e570935b0ddddd8421179ce200065be97874e13d46a47b" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "ahash", "arrow", "datafusion-common", "datafusion-expr-common", "hashbrown 0.14.5", - "rand", + "itertools 0.13.0", ] [[package]] name = "datafusion-physical-optimizer" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7eee8c479522df21d7b395640dff88c5ed05361852dce6544d7c98e9dbcebffe" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow", - "arrow-schema", "datafusion-common", "datafusion-execution", "datafusion-expr-common", "datafusion-physical-expr", "datafusion-physical-plan", "itertools 0.13.0", + "log", ] [[package]] name = "datafusion-physical-plan" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17e1fc2e2c239d14e8556f2622b19a726bf6bc6962cc00c71fc52626274bee24" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "ahash", "arrow", @@ -1246,7 +1257,6 @@ dependencies = [ "datafusion-common-runtime", "datafusion-execution", "datafusion-expr", - "datafusion-functions-aggregate-common", "datafusion-functions-window-common", "datafusion-physical-expr", "datafusion-physical-expr-common", @@ -1256,29 +1266,26 @@ dependencies = [ "indexmap", "itertools 0.13.0", "log", - "once_cell", "parking_lot", "pin-project-lite", - "rand", "tokio", ] [[package]] name = "datafusion-sql" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "63e3a4ed41dbee20a5d947a59ca035c225d67dc9cbe869c10f66dcdf25e7ce51" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow", "arrow-array", "arrow-schema", + "bigdecimal", "datafusion-common", "datafusion-expr", "indexmap", "log", "regex", "sqlparser", - "strum", ] [[package]] @@ -1326,7 +1333,7 @@ checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] @@ -1343,19 +1350,19 @@ checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" [[package]] name = "errno" -version = "0.3.9" +version = "0.3.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "534c5cf6194dfab3db3242765c03bbe257cf92f22b38f6bc0c58d59108a820ba" +checksum = "33d852cb9b869c2a9b3df2f71a3074817f01e1844f839a144f5fcef059a4eb5d" dependencies = [ "libc", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] name = "fastrand" -version = "2.2.0" +version = "2.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "486f806e73c5707928240ddc295403b1b93c96a02038563881c4a2fd84b81ac4" +checksum = "37909eebbb50d72f9059c3b6d82c0463f2ff062c9e95845c43a6c9c0355411be" [[package]] name = "findshlibs" @@ -1377,9 +1384,9 @@ checksum = "0ce7134b9999ecaf8bcd65542e436736ef32ddca1b3e06094cb6ec5755203b80" [[package]] name = "flatbuffers" -version = "24.3.25" +version = "24.12.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8add37afff2d4ffa83bc748a70b4b1370984f6980768554182424ef71447c35f" +checksum = "4f1baf0dbf96932ec9a3038d57900329c015b0bfb7b63d904f3bc27e2b02a096" dependencies = [ "bitflags 1.3.2", "rustc_version", @@ -1466,7 +1473,7 @@ checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] @@ -1555,9 +1562,9 @@ dependencies = [ [[package]] name = "hashbrown" -version = "0.15.1" +version = "0.15.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3a9bfc1af68b1726ea47d3d5109de126281def866b33970e10fbab11b5dafab3" +checksum = "bf151400ff0baff5465007dd2f3e717f3fe502074ca563069ce3a6629d07b289" [[package]] name = "heck" @@ -1574,12 +1581,6 @@ version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2304e00983f87ffb38b55b444b5e3b60a884b5d30c0fca7d82fe33449bbe55ea" -[[package]] -name = "hermit-abi" -version = "0.3.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d231dfb89cfffdbc30e7fc41579ed6066ad03abda9e567ccafae602b97ec5024" - [[package]] name = "hermit-abi" version = "0.4.0" @@ -1594,11 +1595,11 @@ checksum = "7f24254aa9a54b5c858eaee2f5bccdb46aaf0e486a595ed5fd8f86ba55232a70" [[package]] name = "home" -version = "0.5.9" +version = "0.5.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e3d1354bf6b7235cb4a0576c2619fd4ed18183f689b12b006a0ee7329eeff9a5" +checksum = "589533453244b0995c858700322199b2becb13b627df2851f64a2775d024abcf" dependencies = [ - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -1745,7 +1746,7 @@ checksum = "1ec89e9337638ecdc08744df490b221a7399bf8d164eb52a665454e60e075ad6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] @@ -1771,12 +1772,12 @@ dependencies = [ [[package]] name = "indexmap" -version = "2.6.0" +version = "2.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "707907fe3c25f5424cce2cb7e1cbcafee6bdbe735ca90ef77c29e84591e5b9da" +checksum = "62f822373a4fe84d4bb149bf54e584a7f4abec90e072ed49cda0edea5b95471f" dependencies = [ "equivalent", - "hashbrown 0.15.1", + "hashbrown 0.15.2", ] [[package]] @@ -1797,18 +1798,6 @@ dependencies = [ "str_stack", ] -[[package]] -name = "instant" -version = "0.1.13" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e0242819d153cba4b4b05a5a8f2a7e9bbf97b6055b2a002b395c96b5ff3c0222" -dependencies = [ - "cfg-if", - "js-sys", - "wasm-bindgen", - "web-sys", -] - [[package]] name = "integer-encoding" version = "3.0.4" @@ -1821,7 +1810,7 @@ version = "0.4.13" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "261f68e344040fbd0edea105bef17c66edf46f984ddb1115b775ce31be948f4b" dependencies = [ - "hermit-abi 0.4.0", + "hermit-abi", "libc", "windows-sys 0.52.0", ] @@ -1864,9 +1853,9 @@ dependencies = [ [[package]] name = "itoa" -version = "1.0.11" +version = "1.0.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49f1f14873335454500d59611f1cf4a4b0f786f9ac11f4312a78e4cf2566695b" +checksum = "d75a2a4b1b190afb6f5425f10f6a8f959d2ea0b9c2b1d79553551850539e4674" [[package]] name = "java-locator" @@ -1912,10 +1901,11 @@ dependencies = [ [[package]] name = "js-sys" -version = "0.3.72" +version = "0.3.76" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a88f1bda2bd75b0452a14784937d796722fdebfe50df998aeb3f0b7603019a9" +checksum = "6717b6b5b077764fb5966237269cb3c64edddde4b14ce42647430a78ced9e7b7" dependencies = [ + "once_cell", "wasm-bindgen", ] @@ -1927,9 +1917,9 @@ checksum = "bbd2bcb4c963f2ddae06a2efc7e9f3591312473c50c6685e1f298068316e66fe" [[package]] name = "lexical-core" -version = "1.0.2" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0431c65b318a590c1de6b8fd6e72798c92291d27762d94c9e6c37ed7a73d8458" +checksum = "b765c31809609075565a70b4b71402281283aeda7ecaf4818ac14a7b2ade8958" dependencies = [ "lexical-parse-float", "lexical-parse-integer", @@ -1940,9 +1930,9 @@ dependencies = [ [[package]] name = "lexical-parse-float" -version = "1.0.2" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb17a4bdb9b418051aa59d41d65b1c9be5affab314a872e5ad7f06231fb3b4e0" +checksum = "de6f9cb01fb0b08060209a057c048fcbab8717b4c1ecd2eac66ebfe39a65b0f2" dependencies = [ "lexical-parse-integer", "lexical-util", @@ -1951,9 +1941,9 @@ dependencies = [ [[package]] name = "lexical-parse-integer" -version = "1.0.2" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5df98f4a4ab53bf8b175b363a34c7af608fe31f93cc1fb1bf07130622ca4ef61" +checksum = "72207aae22fc0a121ba7b6d479e42cbfea549af1479c3f3a4f12c70dd66df12e" dependencies = [ "lexical-util", "static_assertions", @@ -1961,18 +1951,18 @@ dependencies = [ [[package]] name = "lexical-util" -version = "1.0.3" +version = "1.0.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "85314db53332e5c192b6bca611fb10c114a80d1b831ddac0af1e9be1b9232ca0" +checksum = "5a82e24bf537fd24c177ffbbdc6ebcc8d54732c35b50a3f28cc3f4e4c949a0b3" dependencies = [ "static_assertions", ] [[package]] name = "lexical-write-float" -version = "1.0.2" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e7c3ad4e37db81c1cbe7cf34610340adc09c322871972f74877a712abc6c809" +checksum = "c5afc668a27f460fb45a81a757b6bf2f43c2d7e30cb5a2dcd3abf294c78d62bd" dependencies = [ "lexical-util", "lexical-write-integer", @@ -1981,9 +1971,9 @@ dependencies = [ [[package]] name = "lexical-write-integer" -version = "1.0.2" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb89e9f6958b83258afa3deed90b5de9ef68eef090ad5086c791cd2345610162" +checksum = "629ddff1a914a836fb245616a7888b62903aae58fa771e1d83943035efa0f978" dependencies = [ "lexical-util", "static_assertions", @@ -1991,9 +1981,9 @@ dependencies = [ [[package]] name = "libc" -version = "0.2.162" +version = "0.2.169" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "18d287de67fe55fd7e1581fe933d965a5a9477b38e949cfa9f8574ef01506398" +checksum = "b5aba8db14291edd000dfcc4d620c7ebfb122c613afb886ca8803fa4e128a20a" [[package]] name = "libloading" @@ -2029,9 +2019,9 @@ checksum = "78b3ae25bc7c8c38cec158d1f2757ee79e9b3740fbc7ccf0e59e4b08d793fa89" [[package]] name = "litemap" -version = "0.7.3" +version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "643cb0b8d4fcc284004d5fd0d67ccf61dfffadb7f75e1e71bc420f4688a3a704" +checksum = "4ee93343901ab17bd981295f2cf0026d4ad018c7c31ba84549a4ddbb47a45104" [[package]] name = "lock_api" @@ -2131,9 +2121,9 @@ dependencies = [ [[package]] name = "miniz_oxide" -version = "0.8.0" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2d80299ef12ff69b16a84bb182e3b9df68b5a91574d3d4fa6e41b65deec4df1" +checksum = "4ffbe83022cedc1d264172192511ae958937694cd57ce297164951b8b3568394" dependencies = [ "adler2", ] @@ -2239,30 +2229,20 @@ dependencies = [ "libm", ] -[[package]] -name = "num_cpus" -version = "1.16.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4161fcb6d602d4d2081af7c3a45852d875a03dd337a6bfdd6e06407b61342a43" -dependencies = [ - "hermit-abi 0.3.9", - "libc", -] - [[package]] name = "object" -version = "0.36.5" +version = "0.36.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aedf0a2d09c573ed1d8d85b30c119153926a2b36dce0ab28322c09a117a4683e" +checksum = "62948e14d923ea95ea2c7c86c71013138b66525b86bdc08d2dcc262bdb497b87" dependencies = [ "memchr", ] [[package]] name = "object_store" -version = "0.11.1" +version = "0.11.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6eb4c22c6154a1e759d7099f9ffad7cc5ef8245f9efbab4a41b92623079c82f3" +checksum = "3cfccb68961a56facde1163f9319e0d15743352344e7808a11795fb99698dcaf" dependencies = [ "async-trait", "bytes", @@ -2325,9 +2305,9 @@ dependencies = [ [[package]] name = "parquet" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dea02606ba6f5e856561d8d507dba8bac060aefca2a6c0f1aa1d361fed91ff3e" +checksum = "2b449890367085eb65d7d3321540abc3d7babbd179ce31df0016e90719114191" dependencies = [ "ahash", "arrow-array", @@ -2344,7 +2324,7 @@ dependencies = [ "flate2", "futures", "half", - "hashbrown 0.14.5", + "hashbrown 0.15.2", "lz4_flex", "num", "num-bigint", @@ -2506,9 +2486,9 @@ dependencies = [ [[package]] name = "proc-macro2" -version = "1.0.89" +version = "1.0.92" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f139b0662de085916d1fb67d2b4169d1addddda1919e696f3252b740b629986e" +checksum = "37d3544b3f2748c54e147655edb5025752e2303145b5aefb3c3ea2c78b973bb0" dependencies = [ "unicode-ident", ] @@ -2576,7 +2556,7 @@ dependencies = [ "itertools 0.12.1", "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] @@ -2600,9 +2580,9 @@ dependencies = [ [[package]] name = "quote" -version = "1.0.37" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b5b9d34b8991d19d98081b46eacdd8eb58c6f2b201139f7c5f643cc155a633af" +checksum = "0e4dccaaaf89514f546c693ddc140f729f958c247918a13380cccc6078391acc" dependencies = [ "proc-macro2", ] @@ -2659,9 +2639,9 @@ dependencies = [ [[package]] name = "redox_syscall" -version = "0.5.7" +version = "0.5.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b6dfecf2c74bce2466cabf93f6664d6998a69eb21e39f4207930065b27b771f" +checksum = "03a862b389f93e68874fbf580b9de08dd02facb9a788ebadaf4a3fd33cf58834" dependencies = [ "bitflags 2.6.0", ] @@ -2721,22 +2701,22 @@ dependencies = [ [[package]] name = "rustix" -version = "0.38.40" +version = "0.38.42" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "99e4ea3e1cdc4b559b8e5650f9c8e5998e3e5c1343b4eaf034565f32318d63c0" +checksum = "f93dc38ecbab2eb790ff964bb77fa94faf256fd3e73285fd7ba0903b76bedb85" dependencies = [ "bitflags 2.6.0", "errno", "libc", "linux-raw-sys", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] name = "rustversion" -version = "1.0.18" +version = "1.0.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0e819f2bc632f285be6d7cd36e25940d45b2391dd6d9b939e79de557f7014248" +checksum = "f7c45b9784283f1b2e7fb61b42047c2fd678ef0960d4f6f1eba131594cc369d4" [[package]] name = "ryu" @@ -2761,9 +2741,9 @@ checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" [[package]] name = "semver" -version = "1.0.23" +version = "1.0.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "61697e0a1c7e512e84a621326239844a24d8207b4669b41bc18b32ea5cbf988b" +checksum = "3cb6eb87a131f756572d7fb904f6e7b68633f09cca868c5df1c4b8d1a694bbba" [[package]] name = "seq-macro" @@ -2773,9 +2753,9 @@ checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" [[package]] name = "serde" -version = "1.0.215" +version = "1.0.217" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6513c1ad0b11a9376da888e3e0baa0077f1aed55c17f50e7b2397136129fb88f" +checksum = "02fc4265df13d6fa1d00ecff087228cc0a2b5f3c0e87e258d8b94a156e984c70" dependencies = [ "serde_derive", ] @@ -2792,20 +2772,20 @@ dependencies = [ [[package]] name = "serde_derive" -version = "1.0.215" +version = "1.0.217" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ad1e866f866923f252f05c889987993144fb74e722403468a4ebd70c3cd756c0" +checksum = "5a9bf7cf98d04a2b28aead066b7496853d4779c9cc183c440dbac457641e19a0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] name = "serde_json" -version = "1.0.132" +version = "1.0.134" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d726bfaff4b320266d395898905d0eba0345aae23b54aee3a737e260fd46db03" +checksum = "d00f4175c42ee48b15416f6193a959ba3a0d67fc699a0db9ad12df9f83991c7d" dependencies = [ "itoa", "memchr", @@ -2888,7 +2868,7 @@ dependencies = [ "heck 0.5.0", "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] @@ -2899,9 +2879,9 @@ checksum = "1b6b67fb9a61334225b5b790716f609cd58395f895b3fe8b328786812a40bc3b" [[package]] name = "sqlparser" -version = "0.51.0" +version = "0.53.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5fe11944a61da0da3f592e19a45ebe5ab92dc14a779907ff1f08fbb797bfefc7" +checksum = "05a528114c392209b3264855ad491fcce534b94a38771b0a0b97a79379275ce8" dependencies = [ "log", "sqlparser_derive", @@ -2909,13 +2889,13 @@ dependencies = [ [[package]] name = "sqlparser_derive" -version = "0.2.2" +version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" +checksum = "da5fc6819faabb412da764b99d3b713bb55083c11e7e0c00144d386cd6a1939c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] @@ -2941,9 +2921,6 @@ name = "strum" version = "0.26.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8fec0f0aef304996cf250b31b5a10dee7980c85da9d759361292b8bca5a18f06" -dependencies = [ - "strum_macros", -] [[package]] name = "strum_macros" @@ -2955,7 +2932,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] @@ -2966,9 +2943,9 @@ checksum = "13c2bddecc57b384dee18652358fb23172facb8a2c51ccc10d74c157bdea3292" [[package]] name = "symbolic-common" -version = "12.12.1" +version = "12.12.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3d4d73159efebfb389d819fd479afb2dbd57dcb3e3f4b7fcfa0e675f5a46c1cb" +checksum = "cd33e73f154e36ec223c18013f7064a2c120f1162fc086ac9933542def186b00" dependencies = [ "debugid", "memmap2", @@ -2978,9 +2955,9 @@ dependencies = [ [[package]] name = "symbolic-demangle" -version = "12.12.1" +version = "12.12.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a767859f6549c665011970874c3f541838b4835d5aaaa493d3ee383918be9f10" +checksum = "89e51191290147f071777e37fe111800bb82a9059f9c95b19d2dd41bfeddf477" dependencies = [ "cpp_demangle", "rustc-demangle", @@ -3000,9 +2977,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.87" +version = "2.0.92" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "25aa4ce346d03a6dcd68dd8b4010bcb74e54e62c90c573f394c46eae99aba32d" +checksum = "70ae51629bf965c5c098cc9e87908a3df5301051a9e087d6f9bef5c9771ed126" dependencies = [ "proc-macro2", "quote", @@ -3017,7 +2994,7 @@ checksum = "c8af7666ab7b6390ab78131fb5b0fce11d6b7a6951602017c35fa82800708971" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] @@ -3050,7 +3027,7 @@ checksum = "4fee6c4efc90059e10f81e6d42c60a18f76588c3d74cb83a0b242a2b6c7504c1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] @@ -3105,9 +3082,9 @@ dependencies = [ [[package]] name = "tokio" -version = "1.41.1" +version = "1.42.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22cfb5bee7a6a52939ca9224d6ac897bb669134078daa8735560897f69de4d33" +checksum = "5cec9b21b0450273377fc97bd4c33a8acffc8c996c987a7c5b319a0083707551" dependencies = [ "backtrace", "bytes", @@ -3123,14 +3100,14 @@ checksum = "693d596312e88961bc67d7f1f97af8a70227d9f90c31bba5806eec004978d752" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] name = "tracing" -version = "0.1.40" +version = "0.1.41" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3523ab5a71916ccf420eebdf5521fcef02141234bbc0b8a49f2fdc4544364ef" +checksum = "784e0ac535deb450455cbfa28a6f0df145ea1bb7ae51b821cf5e7927fdcfbdd0" dependencies = [ "pin-project-lite", "tracing-attributes", @@ -3139,20 +3116,20 @@ dependencies = [ [[package]] name = "tracing-attributes" -version = "0.1.27" +version = "0.1.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" +checksum = "395ae124c09f9e6918a2310af6038fba074bcf474ac352496d5910dd59a2226d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] name = "tracing-core" -version = "0.1.32" +version = "0.1.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c06d3da6113f116aaee68e4d601191614c9053067f9ab7f6edbcb161237daa54" +checksum = "e672c95779cf947c5311f83787af4fa8fffd12fb27e4993211a84bdfd9610f9c" dependencies = [ "once_cell", ] @@ -3169,9 +3146,9 @@ dependencies = [ [[package]] name = "twox-hash" -version = "2.0.1" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a6db6856664807f43c17fbaf2718e2381ac1476a449aa104f5f64622defa1245" +checksum = "e7b17f197b3050ba473acf9181f7b1d3b66d1cf7356c6cc57886662276e65908" dependencies = [ "rand", ] @@ -3193,9 +3170,9 @@ checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" [[package]] name = "unicode-ident" -version = "1.0.13" +version = "1.0.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e91b56cd4cadaeb79bbf1a5645f6b4f8dc5bde8834ad5894a8db35fda9efa1fe" +checksum = "adb9e6ca4f869e1180728b7950e35922a7fc6397f7b641499e8f3ef06e50dc83" [[package]] name = "unicode-segmentation" @@ -3226,9 +3203,9 @@ checksum = "673aac59facbab8a9007c7f6108d11f63b603f7cabff99fabf650fea5c32b861" [[package]] name = "url" -version = "2.5.3" +version = "2.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d157f1b96d14500ffdc1f10ba712e780825526c03d9a49b4d0324b0d9113ada" +checksum = "32f8b686cadd1473f4bd0117a5d28d36b1ade384ea9b5069a1c40aefed7fda60" dependencies = [ "form_urlencoded", "idna", @@ -3280,9 +3257,9 @@ checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" [[package]] name = "wasm-bindgen" -version = "0.2.95" +version = "0.2.99" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "128d1e363af62632b8eb57219c8fd7877144af57558fb2ef0368d0087bddeb2e" +checksum = "a474f6281d1d70c17ae7aa6a613c87fce69a127e2624002df63dcb39d6cf6396" dependencies = [ "cfg-if", "once_cell", @@ -3291,24 +3268,23 @@ dependencies = [ [[package]] name = "wasm-bindgen-backend" -version = "0.2.95" +version = "0.2.99" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cb6dd4d3ca0ddffd1dd1c9c04f94b868c37ff5fac97c30b97cff2d74fce3a358" +checksum = "5f89bb38646b4f81674e8f5c3fb81b562be1fd936d84320f3264486418519c79" dependencies = [ "bumpalo", "log", - "once_cell", "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-macro" -version = "0.2.95" +version = "0.2.99" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e79384be7f8f5a9dd5d7167216f022090cf1f9ec128e6e6a482a2cb5c5422c56" +checksum = "2cc6181fd9a7492eef6fef1f33961e3695e4579b9872a6f7c83aee556666d4fe" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -3316,28 +3292,38 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.95" +version = "0.2.99" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26c6ab57572f7a24a4985830b120de1594465e5d500f24afe89e16b4e833ef68" +checksum = "30d7a95b763d3c45903ed6c81f156801839e5ee968bb07e534c44df0fcd330c2" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", "wasm-bindgen-backend", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-shared" -version = "0.2.95" +version = "0.2.99" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "65fc09f10666a9f147042251e0dda9c18f166ff7de300607007e96bdebc1068d" +checksum = "943aab3fdaaa029a6e0271b35ea10b72b943135afe9bffca82384098ad0e06a6" [[package]] name = "web-sys" -version = "0.3.72" +version = "0.3.76" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f6488b90108c040df0fe62fa815cbdee25124641df01814dd7282749234c6112" +checksum = "04dd7223427d52553d3702c004d3b2fe07c148165faa56313cb00211e31c12bc" +dependencies = [ + "js-sys", + "wasm-bindgen", +] + +[[package]] +name = "web-time" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a6580f308b1fad9207618087a65c04e7a10bc77e02c8e84e9b00dd4b12fa0bb" dependencies = [ "js-sys", "wasm-bindgen", @@ -3557,9 +3543,9 @@ checksum = "1e9df38ee2d2c3c5948ea468a8406ff0db0b29ae1ffde1bcf20ef305bcc95c51" [[package]] name = "yoke" -version = "0.7.4" +version = "0.7.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c5b1314b079b0930c31e3af543d8ee1757b1951ae1e1565ec704403a7240ca5" +checksum = "120e6aef9aa629e3d4f52dc8cc43a015c7724194c97dfaf45180d2daf2b77f40" dependencies = [ "serde", "stable_deref_trait", @@ -3569,13 +3555,13 @@ dependencies = [ [[package]] name = "yoke-derive" -version = "0.7.4" +version = "0.7.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "28cc31741b18cb6f1d5ff12f5b7523e3d6eb0852bbbad19d73905511d9849b95" +checksum = "2380878cad4ac9aac1e2435f3eb4020e8374b5f13c296cb75b4620ff8e229154" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", "synstructure", ] @@ -3597,27 +3583,27 @@ checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] name = "zerofrom" -version = "0.1.4" +version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "91ec111ce797d0e0784a1116d0ddcdbea84322cd79e5d5ad173daeba4f93ab55" +checksum = "cff3ee08c995dee1859d998dea82f7374f2826091dd9cd47def953cae446cd2e" dependencies = [ "zerofrom-derive", ] [[package]] name = "zerofrom-derive" -version = "0.1.4" +version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ea7b4a3637ea8669cedf0f1fd5c286a17f3de97b8dd5a70a6c167a1730e63a5" +checksum = "595eed982f7d355beb85837f651fa22e90b3c044842dc7f2c2842c086f295808" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", "synstructure", ] @@ -3640,7 +3626,7 @@ checksum = "6eafa6dfb17584ea3e2bd6e76e0cc15ad7af12b09abdd1ca55961bed9b1063c6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] diff --git a/native/Cargo.toml b/native/Cargo.toml index bd46cf0c9f..cf4921ebed 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -33,20 +33,21 @@ edition = "2021" rust-version = "1.79" [workspace.dependencies] -arrow = { version = "53.2.0", features = ["prettyprint", "ffi", "chrono-tz"] } -arrow-array = { version = "53.2.0" } -arrow-buffer = { version = "53.2.0" } -arrow-data = { version = "53.2.0" } -arrow-schema = { version = "53.2.0" } -parquet = { version = "53.2.0", default-features = false, features = ["experimental"] } -datafusion = { version = "43.0.0", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } -datafusion-common = { version = "43.0.0" } -datafusion-functions = { version = "43.0.0", features = ["crypto_expressions"] } -datafusion-functions-nested = { version = "43.0.0", default-features = false } -datafusion-expr = { version = "43.0.0", default-features = false } -datafusion-execution = { version = "43.0.0", default-features = false } -datafusion-physical-plan = { version = "43.0.0", default-features = false } -datafusion-physical-expr = { version = "43.0.0", default-features = false } +arrow = { version = "53.3.0", features = ["prettyprint", "ffi", "chrono-tz"] } +arrow-array = { version = "53.3.0" } +arrow-buffer = { version = "53.3.0" } +arrow-data = { version = "53.3.0" } +arrow-schema = { version = "53.3.0" } +parquet = { version = "53.3.0", default-features = false, features = ["experimental"] } +datafusion = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } +datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false } +datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false, features = ["crypto_expressions"] } +datafusion-functions-nested = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false } +datafusion-expr = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false } +datafusion-expr-common = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false } +datafusion-execution = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false } +datafusion-physical-plan = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false } +datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false } datafusion-comet-spark-expr = { path = "spark-expr", version = "0.5.0" } datafusion-comet-proto = { path = "proto", version = "0.5.0" } chrono = { version = "0.4", default-features = false, features = ["clock"] } diff --git a/native/core/src/execution/expressions/bloom_filter_might_contain.rs b/native/core/src/execution/expressions/bloom_filter_might_contain.rs index af6a5a47a3..b14fab62f6 100644 --- a/native/core/src/execution/expressions/bloom_filter_might_contain.rs +++ b/native/core/src/execution/expressions/bloom_filter_might_contain.rs @@ -19,26 +19,37 @@ use crate::{execution::util::spark_bloom_filter::SparkBloomFilter, parquet::data use arrow::record_batch::RecordBatch; use arrow_array::cast::as_primitive_array; use arrow_schema::{DataType, Schema}; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion::physical_plan::ColumnarValue; use datafusion_common::{internal_err, Result, ScalarValue}; use datafusion_physical_expr::PhysicalExpr; -use std::{ - any::Any, - fmt::Display, - hash::{Hash, Hasher}, - sync::Arc, -}; +use std::hash::Hash; +use std::{any::Any, fmt::Display, sync::Arc}; /// A physical expression that checks if a value might be in a bloom filter. It corresponds to the /// Spark's `BloomFilterMightContain` expression. -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct BloomFilterMightContain { pub bloom_filter_expr: Arc, pub value_expr: Arc, bloom_filter: Option, } +impl Hash for BloomFilterMightContain { + fn hash(&self, state: &mut H) { + self.bloom_filter_expr.hash(state); + self.value_expr.hash(state); + self.bloom_filter.hash(state); + } +} + +impl PartialEq for BloomFilterMightContain { + fn eq(&self, other: &Self) -> bool { + self.bloom_filter_expr.eq(&other.bloom_filter_expr) + && self.value_expr.eq(&other.value_expr) + && self.bloom_filter.eq(&other.bloom_filter) + } +} + impl Display for BloomFilterMightContain { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { write!( @@ -49,18 +60,6 @@ impl Display for BloomFilterMightContain { } } -impl PartialEq for BloomFilterMightContain { - fn eq(&self, _other: &dyn Any) -> bool { - down_cast_any_ref(_other) - .downcast_ref::() - .map(|other| { - self.bloom_filter_expr.eq(&other.bloom_filter_expr) - && self.value_expr.eq(&other.value_expr) - }) - .unwrap_or(false) - } -} - fn evaluate_bloom_filter( bloom_filter_expr: &Arc, ) -> Result> { @@ -141,11 +140,4 @@ impl PhysicalExpr for BloomFilterMightContain { Arc::clone(&children[1]), )?)) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.bloom_filter_expr.hash(&mut s); - self.value_expr.hash(&mut s); - self.hash(&mut s); - } } diff --git a/native/core/src/execution/expressions/subquery.rs b/native/core/src/execution/expressions/subquery.rs index 3eeb29c16e..d933a6096a 100644 --- a/native/core/src/execution/expressions/subquery.rs +++ b/native/core/src/execution/expressions/subquery.rs @@ -22,7 +22,6 @@ use crate::{ use arrow_array::RecordBatch; use arrow_schema::{DataType, Schema, TimeUnit}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{internal_err, ScalarValue}; use datafusion_physical_expr::PhysicalExpr; use jni::{ @@ -32,11 +31,11 @@ use jni::{ use std::{ any::Any, fmt::{Display, Formatter}, - hash::{Hash, Hasher}, + hash::Hash, sync::Arc, }; -#[derive(Debug, Hash)] +#[derive(Debug, Hash, PartialEq, Eq)] pub struct Subquery { /// The ID of the execution context that owns this subquery. We use this ID to retrieve the /// subquery result. @@ -63,19 +62,6 @@ impl Display for Subquery { } } -impl PartialEq for Subquery { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.id.eq(&x.id) - && self.data_type.eq(&x.data_type) - && self.exec_context_id.eq(&x.exec_context_id) - }) - .unwrap_or(false) - } -} - impl PhysicalExpr for Subquery { fn as_any(&self) -> &dyn Any { self @@ -209,9 +195,4 @@ impl PhysicalExpr for Subquery { ) -> datafusion_common::Result> { Ok(self) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.hash(&mut s) - } } diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index 2c1a55f48f..09caf5e279 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -20,10 +20,7 @@ use arrow::datatypes::DataType as ArrowDataType; use arrow_array::RecordBatch; use datafusion::{ - execution::{ - disk_manager::DiskManagerConfig, - runtime_env::{RuntimeConfig, RuntimeEnv}, - }, + execution::{disk_manager::DiskManagerConfig, runtime_env::RuntimeEnv}, physical_plan::{display::DisplayableExecutionPlan, SendableRecordBatchStream}, prelude::{SessionConfig, SessionContext}, }; @@ -52,6 +49,7 @@ use crate::{ }; use datafusion_comet_proto::spark_operator::Operator; use datafusion_common::ScalarValue; +use datafusion_execution::runtime_env::RuntimeEnvBuilder; use futures::stream::StreamExt; use jni::{ objects::GlobalRef, @@ -188,7 +186,7 @@ fn prepare_datafusion_session_context( memory_fraction: f64, comet_task_memory_manager: Arc, ) -> CometResult { - let mut rt_config = RuntimeConfig::new().with_disk_manager(DiskManagerConfig::NewOs); + let mut rt_config = RuntimeEnvBuilder::new().with_disk_manager(DiskManagerConfig::NewOs); // Check if we are using unified memory manager integrated with Spark. if use_unified_memory_manager { @@ -216,6 +214,7 @@ fn prepare_datafusion_session_context( &ScalarValue::Float64(Some(1.1)), ); + #[allow(deprecated)] let runtime = RuntimeEnv::try_new(rt_config)?; let mut session_ctx = SessionContext::new_with_config_rt(session_config, Arc::new(runtime)); diff --git a/native/core/src/execution/operators/copy.rs b/native/core/src/execution/operators/copy.rs index 8eeda8a5ad..cec00eb28c 100644 --- a/native/core/src/execution/operators/copy.rs +++ b/native/core/src/execution/operators/copy.rs @@ -30,6 +30,7 @@ use arrow_array::{ use arrow_data::transform::MutableArrayData; use arrow_schema::{ArrowError, DataType, Field, FieldRef, Schema, SchemaRef}; +use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::physical_plan::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use datafusion::{execution::TaskContext, physical_expr::*, physical_plan::*}; use datafusion_common::{arrow_datafusion_err, DataFusionError, Result as DataFusionResult}; @@ -78,7 +79,8 @@ impl CopyExec { let cache = PlanProperties::new( EquivalenceProperties::new(Arc::clone(&schema)), Partitioning::UnknownPartitioning(1), - ExecutionMode::Bounded, + EmissionType::Final, + Boundedness::Bounded, ); Self { diff --git a/native/core/src/execution/operators/expand.rs b/native/core/src/execution/operators/expand.rs index fb43a6e49f..f75822d408 100644 --- a/native/core/src/execution/operators/expand.rs +++ b/native/core/src/execution/operators/expand.rs @@ -17,10 +17,11 @@ use arrow_array::{RecordBatch, RecordBatchOptions}; use arrow_schema::SchemaRef; +use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::{ execution::TaskContext, physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, PlanProperties, + DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, RecordBatchStream, SendableRecordBatchStream, }, }; @@ -54,7 +55,8 @@ impl ExpandExec { let cache = PlanProperties::new( EquivalenceProperties::new(Arc::clone(&schema)), Partitioning::UnknownPartitioning(1), - ExecutionMode::Bounded, + EmissionType::Final, + Boundedness::Bounded, ); Self { diff --git a/native/core/src/execution/operators/filter.rs b/native/core/src/execution/operators/filter.rs index d9a54712dc..eab30a3560 100644 --- a/native/core/src/execution/operators/filter.rs +++ b/native/core/src/execution/operators/filter.rs @@ -210,7 +210,8 @@ impl FilterExec { Ok(PlanProperties::new( eq_properties, input.output_partitioning().clone(), // Output Partitioning - input.execution_mode(), // Execution Mode + input.pipeline_behavior(), + input.boundedness(), )) } } diff --git a/native/core/src/execution/operators/scan.rs b/native/core/src/execution/operators/scan.rs index a297f87c1f..888cd2fdb5 100644 --- a/native/core/src/execution/operators/scan.rs +++ b/native/core/src/execution/operators/scan.rs @@ -28,6 +28,7 @@ use arrow_data::ffi::FFI_ArrowArray; use arrow_data::ArrayData; use arrow_schema::ffi::FFI_ArrowSchema; use arrow_schema::{DataType, Field, Schema, SchemaRef}; +use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::physical_plan::metrics::{ BaselineMetrics, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, Time, }; @@ -122,7 +123,8 @@ impl ScanExec { // The partitioning is not important because we are not using DataFusion's // query planner or optimizer Partitioning::UnknownPartitioning(1), - ExecutionMode::Bounded, + EmissionType::Final, + Boundedness::Bounded, ); Ok(Self { diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 0a74933547..5a35c62e33 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -94,7 +94,6 @@ use datafusion_common::{ tree_node::{Transformed, TransformedResult, TreeNode, TreeNodeRecursion, TreeNodeRewriter}, JoinType as DFJoinType, ScalarValue, }; -use datafusion_expr::expr::find_df_window_func; use datafusion_expr::{ AggregateUDF, ScalarUDF, WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, @@ -1515,10 +1514,7 @@ impl PhysicalPlanner { let builder = match datatype { DataType::Decimal128(_, _) => { - let func = AggregateUDF::new_from_impl(SumDecimal::try_new( - Arc::clone(&child), - datatype, - )?); + let func = AggregateUDF::new_from_impl(SumDecimal::try_new(datatype)?); AggregateExprBuilder::new(Arc::new(func), vec![child]) } _ => { @@ -1543,11 +1539,8 @@ impl PhysicalPlanner { let input_datatype = to_arrow_datatype(expr.sum_datatype.as_ref().unwrap()); let builder = match datatype { DataType::Decimal128(_, _) => { - let func = AggregateUDF::new_from_impl(AvgDecimal::new( - Arc::clone(&child), - datatype, - input_datatype, - )); + let func = + AggregateUDF::new_from_impl(AvgDecimal::new(datatype, input_datatype)); AggregateExprBuilder::new(Arc::new(func), vec![child]) } _ => { @@ -1556,11 +1549,7 @@ impl PhysicalPlanner { // failure since it should have already been checked at Spark side. let child: Arc = Arc::new(CastExpr::new(Arc::clone(&child), datatype.clone(), None)); - let func = AggregateUDF::new_from_impl(Avg::new( - Arc::clone(&child), - "avg", - datatype, - )); + let func = AggregateUDF::new_from_impl(Avg::new("avg", datatype)); AggregateExprBuilder::new(Arc::new(func), vec![child]) } }; @@ -1638,8 +1627,6 @@ impl PhysicalPlanner { match expr.stats_type { 0 => { let func = AggregateUDF::new_from_impl(Covariance::new( - Arc::clone(&child1), - Arc::clone(&child2), "covariance", datatype, StatsType::Sample, @@ -1655,8 +1642,6 @@ impl PhysicalPlanner { } 1 => { let func = AggregateUDF::new_from_impl(Covariance::new( - Arc::clone(&child1), - Arc::clone(&child2), "covariance_pop", datatype, StatsType::Population, @@ -1682,7 +1667,6 @@ impl PhysicalPlanner { match expr.stats_type { 0 => { let func = AggregateUDF::new_from_impl(Variance::new( - Arc::clone(&child), "variance", datatype, StatsType::Sample, @@ -1693,7 +1677,6 @@ impl PhysicalPlanner { } 1 => { let func = AggregateUDF::new_from_impl(Variance::new( - Arc::clone(&child), "variance_pop", datatype, StatsType::Population, @@ -1714,7 +1697,6 @@ impl PhysicalPlanner { match expr.stats_type { 0 => { let func = AggregateUDF::new_from_impl(Stddev::new( - Arc::clone(&child), "stddev", datatype, StatsType::Sample, @@ -1725,7 +1707,6 @@ impl PhysicalPlanner { } 1 => { let func = AggregateUDF::new_from_impl(Stddev::new( - Arc::clone(&child), "stddev_pop", datatype, StatsType::Population, @@ -1747,8 +1728,6 @@ impl PhysicalPlanner { self.create_expr(expr.child2.as_ref().unwrap(), Arc::clone(&schema))?; let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap()); let func = AggregateUDF::new_from_impl(Correlation::new( - Arc::clone(&child1), - Arc::clone(&child2), "correlation", datatype, expr.null_on_divide_by_zero, @@ -1935,7 +1914,7 @@ impl PhysicalPlanner { window_func_name, &window_args, partition_by, - sort_exprs, + &LexOrdering::new(sort_exprs.to_vec()), window_frame.into(), input_schema.as_ref(), false, // TODO: Ignore nulls @@ -1985,15 +1964,11 @@ impl PhysicalPlanner { /// Find DataFusion's built-in window function by name. fn find_df_window_function(&self, name: &str) -> Option { - if let Some(f) = find_df_window_func(name) { - Some(f) - } else { - let registry = &self.session_ctx.state(); - registry - .udaf(name) - .map(WindowFunctionDefinition::AggregateUDF) - .ok() - } + let registry = &self.session_ctx.state(); + registry + .udaf(name) + .map(WindowFunctionDefinition::AggregateUDF) + .ok() } /// Create a DataFusion physical partitioning from Spark physical partitioning @@ -2049,7 +2024,15 @@ impl PhysicalPlanner { .coerce_types(&input_expr_types) .unwrap_or_else(|_| input_expr_types.clone()); - let data_type = func.inner().return_type(&coerced_types)?; + let data_type = match fun_name { + // workaround for https://github.com/apache/datafusion/issues/13716 + "datepart" => DataType::Int32, + _ => { + // TODO need to call `return_type_from_exprs` instead + #[allow(deprecated)] + func.inner().return_type(&coerced_types)? + } + }; (data_type, coerced_types) } diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 01117199eb..f3fa685b88 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -25,6 +25,7 @@ use arrow::{datatypes::*, ipc::writer::StreamWriter}; use async_trait::async_trait; use bytes::Buf; use crc32fast::Hasher; +use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::{ arrow::{ array::*, @@ -44,7 +45,7 @@ use datafusion::{ BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, Time, }, stream::RecordBatchStreamAdapter, - DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, PlanProperties, + DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }, }; @@ -191,7 +192,8 @@ impl ShuffleWriterExec { let cache = PlanProperties::new( EquivalenceProperties::new(Arc::clone(&input.schema())), partitioning.clone(), - ExecutionMode::Bounded, + EmissionType::Final, + Boundedness::Bounded, ); Ok(ShuffleWriterExec { diff --git a/native/core/src/execution/util/spark_bit_array.rs b/native/core/src/execution/util/spark_bit_array.rs index 6cfecc1bfb..3ac8b199bf 100644 --- a/native/core/src/execution/util/spark_bit_array.rs +++ b/native/core/src/execution/util/spark_bit_array.rs @@ -22,7 +22,7 @@ use std::iter::zip; /// A simple bit array implementation that simulates the behavior of Spark's BitArray which is /// used in the BloomFilter implementation. Some methods are not implemented as they are not /// required for the current use case. -#[derive(Debug, Hash)] +#[derive(Debug, Hash, PartialEq, Eq)] pub struct SparkBitArray { data: Vec, bit_count: usize, diff --git a/native/core/src/execution/util/spark_bloom_filter.rs b/native/core/src/execution/util/spark_bloom_filter.rs index 2c3af16916..61245757cf 100644 --- a/native/core/src/execution/util/spark_bloom_filter.rs +++ b/native/core/src/execution/util/spark_bloom_filter.rs @@ -27,7 +27,7 @@ const SPARK_BLOOM_FILTER_VERSION_1: i32 = 1; /// A Bloom filter implementation that simulates the behavior of Spark's BloomFilter. /// It's not a complete implementation of Spark's BloomFilter, but just add the minimum /// methods to support mightContainsLong in the native side. -#[derive(Debug, Hash)] +#[derive(Debug, Hash, PartialEq, Eq)] pub struct SparkBloomFilter { bits: SparkBitArray, num_hash_functions: u32, diff --git a/native/spark-expr/Cargo.toml b/native/spark-expr/Cargo.toml index 27367d83e1..fc348f81bf 100644 --- a/native/spark-expr/Cargo.toml +++ b/native/spark-expr/Cargo.toml @@ -36,6 +36,7 @@ chrono = { workspace = true } datafusion = { workspace = true, features = ["parquet"] } datafusion-common = { workspace = true } datafusion-expr = { workspace = true } +datafusion-expr-common = { workspace = true } datafusion-physical-expr = { workspace = true } chrono-tz = { workspace = true } num = { workspace = true } diff --git a/native/spark-expr/benches/aggregate.rs b/native/spark-expr/benches/aggregate.rs index 43194fdda2..051ac5eb62 100644 --- a/native/spark-expr/benches/aggregate.rs +++ b/native/spark-expr/benches/aggregate.rs @@ -66,7 +66,6 @@ fn criterion_benchmark(c: &mut Criterion) { group.bench_function("avg_decimal_comet", |b| { let comet_avg_decimal = Arc::new(AggregateUDF::new_from_impl(AvgDecimal::new( - Arc::clone(&c1), DataType::Decimal128(38, 10), DataType::Decimal128(38, 10), ))); @@ -96,7 +95,7 @@ fn criterion_benchmark(c: &mut Criterion) { group.bench_function("sum_decimal_comet", |b| { let comet_sum_decimal = Arc::new(AggregateUDF::new_from_impl( - SumDecimal::try_new(Arc::clone(&c1), DataType::Decimal128(38, 10)).unwrap(), + SumDecimal::try_new(DataType::Decimal128(38, 10)).unwrap(), )); b.to_async(&rt).iter(|| { black_box(agg_test( diff --git a/native/spark-expr/src/avg.rs b/native/spark-expr/src/avg.rs index 7820497d46..816440ac9a 100644 --- a/native/spark-expr/src/avg.rs +++ b/native/spark-expr/src/avg.rs @@ -27,11 +27,10 @@ use datafusion::logical_expr::{ type_coercion::aggregates::avg_return_type, Accumulator, EmitTo, GroupsAccumulator, Signature, }; use datafusion_common::{not_impl_err, Result, ScalarValue}; -use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; +use datafusion_physical_expr::expressions::format_state_name; use std::{any::Any, sync::Arc}; use arrow_array::ArrowNativeTypeOp; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::Volatility::Immutable; use datafusion_expr::{AggregateUDFImpl, ReversedUDAF}; @@ -42,20 +41,19 @@ use DataType::*; pub struct Avg { name: String, signature: Signature, - expr: Arc, + // expr: Arc, input_data_type: DataType, result_data_type: DataType, } impl Avg { /// Create a new AVG aggregate function - pub fn new(expr: Arc, name: impl Into, data_type: DataType) -> Self { + pub fn new(name: impl Into, data_type: DataType) -> Self { let result_data_type = avg_return_type("avg", &data_type).unwrap(); Self { name: name.into(), signature: Signature::user_defined(Immutable), - expr, input_data_type: data_type, result_data_type, } @@ -139,20 +137,6 @@ impl AggregateUDFImpl for Avg { } } -impl PartialEq for Avg { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name - && self.input_data_type == x.input_data_type - && self.result_data_type == x.result_data_type - && self.expr.eq(&x.expr) - }) - .unwrap_or(false) - } -} - /// An accumulator to compute the average #[derive(Debug, Default)] pub struct AvgAccumulator { diff --git a/native/spark-expr/src/avg_decimal.rs b/native/spark-expr/src/avg_decimal.rs index 163e1560b6..05fc28e583 100644 --- a/native/spark-expr/src/avg_decimal.rs +++ b/native/spark-expr/src/avg_decimal.rs @@ -25,14 +25,13 @@ use arrow_array::{ use arrow_schema::{DataType, Field}; use datafusion::logical_expr::{Accumulator, EmitTo, GroupsAccumulator, Signature}; use datafusion_common::{not_impl_err, Result, ScalarValue}; -use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; +use datafusion_physical_expr::expressions::format_state_name; use std::{any::Any, sync::Arc}; use crate::utils::is_valid_decimal_precision; use arrow_array::ArrowNativeTypeOp; use arrow_data::decimal::{MAX_DECIMAL_FOR_EACH_PRECISION, MIN_DECIMAL_FOR_EACH_PRECISION}; use datafusion::logical_expr::Volatility::Immutable; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::type_coercion::aggregates::avg_return_type; use datafusion_expr::{AggregateUDFImpl, ReversedUDAF}; @@ -43,17 +42,15 @@ use DataType::*; #[derive(Debug, Clone)] pub struct AvgDecimal { signature: Signature, - expr: Arc, sum_data_type: DataType, result_data_type: DataType, } impl AvgDecimal { /// Create a new AVG aggregate function - pub fn new(expr: Arc, result_type: DataType, sum_type: DataType) -> Self { + pub fn new(result_type: DataType, sum_type: DataType) -> Self { Self { signature: Signature::user_defined(Immutable), - expr, result_data_type: result_type, sum_data_type: sum_type, } @@ -156,19 +153,6 @@ impl AggregateUDFImpl for AvgDecimal { } } -impl PartialEq for AvgDecimal { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.sum_data_type == x.sum_data_type - && self.result_data_type == x.result_data_type - && self.expr.eq(&x.expr) - }) - .unwrap_or(false) - } -} - /// An accumulator to compute the average for decimals #[derive(Debug)] struct AvgDecimalAccumulator { diff --git a/native/spark-expr/src/bitwise_not.rs b/native/spark-expr/src/bitwise_not.rs index 36234935e1..d7c31836ff 100644 --- a/native/spark-expr/src/bitwise_not.rs +++ b/native/spark-expr/src/bitwise_not.rs @@ -15,21 +15,16 @@ // specific language governing permissions and limitations // under the License. -use std::{ - any::Any, - hash::{Hash, Hasher}, - sync::Arc, -}; - use arrow::{ array::*, datatypes::{DataType, Schema}, record_batch::RecordBatch, }; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion::{error::DataFusionError, logical_expr::ColumnarValue}; use datafusion_common::Result; use datafusion_physical_expr::PhysicalExpr; +use std::hash::Hash; +use std::{any::Any, sync::Arc}; macro_rules! compute_op { ($OPERAND:expr, $DT:ident) => {{ @@ -43,12 +38,24 @@ macro_rules! compute_op { } /// BitwiseNot expression -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct BitwiseNotExpr { /// Input expression arg: Arc, } +impl Hash for BitwiseNotExpr { + fn hash(&self, state: &mut H) { + self.arg.hash(state); + } +} + +impl PartialEq for BitwiseNotExpr { + fn eq(&self, other: &Self) -> bool { + self.arg.eq(&other.arg) + } +} + impl BitwiseNotExpr { /// Create new bitwise not expression pub fn new(arg: Arc) -> Self { @@ -114,21 +121,6 @@ impl PhysicalExpr for BitwiseNotExpr { ) -> Result> { Ok(Arc::new(BitwiseNotExpr::new(Arc::clone(&children[0])))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.arg.hash(&mut s); - self.hash(&mut s); - } -} - -impl PartialEq for BitwiseNotExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.arg.eq(&x.arg)) - .unwrap_or(false) - } } pub fn bitwise_not(arg: Arc) -> Result> { diff --git a/native/spark-expr/src/cast.rs b/native/spark-expr/src/cast.rs index d96bcbbdb6..6e0e0915cd 100644 --- a/native/spark-expr/src/cast.rs +++ b/native/spark-expr/src/cast.rs @@ -39,7 +39,6 @@ use arrow_array::builder::StringBuilder; use arrow_array::{DictionaryArray, StringArray, StructArray}; use arrow_schema::{DataType, Field, Schema}; use chrono::{NaiveDate, NaiveDateTime, TimeZone, Timelike}; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{ cast::as_generic_string_array, internal_err, Result as DataFusionResult, ScalarValue, }; @@ -54,7 +53,7 @@ use std::str::FromStr; use std::{ any::Any, fmt::{Debug, Display, Formatter}, - hash::{Hash, Hasher}, + hash::Hash, num::Wrapping, sync::Arc, }; @@ -131,13 +130,29 @@ impl TimeStampInfo { } } -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct Cast { pub child: Arc, pub data_type: DataType, pub cast_options: SparkCastOptions, } +impl PartialEq for Cast { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) + && self.data_type.eq(&other.data_type) + && self.cast_options.eq(&other.cast_options) + } +} + +impl Hash for Cast { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.data_type.hash(state); + self.cast_options.hash(state); + } +} + /// Determine if Comet supports a cast, taking options such as EvalMode and Timezone into account. pub fn cast_supported( from_type: &DataType, @@ -1681,19 +1696,6 @@ impl Display for Cast { } } -impl PartialEq for Cast { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.child.eq(&x.child) - && self.cast_options.eq(&x.cast_options) - && self.data_type.eq(&x.data_type) - }) - .unwrap_or(false) - } -} - impl PhysicalExpr for Cast { fn as_any(&self) -> &dyn Any { self @@ -1729,14 +1731,6 @@ impl PhysicalExpr for Cast { _ => internal_err!("Cast should have exactly one child"), } } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.data_type.hash(&mut s); - self.cast_options.hash(&mut s); - self.hash(&mut s); - } } fn timestamp_parser( diff --git a/native/spark-expr/src/checkoverflow.rs b/native/spark-expr/src/checkoverflow.rs index e922171bd2..528bbd5d96 100644 --- a/native/spark-expr/src/checkoverflow.rs +++ b/native/spark-expr/src/checkoverflow.rs @@ -15,13 +15,6 @@ // specific language governing permissions and limitations // under the License. -use std::{ - any::Any, - fmt::{Display, Formatter}, - hash::{Hash, Hasher}, - sync::Arc, -}; - use arrow::{ array::{as_primitive_array, Array, ArrayRef, Decimal128Array}, datatypes::{Decimal128Type, DecimalType}, @@ -29,21 +22,42 @@ use arrow::{ }; use arrow_schema::{DataType, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{DataFusionError, ScalarValue}; use datafusion_physical_expr::PhysicalExpr; +use std::hash::Hash; +use std::{ + any::Any, + fmt::{Display, Formatter}, + sync::Arc, +}; /// This is from Spark `CheckOverflow` expression. Spark `CheckOverflow` expression rounds decimals /// to given scale and check if the decimals can fit in given precision. As `cast` kernel rounds /// decimals already, Comet `CheckOverflow` expression only checks if the decimals can fit in the /// precision. -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct CheckOverflow { pub child: Arc, pub data_type: DataType, pub fail_on_error: bool, } +impl Hash for CheckOverflow { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.data_type.hash(state); + self.fail_on_error.hash(state); + } +} + +impl PartialEq for CheckOverflow { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) + && self.data_type.eq(&other.data_type) + && self.fail_on_error.eq(&other.fail_on_error) + } +} + impl CheckOverflow { pub fn new(child: Arc, data_type: DataType, fail_on_error: bool) -> Self { Self { @@ -64,19 +78,6 @@ impl Display for CheckOverflow { } } -impl PartialEq for CheckOverflow { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.child.eq(&x.child) - && self.data_type.eq(&x.data_type) - && self.fail_on_error.eq(&x.fail_on_error) - }) - .unwrap_or(false) - } -} - impl PhysicalExpr for CheckOverflow { fn as_any(&self) -> &dyn Any { self @@ -162,12 +163,4 @@ impl PhysicalExpr for CheckOverflow { self.fail_on_error, ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.data_type.hash(&mut s); - self.fail_on_error.hash(&mut s); - self.hash(&mut s); - } } diff --git a/native/spark-expr/src/correlation.rs b/native/spark-expr/src/correlation.rs index e5f36c6f95..e4ddab95de 100644 --- a/native/spark-expr/src/correlation.rs +++ b/native/spark-expr/src/correlation.rs @@ -26,13 +26,12 @@ use arrow::{ datatypes::{DataType, Field}, }; use datafusion::logical_expr::Accumulator; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::type_coercion::aggregates::NUMERICS; use datafusion_expr::{AggregateUDFImpl, Signature, Volatility}; +use datafusion_physical_expr::expressions::format_state_name; use datafusion_physical_expr::expressions::StatsType; -use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; /// CORR aggregate expression /// The implementation mostly is the same as the DataFusion's implementation. The reason @@ -43,26 +42,16 @@ use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; pub struct Correlation { name: String, signature: Signature, - expr1: Arc, - expr2: Arc, null_on_divide_by_zero: bool, } impl Correlation { - pub fn new( - expr1: Arc, - expr2: Arc, - name: impl Into, - data_type: DataType, - null_on_divide_by_zero: bool, - ) -> Self { + pub fn new(name: impl Into, data_type: DataType, null_on_divide_by_zero: bool) -> Self { // the result of correlation just support FLOAT64 data type. assert!(matches!(data_type, DataType::Float64)); Self { name: name.into(), signature: Signature::uniform(2, NUMERICS.to_vec(), Volatility::Immutable), - expr1, - expr2, null_on_divide_by_zero, } } @@ -131,20 +120,6 @@ impl AggregateUDFImpl for Correlation { } } -impl PartialEq for Correlation { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name - && self.expr1.eq(&x.expr1) - && self.expr2.eq(&x.expr2) - && self.null_on_divide_by_zero == x.null_on_divide_by_zero - }) - .unwrap_or(false) - } -} - /// An accumulator to compute correlation #[derive(Debug)] pub struct CorrelationAccumulator { diff --git a/native/spark-expr/src/covariance.rs b/native/spark-expr/src/covariance.rs index 9166e39766..fa3563cdea 100644 --- a/native/spark-expr/src/covariance.rs +++ b/native/spark-expr/src/covariance.rs @@ -17,7 +17,7 @@ * under the License. */ -use std::{any::Any, sync::Arc}; +use std::any::Any; use arrow::{ array::{ArrayRef, Float64Array}, @@ -25,15 +25,14 @@ use arrow::{ datatypes::{DataType, Field}, }; use datafusion::logical_expr::Accumulator; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{ downcast_value, unwrap_or_internal_err, DataFusionError, Result, ScalarValue, }; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::type_coercion::aggregates::NUMERICS; use datafusion_expr::{AggregateUDFImpl, Signature, Volatility}; +use datafusion_physical_expr::expressions::format_state_name; use datafusion_physical_expr::expressions::StatsType; -use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; /// COVAR_SAMP and COVAR_POP aggregate expression /// The implementation mostly is the same as the DataFusion's implementation. The reason @@ -43,8 +42,6 @@ use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; pub struct Covariance { name: String, signature: Signature, - expr1: Arc, - expr2: Arc, stats_type: StatsType, null_on_divide_by_zero: bool, } @@ -52,8 +49,6 @@ pub struct Covariance { impl Covariance { /// Create a new COVAR aggregate function pub fn new( - expr1: Arc, - expr2: Arc, name: impl Into, data_type: DataType, stats_type: StatsType, @@ -64,8 +59,6 @@ impl Covariance { Self { name: name.into(), signature: Signature::uniform(2, NUMERICS.to_vec(), Volatility::Immutable), - expr1, - expr2, stats_type, null_on_divide_by_zero, } @@ -126,21 +119,6 @@ impl AggregateUDFImpl for Covariance { } } -impl PartialEq for Covariance { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name - && self.expr1.eq(&x.expr1) - && self.expr2.eq(&x.expr2) - && self.stats_type == x.stats_type - && self.null_on_divide_by_zero == x.null_on_divide_by_zero - }) - .unwrap_or(false) - } -} - /// An accumulator to compute covariance #[derive(Debug)] pub struct CovarianceAccumulator { diff --git a/native/spark-expr/src/if_expr.rs b/native/spark-expr/src/if_expr.rs index 193a90fb55..01c754ad6d 100644 --- a/native/spark-expr/src/if_expr.rs +++ b/native/spark-expr/src/if_expr.rs @@ -15,24 +15,19 @@ // specific language governing permissions and limitations // under the License. -use std::{ - any::Any, - hash::{Hash, Hasher}, - sync::Arc, -}; - use arrow::{ datatypes::{DataType, Schema}, record_batch::RecordBatch, }; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::Result; use datafusion_physical_expr::{expressions::CaseExpr, PhysicalExpr}; +use std::hash::Hash; +use std::{any::Any, sync::Arc}; /// IfExpr is a wrapper around CaseExpr, because `IF(a, b, c)` is semantically equivalent to /// `CASE WHEN a THEN b ELSE c END`. -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct IfExpr { if_expr: Arc, true_expr: Arc, @@ -41,6 +36,23 @@ pub struct IfExpr { case_expr: Arc, } +impl Hash for IfExpr { + fn hash(&self, state: &mut H) { + self.if_expr.hash(state); + self.true_expr.hash(state); + self.false_expr.hash(state); + self.case_expr.hash(state); + } +} +impl PartialEq for IfExpr { + fn eq(&self, other: &Self) -> bool { + self.if_expr.eq(&other.if_expr) + && self.true_expr.eq(&other.true_expr) + && self.false_expr.eq(&other.false_expr) + && self.case_expr.eq(&other.case_expr) + } +} + impl std::fmt::Display for IfExpr { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { write!( @@ -106,27 +118,6 @@ impl PhysicalExpr for IfExpr { Arc::clone(&children[2]), ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.if_expr.hash(&mut s); - self.true_expr.hash(&mut s); - self.false_expr.hash(&mut s); - self.hash(&mut s); - } -} - -impl PartialEq for IfExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.if_expr.eq(&x.if_expr) - && self.true_expr.eq(&x.true_expr) - && self.false_expr.eq(&x.false_expr) - }) - .unwrap_or(false) - } } #[cfg(test)] diff --git a/native/spark-expr/src/list.rs b/native/spark-expr/src/list.rs index 7dc17b5688..fc31b11a0b 100644 --- a/native/spark-expr/src/list.rs +++ b/native/spark-expr/src/list.rs @@ -26,16 +26,15 @@ use arrow_array::{ }; use arrow_schema::{DataType, Field, FieldRef, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{ cast::{as_int32_array, as_large_list_array, as_list_array}, internal_err, DataFusionError, Result as DataFusionResult, ScalarValue, }; use datafusion_physical_expr::PhysicalExpr; +use std::hash::Hash; use std::{ any::Any, fmt::{Debug, Display, Formatter}, - hash::{Hash, Hasher}, sync::Arc, }; @@ -44,7 +43,7 @@ use std::{ // https://github.com/apache/spark/blob/master/common/utils/src/main/java/org/apache/spark/unsafe/array/ByteArrayUtils.java const MAX_ROUNDED_ARRAY_LENGTH: usize = 2147483632; -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct ListExtract { child: Arc, ordinal: Arc, @@ -53,6 +52,25 @@ pub struct ListExtract { fail_on_error: bool, } +impl Hash for ListExtract { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.ordinal.hash(state); + self.default_value.hash(state); + self.one_based.hash(state); + self.fail_on_error.hash(state); + } +} +impl PartialEq for ListExtract { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) + && self.ordinal.eq(&other.ordinal) + && self.default_value.eq(&other.default_value) + && self.one_based.eq(&other.one_based) + && self.fail_on_error.eq(&other.fail_on_error) + } +} + impl ListExtract { pub fn new( child: Arc, @@ -176,16 +194,6 @@ impl PhysicalExpr for ListExtract { _ => internal_err!("ListExtract should have exactly two children"), } } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.ordinal.hash(&mut s); - self.default_value.hash(&mut s); - self.one_based.hash(&mut s); - self.fail_on_error.hash(&mut s); - self.hash(&mut s); - } } fn one_based_index(index: i32, len: usize) -> DataFusionResult> { @@ -267,33 +275,24 @@ impl Display for ListExtract { } } -impl PartialEq for ListExtract { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.child.eq(&x.child) - && self.ordinal.eq(&x.ordinal) - && (self.default_value.is_none() == x.default_value.is_none()) - && self - .default_value - .as_ref() - .zip(x.default_value.as_ref()) - .map(|(s, x)| s.eq(x)) - .unwrap_or(true) - && self.one_based.eq(&x.one_based) - && self.fail_on_error.eq(&x.fail_on_error) - }) - .unwrap_or(false) - } -} - -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct GetArrayStructFields { child: Arc, ordinal: usize, } +impl Hash for GetArrayStructFields { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.ordinal.hash(state); + } +} +impl PartialEq for GetArrayStructFields { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.ordinal.eq(&other.ordinal) + } +} + impl GetArrayStructFields { pub fn new(child: Arc, ordinal: usize) -> Self { Self { child, ordinal } @@ -379,13 +378,6 @@ impl PhysicalExpr for GetArrayStructFields { _ => internal_err!("GetArrayStructFields should have exactly one child"), } } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.ordinal.hash(&mut s); - self.hash(&mut s); - } } fn get_array_struct_fields( @@ -417,16 +409,7 @@ impl Display for GetArrayStructFields { } } -impl PartialEq for GetArrayStructFields { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.ordinal.eq(&x.ordinal)) - .unwrap_or(false) - } -} - -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct ArrayInsert { src_array_expr: Arc, pos_expr: Arc, @@ -434,6 +417,23 @@ pub struct ArrayInsert { legacy_negative_index: bool, } +impl Hash for ArrayInsert { + fn hash(&self, state: &mut H) { + self.src_array_expr.hash(state); + self.pos_expr.hash(state); + self.item_expr.hash(state); + self.legacy_negative_index.hash(state); + } +} +impl PartialEq for ArrayInsert { + fn eq(&self, other: &Self) -> bool { + self.src_array_expr.eq(&other.src_array_expr) + && self.pos_expr.eq(&other.pos_expr) + && self.item_expr.eq(&other.item_expr) + && self.legacy_negative_index.eq(&other.legacy_negative_index) + } +} + impl ArrayInsert { pub fn new( src_array_expr: Arc, @@ -555,15 +555,6 @@ impl PhysicalExpr for ArrayInsert { _ => internal_err!("ArrayInsert should have exactly three childrens"), } } - - fn dyn_hash(&self, _state: &mut dyn Hasher) { - let mut s = _state; - self.src_array_expr.hash(&mut s); - self.pos_expr.hash(&mut s); - self.item_expr.hash(&mut s); - self.legacy_negative_index.hash(&mut s); - self.hash(&mut s); - } } fn array_insert( @@ -694,20 +685,6 @@ impl Display for ArrayInsert { } } -impl PartialEq for ArrayInsert { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.src_array_expr.eq(&x.src_array_expr) - && self.pos_expr.eq(&x.pos_expr) - && self.item_expr.eq(&x.item_expr) - && self.legacy_negative_index.eq(&x.legacy_negative_index) - }) - .unwrap_or(false) - } -} - #[cfg(test)] mod test { use crate::list::{array_insert, list_extract, zero_based_index}; diff --git a/native/spark-expr/src/negative.rs b/native/spark-expr/src/negative.rs index 3d9063e783..7fb5089179 100644 --- a/native/spark-expr/src/negative.rs +++ b/native/spark-expr/src/negative.rs @@ -21,18 +21,14 @@ use arrow::{compute::kernels::numeric::neg_wrapping, datatypes::IntervalDayTimeT use arrow_array::RecordBatch; use arrow_buffer::IntervalDayTime; use arrow_schema::{DataType, Schema}; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion::{ logical_expr::{interval_arithmetic::Interval, ColumnarValue}, physical_expr::PhysicalExpr, }; use datafusion_common::{DataFusionError, Result, ScalarValue}; use datafusion_expr::sort_properties::ExprProperties; -use std::{ - any::Any, - hash::{Hash, Hasher}, - sync::Arc, -}; +use std::hash::Hash; +use std::{any::Any, sync::Arc}; pub fn create_negate_expr( expr: Arc, @@ -42,13 +38,26 @@ pub fn create_negate_expr( } /// Negative expression -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct NegativeExpr { /// Input expression arg: Arc, fail_on_error: bool, } +impl Hash for NegativeExpr { + fn hash(&self, state: &mut H) { + self.arg.hash(state); + self.fail_on_error.hash(state); + } +} + +impl PartialEq for NegativeExpr { + fn eq(&self, other: &Self) -> bool { + self.arg.eq(&other.arg) && self.fail_on_error.eq(&other.fail_on_error) + } +} + macro_rules! check_overflow { ($array:expr, $array_type:ty, $min_val:expr, $type_name:expr) => {{ let typed_array = $array @@ -204,11 +213,6 @@ impl PhysicalExpr for NegativeExpr { ))) } - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.hash(&mut s); - } - /// Given the child interval of a NegativeExpr, it calculates the NegativeExpr's interval. /// It replaces the upper and lower bounds after multiplying them with -1. /// Ex: `(a, b]` => `[-b, -a)` @@ -255,12 +259,3 @@ impl PhysicalExpr for NegativeExpr { Ok(properties) } } - -impl PartialEq for NegativeExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.arg.eq(&x.arg)) - .unwrap_or(false) - } -} diff --git a/native/spark-expr/src/normalize_nan.rs b/native/spark-expr/src/normalize_nan.rs index c5331ad7bd..078ce4b5a4 100644 --- a/native/spark-expr/src/normalize_nan.rs +++ b/native/spark-expr/src/normalize_nan.rs @@ -15,13 +15,6 @@ // specific language governing permissions and limitations // under the License. -use std::{ - any::Any, - fmt::{Display, Formatter}, - hash::{Hash, Hasher}, - sync::Arc, -}; - use arrow::{ array::{as_primitive_array, ArrayAccessor, ArrayIter, Float32Array, Float64Array}, datatypes::{ArrowNativeType, Float32Type, Float64Type}, @@ -29,15 +22,33 @@ use arrow::{ }; use arrow_schema::{DataType, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_physical_expr::PhysicalExpr; +use std::hash::Hash; +use std::{ + any::Any, + fmt::{Display, Formatter}, + sync::Arc, +}; -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct NormalizeNaNAndZero { pub data_type: DataType, pub child: Arc, } +impl PartialEq for NormalizeNaNAndZero { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.data_type.eq(&other.data_type) + } +} + +impl Hash for NormalizeNaNAndZero { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.data_type.hash(state); + } +} + impl NormalizeNaNAndZero { pub fn new(data_type: DataType, child: Arc) -> Self { Self { data_type, child } @@ -89,13 +100,6 @@ impl PhysicalExpr for NormalizeNaNAndZero { Arc::clone(&children[0]), ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.data_type.hash(&mut s); - self.hash(&mut s); - } } fn eval_typed>(input: T) -> Vec> { @@ -120,15 +124,6 @@ impl Display for NormalizeNaNAndZero { } } -impl PartialEq for NormalizeNaNAndZero { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.data_type.eq(&x.data_type)) - .unwrap_or(false) - } -} - trait FloatDouble: ArrowNativeType { fn is_nan(&self) -> bool; fn nan(&self) -> Self; diff --git a/native/spark-expr/src/regexp.rs b/native/spark-expr/src/regexp.rs index c7626285a2..7f367a8bb9 100644 --- a/native/spark-expr/src/regexp.rs +++ b/native/spark-expr/src/regexp.rs @@ -21,7 +21,7 @@ use arrow_array::builder::BooleanBuilder; use arrow_array::types::Int32Type; use arrow_array::{Array, BooleanArray, DictionaryArray, RecordBatch, StringArray}; use arrow_schema::{DataType, Schema}; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; +use datafusion::physical_expr_common::physical_expr::DynEq; use datafusion_common::{internal_err, Result}; use datafusion_expr::ColumnarValue; use datafusion_physical_expr::PhysicalExpr; @@ -53,6 +53,16 @@ impl Hash for RLike { } } +impl DynEq for RLike { + fn dyn_eq(&self, other: &dyn Any) -> bool { + if let Some(other) = other.downcast_ref::() { + self.pattern_str == other.pattern_str + } else { + false + } + } +} + impl RLike { pub fn try_new(child: Arc, pattern: &str) -> Result { Ok(Self { @@ -93,15 +103,6 @@ impl Display for RLike { } } -impl PartialEq for RLike { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.pattern_str.eq(&x.pattern_str)) - .unwrap_or(false) - } -} - impl PhysicalExpr for RLike { fn as_any(&self) -> &dyn Any { self @@ -161,10 +162,4 @@ impl PhysicalExpr for RLike { &self.pattern_str, )?)) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - use std::hash::Hash; - let mut s = state; - self.hash(&mut s); - } } diff --git a/native/spark-expr/src/stddev.rs b/native/spark-expr/src/stddev.rs index 3cf604da0b..1ec5ffb69a 100644 --- a/native/spark-expr/src/stddev.rs +++ b/native/spark-expr/src/stddev.rs @@ -23,12 +23,12 @@ use arrow::{ datatypes::{DataType, Field}, }; use datafusion::logical_expr::Accumulator; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; +use datafusion_common::types::NativeType; use datafusion_common::{internal_err, Result, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::{AggregateUDFImpl, Signature, Volatility}; +use datafusion_physical_expr::expressions::format_state_name; use datafusion_physical_expr::expressions::StatsType; -use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; /// STDDEV and STDDEV_SAMP (standard deviation) aggregate expression /// The implementation mostly is the same as the DataFusion's implementation. The reason @@ -39,7 +39,6 @@ use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; pub struct Stddev { name: String, signature: Signature, - expr: Arc, stats_type: StatsType, null_on_divide_by_zero: bool, } @@ -47,7 +46,6 @@ pub struct Stddev { impl Stddev { /// Create a new STDDEV aggregate function pub fn new( - expr: Arc, name: impl Into, data_type: DataType, stats_type: StatsType, @@ -57,8 +55,14 @@ impl Stddev { assert!(matches!(data_type, DataType::Float64)); Self { name: name.into(), - signature: Signature::coercible(vec![DataType::Float64], Volatility::Immutable), - expr, + signature: Signature::coercible( + vec![ + datafusion_expr_common::signature::TypeSignatureClass::Native(Arc::new( + NativeType::Float64, + )), + ], + Volatility::Immutable, + ), stats_type, null_on_divide_by_zero, } @@ -121,20 +125,6 @@ impl AggregateUDFImpl for Stddev { } } -impl PartialEq for Stddev { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name - && self.expr.eq(&x.expr) - && self.null_on_divide_by_zero == x.null_on_divide_by_zero - && self.stats_type == x.stats_type - }) - .unwrap_or(false) - } -} - /// An accumulator to compute the standard deviation #[derive(Debug)] pub struct StddevAccumulator { diff --git a/native/spark-expr/src/strings.rs b/native/spark-expr/src/strings.rs index a8aab6aee9..c2706b5896 100644 --- a/native/spark-expr/src/strings.rs +++ b/native/spark-expr/src/strings.rs @@ -27,19 +27,18 @@ use arrow::{ }; use arrow_schema::{DataType, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{DataFusionError, ScalarValue::Utf8}; use datafusion_physical_expr::PhysicalExpr; use std::{ any::Any, fmt::{Display, Formatter}, - hash::{Hash, Hasher}, + hash::Hash, sync::Arc, }; macro_rules! make_predicate_function { ($name: ident, $kernel: ident, $str_scalar_kernel: ident) => { - #[derive(Debug, Hash)] + #[derive(Debug, Eq)] pub struct $name { left: Arc, right: Arc, @@ -57,12 +56,16 @@ macro_rules! make_predicate_function { } } - impl PartialEq for $name { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.left.eq(&x.left) && self.right.eq(&x.right)) - .unwrap_or(false) + impl Hash for $name { + fn hash(&self, state: &mut H) { + self.left.hash(state); + self.right.hash(state); + } + } + + impl PartialEq for $name { + fn eq(&self, other: &Self) -> bool { + self.left.eq(&other.left) && self.right.eq(&other.right) } } @@ -122,13 +125,6 @@ macro_rules! make_predicate_function { children[1].clone(), ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.left.hash(&mut s); - self.right.hash(&mut s); - self.hash(&mut s); - } } }; } @@ -141,18 +137,43 @@ make_predicate_function!(EndsWith, ends_with_dyn, ends_with_utf8_scalar_dyn); make_predicate_function!(Contains, contains_dyn, contains_utf8_scalar_dyn); -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct SubstringExpr { pub child: Arc, pub start: i64, pub len: u64, } -#[derive(Debug, Hash)] +impl Hash for SubstringExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.start.hash(state); + self.len.hash(state); + } +} + +impl PartialEq for SubstringExpr { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.start.eq(&other.start) && self.len.eq(&other.len) + } +} +#[derive(Debug, Eq)] pub struct StringSpaceExpr { pub child: Arc, } +impl Hash for StringSpaceExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + } +} + +impl PartialEq for StringSpaceExpr { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) + } +} + impl SubstringExpr { pub fn new(child: Arc, start: i64, len: u64) -> Self { Self { child, start, len } @@ -181,15 +202,6 @@ impl Display for StringSpaceExpr { } } -impl PartialEq for SubstringExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.start.eq(&x.start) && self.len.eq(&x.len)) - .unwrap_or(false) - } -} - impl PhysicalExpr for SubstringExpr { fn as_any(&self) -> &dyn Any { self @@ -231,23 +243,6 @@ impl PhysicalExpr for SubstringExpr { self.len, ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.start.hash(&mut s); - self.len.hash(&mut s); - self.hash(&mut s); - } -} - -impl PartialEq for StringSpaceExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child)) - .unwrap_or(false) - } } impl PhysicalExpr for StringSpaceExpr { @@ -292,10 +287,4 @@ impl PhysicalExpr for StringSpaceExpr { ) -> datafusion_common::Result> { Ok(Arc::new(StringSpaceExpr::new(Arc::clone(&children[0])))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.hash(&mut s); - } } diff --git a/native/spark-expr/src/structs.rs b/native/spark-expr/src/structs.rs index cda8246d90..7cc49e4281 100644 --- a/native/spark-expr/src/structs.rs +++ b/native/spark-expr/src/structs.rs @@ -19,17 +19,16 @@ use arrow::record_batch::RecordBatch; use arrow_array::{Array, StructArray}; use arrow_schema::{DataType, Field, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{DataFusionError, Result as DataFusionResult, ScalarValue}; use datafusion_physical_expr::PhysicalExpr; use std::{ any::Any, fmt::{Display, Formatter}, - hash::{Hash, Hasher}, + hash::Hash, sync::Arc, }; -#[derive(Debug, Hash)] +#[derive(Debug, Hash, PartialEq, Eq)] pub struct CreateNamedStruct { values: Vec>, names: Vec, @@ -95,13 +94,6 @@ impl PhysicalExpr for CreateNamedStruct { self.names.clone(), ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.values.hash(&mut s); - self.names.hash(&mut s); - self.hash(&mut s); - } } impl Display for CreateNamedStruct { @@ -114,29 +106,24 @@ impl Display for CreateNamedStruct { } } -impl PartialEq for CreateNamedStruct { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.values - .iter() - .zip(x.values.iter()) - .all(|(a, b)| a.eq(b)) - && self.values.len() == x.values.len() - && self.names.iter().zip(x.names.iter()).all(|(a, b)| a.eq(b)) - && self.names.len() == x.names.len() - }) - .unwrap_or(false) - } -} - -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct GetStructField { child: Arc, ordinal: usize, } +impl Hash for GetStructField { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.ordinal.hash(state); + } +} +impl PartialEq for GetStructField { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.ordinal.eq(&other.ordinal) + } +} + impl GetStructField { pub fn new(child: Arc, ordinal: usize) -> Self { Self { child, ordinal } @@ -203,13 +190,6 @@ impl PhysicalExpr for GetStructField { self.ordinal, ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.ordinal.hash(&mut s); - self.hash(&mut s); - } } impl Display for GetStructField { @@ -222,15 +202,6 @@ impl Display for GetStructField { } } -impl PartialEq for GetStructField { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.ordinal.eq(&x.ordinal)) - .unwrap_or(false) - } -} - #[cfg(test)] mod test { use super::CreateNamedStruct; diff --git a/native/spark-expr/src/sum_decimal.rs b/native/spark-expr/src/sum_decimal.rs index ab142aee69..f3f34d9bfa 100644 --- a/native/spark-expr/src/sum_decimal.rs +++ b/native/spark-expr/src/sum_decimal.rs @@ -25,20 +25,16 @@ use arrow_array::{ }; use arrow_schema::{DataType, Field}; use datafusion::logical_expr::{Accumulator, EmitTo, GroupsAccumulator}; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{DataFusionError, Result as DFResult, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::Volatility::Immutable; use datafusion_expr::{AggregateUDFImpl, ReversedUDAF, Signature}; -use datafusion_physical_expr::PhysicalExpr; use std::{any::Any, ops::BitAnd, sync::Arc}; #[derive(Debug)] pub struct SumDecimal { /// Aggregate function signature signature: Signature, - /// The expression that provides the input decimal values to be summed - expr: Arc, /// The data type of the SUM result. This will always be a decimal type /// with the same precision and scale as specified in this struct result_type: DataType, @@ -49,7 +45,7 @@ pub struct SumDecimal { } impl SumDecimal { - pub fn try_new(expr: Arc, data_type: DataType) -> DFResult { + pub fn try_new(data_type: DataType) -> DFResult { // The `data_type` is the SUM result type passed from Spark side let (precision, scale) = match data_type { DataType::Decimal128(p, s) => (p, s), @@ -61,7 +57,6 @@ impl SumDecimal { }; Ok(Self { signature: Signature::user_defined(Immutable), - expr, result_type: data_type, precision, scale, @@ -132,20 +127,6 @@ impl AggregateUDFImpl for SumDecimal { } } -impl PartialEq for SumDecimal { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - // note that we do not compare result_type because this - // is guaranteed to match if the precision and scale - // match - self.precision == x.precision && self.scale == x.scale && self.expr.eq(&x.expr) - }) - .unwrap_or(false) - } -} - #[derive(Debug)] struct SumDecimalAccumulator { sum: i128, @@ -491,13 +472,13 @@ mod tests { use datafusion_common::Result; use datafusion_expr::AggregateUDF; use datafusion_physical_expr::aggregate::AggregateExprBuilder; - use datafusion_physical_expr::expressions::{Column, Literal}; + use datafusion_physical_expr::expressions::Column; + use datafusion_physical_expr::PhysicalExpr; use futures::StreamExt; #[test] fn invalid_data_type() { - let expr = Arc::new(Literal::new(ScalarValue::Int32(Some(1)))); - assert!(SumDecimal::try_new(expr, DataType::Int32).is_err()); + assert!(SumDecimal::try_new(DataType::Int32).is_err()); } #[tokio::test] @@ -518,7 +499,6 @@ mod tests { Arc::new(MemoryExec::try_new(partitions, Arc::clone(&schema), None).unwrap()); let aggregate_udf = Arc::new(AggregateUDF::new_from_impl(SumDecimal::try_new( - Arc::clone(&c1), data_type.clone(), )?)); diff --git a/native/spark-expr/src/temporal.rs b/native/spark-expr/src/temporal.rs index 91953dd600..fb549f9ce8 100644 --- a/native/spark-expr/src/temporal.rs +++ b/native/spark-expr/src/temporal.rs @@ -15,36 +15,45 @@ // specific language governing permissions and limitations // under the License. -use std::{ - any::Any, - fmt::{Debug, Display, Formatter}, - hash::{Hash, Hasher}, - sync::Arc, -}; - +use crate::utils::array_with_timezone; use arrow::{ compute::{date_part, DatePart}, record_batch::RecordBatch, }; use arrow_schema::{DataType, Schema, TimeUnit::Microsecond}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{DataFusionError, ScalarValue::Utf8}; use datafusion_physical_expr::PhysicalExpr; - -use crate::utils::array_with_timezone; +use std::hash::Hash; +use std::{ + any::Any, + fmt::{Debug, Display, Formatter}, + sync::Arc, +}; use crate::kernels::temporal::{ date_trunc_array_fmt_dyn, date_trunc_dyn, timestamp_trunc_array_fmt_dyn, timestamp_trunc_dyn, }; -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct HourExpr { /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) child: Arc, timezone: String, } +impl Hash for HourExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.timezone.hash(state); + } +} +impl PartialEq for HourExpr { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.timezone.eq(&other.timezone) + } +} + impl HourExpr { pub fn new(child: Arc, timezone: String) -> Self { HourExpr { child, timezone } @@ -61,15 +70,6 @@ impl Display for HourExpr { } } -impl PartialEq for HourExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.timezone.eq(&x.timezone)) - .unwrap_or(false) - } -} - impl PhysicalExpr for HourExpr { fn as_any(&self) -> &dyn Any { self @@ -123,22 +123,27 @@ impl PhysicalExpr for HourExpr { self.timezone.clone(), ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.timezone.hash(&mut s); - self.hash(&mut s); - } } -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct MinuteExpr { /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) child: Arc, timezone: String, } +impl Hash for MinuteExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.timezone.hash(state); + } +} +impl PartialEq for MinuteExpr { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.timezone.eq(&other.timezone) + } +} + impl MinuteExpr { pub fn new(child: Arc, timezone: String) -> Self { MinuteExpr { child, timezone } @@ -155,15 +160,6 @@ impl Display for MinuteExpr { } } -impl PartialEq for MinuteExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.timezone.eq(&x.timezone)) - .unwrap_or(false) - } -} - impl PhysicalExpr for MinuteExpr { fn as_any(&self) -> &dyn Any { self @@ -217,22 +213,27 @@ impl PhysicalExpr for MinuteExpr { self.timezone.clone(), ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.timezone.hash(&mut s); - self.hash(&mut s); - } } -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct SecondExpr { /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) child: Arc, timezone: String, } +impl Hash for SecondExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.timezone.hash(state); + } +} +impl PartialEq for SecondExpr { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.timezone.eq(&other.timezone) + } +} + impl SecondExpr { pub fn new(child: Arc, timezone: String) -> Self { SecondExpr { child, timezone } @@ -249,15 +250,6 @@ impl Display for SecondExpr { } } -impl PartialEq for SecondExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.timezone.eq(&x.timezone)) - .unwrap_or(false) - } -} - impl PhysicalExpr for SecondExpr { fn as_any(&self) -> &dyn Any { self @@ -311,16 +303,9 @@ impl PhysicalExpr for SecondExpr { self.timezone.clone(), ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.timezone.hash(&mut s); - self.hash(&mut s); - } } -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct DateTruncExpr { /// An array with DataType::Date32 child: Arc, @@ -328,6 +313,18 @@ pub struct DateTruncExpr { format: Arc, } +impl Hash for DateTruncExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.format.hash(state); + } +} +impl PartialEq for DateTruncExpr { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.format.eq(&other.format) + } +} + impl DateTruncExpr { pub fn new(child: Arc, format: Arc) -> Self { DateTruncExpr { child, format } @@ -344,15 +341,6 @@ impl Display for DateTruncExpr { } } -impl PartialEq for DateTruncExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.format.eq(&x.format)) - .unwrap_or(false) - } -} - impl PhysicalExpr for DateTruncExpr { fn as_any(&self) -> &dyn Any { self @@ -398,16 +386,9 @@ impl PhysicalExpr for DateTruncExpr { Arc::clone(&self.format), ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.format.hash(&mut s); - self.hash(&mut s); - } } -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct TimestampTruncExpr { /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) child: Arc, @@ -422,6 +403,21 @@ pub struct TimestampTruncExpr { timezone: String, } +impl Hash for TimestampTruncExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.format.hash(state); + self.timezone.hash(state); + } +} +impl PartialEq for TimestampTruncExpr { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) + && self.format.eq(&other.format) + && self.timezone.eq(&other.timezone) + } +} + impl TimestampTruncExpr { pub fn new( child: Arc, @@ -446,19 +442,6 @@ impl Display for TimestampTruncExpr { } } -impl PartialEq for TimestampTruncExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.child.eq(&x.child) - && self.format.eq(&x.format) - && self.timezone.eq(&x.timezone) - }) - .unwrap_or(false) - } -} - impl PhysicalExpr for TimestampTruncExpr { fn as_any(&self) -> &dyn Any { self @@ -524,12 +507,4 @@ impl PhysicalExpr for TimestampTruncExpr { self.timezone.clone(), ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.format.hash(&mut s); - self.timezone.hash(&mut s); - self.hash(&mut s); - } } diff --git a/native/spark-expr/src/to_json.rs b/native/spark-expr/src/to_json.rs index 1f68eb8606..91b46c6f04 100644 --- a/native/spark-expr/src/to_json.rs +++ b/native/spark-expr/src/to_json.rs @@ -29,11 +29,11 @@ use datafusion_expr::ColumnarValue; use datafusion_physical_expr::PhysicalExpr; use std::any::Any; use std::fmt::{Debug, Display, Formatter}; -use std::hash::{Hash, Hasher}; +use std::hash::Hash; use std::sync::Arc; /// to_json function -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct ToJson { /// The input to convert to JSON expr: Arc, @@ -41,6 +41,18 @@ pub struct ToJson { timezone: String, } +impl Hash for ToJson { + fn hash(&self, state: &mut H) { + self.expr.hash(state); + self.timezone.hash(state); + } +} +impl PartialEq for ToJson { + fn eq(&self, other: &Self) -> bool { + self.expr.eq(&other.expr) && self.timezone.eq(&other.timezone) + } +} + impl ToJson { pub fn new(expr: Arc, timezone: &str) -> Self { Self { @@ -101,13 +113,6 @@ impl PhysicalExpr for ToJson { &self.timezone, ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.expr.hash(&mut s); - self.timezone.hash(&mut s); - self.hash(&mut s); - } } /// Convert an array into a JSON value string representation diff --git a/native/spark-expr/src/unbound.rs b/native/spark-expr/src/unbound.rs index a6babd0f7e..14f68c9cd6 100644 --- a/native/spark-expr/src/unbound.rs +++ b/native/spark-expr/src/unbound.rs @@ -17,15 +17,10 @@ use arrow_array::RecordBatch; use arrow_schema::{DataType, Schema}; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion::physical_plan::ColumnarValue; use datafusion_common::{internal_err, Result}; use datafusion_physical_expr::PhysicalExpr; -use std::{ - any::Any, - hash::{Hash, Hasher}, - sync::Arc, -}; +use std::{hash::Hash, sync::Arc}; /// This is similar to `UnKnownColumn` in DataFusion, but it has data type. /// This is only used when the column is not bound to a schema, for example, the @@ -93,18 +88,4 @@ impl PhysicalExpr for UnboundColumn { ) -> Result> { Ok(self) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.hash(&mut s); - } -} - -impl PartialEq for UnboundColumn { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self == x) - .unwrap_or(false) - } } diff --git a/native/spark-expr/src/variance.rs b/native/spark-expr/src/variance.rs index 4370d89ff4..e71d713f59 100644 --- a/native/spark-expr/src/variance.rs +++ b/native/spark-expr/src/variance.rs @@ -15,20 +15,19 @@ // specific language governing permissions and limitations // under the License. -use std::{any::Any, sync::Arc}; +use std::any::Any; use arrow::{ array::{ArrayRef, Float64Array}, datatypes::{DataType, Field}, }; use datafusion::logical_expr::Accumulator; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{downcast_value, DataFusionError, Result, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::Volatility::Immutable; use datafusion_expr::{AggregateUDFImpl, Signature}; +use datafusion_physical_expr::expressions::format_state_name; use datafusion_physical_expr::expressions::StatsType; -use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; /// VAR_SAMP and VAR_POP aggregate expression /// The implementation mostly is the same as the DataFusion's implementation. The reason @@ -39,7 +38,6 @@ use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; pub struct Variance { name: String, signature: Signature, - expr: Arc, stats_type: StatsType, null_on_divide_by_zero: bool, } @@ -47,7 +45,6 @@ pub struct Variance { impl Variance { /// Create a new VARIANCE aggregate function pub fn new( - expr: Arc, name: impl Into, data_type: DataType, stats_type: StatsType, @@ -58,7 +55,6 @@ impl Variance { Self { name: name.into(), signature: Signature::numeric(1, Immutable), - expr, stats_type, null_on_divide_by_zero, } @@ -118,17 +114,6 @@ impl AggregateUDFImpl for Variance { } } -impl PartialEq for Variance { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name && self.expr.eq(&x.expr) && self.stats_type == x.stats_type - }) - .unwrap_or(false) - } -} - /// An accumulator to compute variance #[derive(Debug)] pub struct VarianceAccumulator { From 4f8ce757b2832730e3d6974848cf5d3d89072283 Mon Sep 17 00:00:00 2001 From: Dharan Aditya Date: Fri, 3 Jan 2025 01:13:02 +0530 Subject: [PATCH 48/83] feat: add support for array_contains expression (#1163) * feat: add support for array_contains expression * test: add unit test for array_contains function * Removes unnecessary case expression for handling null values * chore: Move more expressions from core crate to spark-expr crate (#1152) * move aggregate expressions to spark-expr crate * move more expressions * move benchmark * normalize_nan * bitwise not * comet scalar funcs * update bench imports * remove dead code (#1155) * fix: Spark 4.0-preview1 SPARK-47120 (#1156) ## Which issue does this PR close? Part of https://github.com/apache/datafusion-comet/issues/372 and https://github.com/apache/datafusion-comet/issues/551 ## Rationale for this change To be ready for Spark 4.0 ## What changes are included in this PR? This PR fixes the new test SPARK-47120 added in Spark 4.0 ## How are these changes tested? tests enabled * chore: Move string kernels and expressions to spark-expr crate (#1164) * Move string kernels and expressions to spark-expr crate * remove unused hash kernel * remove unused dependencies * chore: Move remaining expressions to spark-expr crate + some minor refactoring (#1165) * move CheckOverflow to spark-expr crate * move NegativeExpr to spark-expr crate * move UnboundColumn to spark-expr crate * move ExpandExec from execution::datafusion::operators to execution::operators * refactoring to remove datafusion subpackage * update imports in benches * fix * fix * chore: Add ignored tests for reading complex types from Parquet (#1167) * Add ignored tests for reading structs from Parquet * add basic map test * add tests for Map and Array * feat: Add Spark-compatible implementation of SchemaAdapterFactory (#1169) * Add Spark-compatible SchemaAdapterFactory implementation * remove prototype code * fix * refactor * implement more cast logic * implement more cast logic * add basic test * improve test * cleanup * fmt * add support for casting unsigned int to signed int * clippy * address feedback * fix test * fix: Document enabling comet explain plan usage in Spark (4.0) (#1176) * test: enabling Spark tests with offHeap requirement (#1177) ## Which issue does this PR close? ## Rationale for this change After https://github.com/apache/datafusion-comet/pull/1062 We have not running Spark tests for native execution ## What changes are included in this PR? Removed the off heap requirement for testing ## How are these changes tested? Bringing back Spark tests for native execution * feat: Improve shuffle metrics (second attempt) (#1175) * improve shuffle metrics * docs * more metrics * refactor * address feedback * fix: stddev_pop should not directly return 0.0 when count is 1.0 (#1184) * add test * fix * fix * fix * feat: Make native shuffle compression configurable and respect `spark.shuffle.compress` (#1185) * Make shuffle compression codec and level configurable * remove lz4 references * docs * update comment * clippy * fix benches * clippy * clippy * disable test for miri * remove lz4 reference from proto * minor: move shuffle classes from common to spark (#1193) * minor: refactor decodeBatches to make private in broadcast exchange (#1195) * minor: refactor prepare_output so that it does not require an ExecutionContext (#1194) * fix: fix missing explanation for then branch in case when (#1200) * minor: remove unused source files (#1202) * chore: Upgrade to DataFusion 44.0.0-rc2 (#1154) * move aggregate expressions to spark-expr crate * move more expressions * move benchmark * normalize_nan * bitwise not * comet scalar funcs * update bench imports * save * save * save * remove unused imports * clippy * implement more hashers * implement Hash and PartialEq * implement Hash and PartialEq * implement Hash and PartialEq * benches * fix ScalarUDFImpl.return_type failure * exclude test from miri * ignore correct test * ignore another test * remove miri checks * use return_type_from_exprs * Revert "use return_type_from_exprs" This reverts commit febc1f1ec1301f9b359fc23ad6a117224fce35b7. * use DF main branch * hacky workaround for regression in ScalarUDFImpl.return_type * fix repo url * pin to revision * bump to latest rev * bump to latest DF rev * bump DF to rev 9f530dd * add Cargo.lock * bump DF version * no default features * Revert "remove miri checks" This reverts commit 4638fe3aa5501966cd5d8b53acf26c698b10b3c9. * Update pin to DataFusion e99e02b9b9093ceb0c13a2dd32a2a89beba47930 * update pin * Update Cargo.toml Bump to 44.0.0-rc2 * update cargo lock * revert miri change --------- Co-authored-by: Andrew Lamb * update UT Signed-off-by: Dharan Aditya * fix typo in UT Signed-off-by: Dharan Aditya --------- Signed-off-by: Dharan Aditya Co-authored-by: Andy Grove Co-authored-by: KAZUYUKI TANIMURA Co-authored-by: Parth Chandra Co-authored-by: Liang-Chi Hsieh Co-authored-by: Raz Luvaton <16746759+rluvaton@users.noreply.github.com> Co-authored-by: Andrew Lamb --- native/core/src/execution/planner.rs | 15 +++++++++++++++ native/proto/src/proto/expr.proto | 1 + .../org/apache/comet/serde/QueryPlanSerde.scala | 6 ++++++ .../org/apache/comet/CometExpressionSuite.scala | 12 ++++++++++++ 4 files changed, 34 insertions(+) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 5a35c62e33..c40e2e73f9 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -98,6 +98,7 @@ use datafusion_expr::{ AggregateUDF, ScalarUDF, WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, }; +use datafusion_functions_nested::array_has::ArrayHas; use datafusion_physical_expr::expressions::{Literal, StatsType}; use datafusion_physical_expr::window::WindowExpr; use datafusion_physical_expr::LexOrdering; @@ -719,6 +720,20 @@ impl PhysicalPlanner { expr.legacy_negative_index, ))) } + ExprStruct::ArrayContains(expr) => { + let src_array_expr = + self.create_expr(expr.left.as_ref().unwrap(), Arc::clone(&input_schema))?; + let key_expr = + self.create_expr(expr.right.as_ref().unwrap(), Arc::clone(&input_schema))?; + let args = vec![Arc::clone(&src_array_expr), key_expr]; + let array_has_expr = Arc::new(ScalarFunctionExpr::new( + "array_has", + Arc::new(ScalarUDF::new_from_impl(ArrayHas::new())), + args, + DataType::Boolean, + )); + Ok(array_has_expr) + } expr => Err(ExecutionError::GeneralError(format!( "Not implemented: {:?}", expr diff --git a/native/proto/src/proto/expr.proto b/native/proto/src/proto/expr.proto index 7a8ea78d57..e76ecdccf1 100644 --- a/native/proto/src/proto/expr.proto +++ b/native/proto/src/proto/expr.proto @@ -84,6 +84,7 @@ message Expr { GetArrayStructFields get_array_struct_fields = 57; BinaryExpr array_append = 58; ArrayInsert array_insert = 59; + BinaryExpr array_contains = 60; } } diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index 518fa06858..dc081b196b 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -2266,6 +2266,12 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim withInfo(expr, "unsupported arguments for GetArrayStructFields", child) None } + case expr if expr.prettyName == "array_contains" => + createBinaryExpr( + expr.children(0), + expr.children(1), + inputs, + (builder, binaryExpr) => builder.setArrayContains(binaryExpr)) case _ if expr.prettyName == "array_append" => createBinaryExpr( expr.children(0), diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index cce7cb20a1..36d3706506 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -2517,4 +2517,16 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { checkSparkAnswer(df.select("arrUnsupportedArgs")) } } + + test("array_contains") { + withTempDir { dir => + val path = new Path(dir.toURI.toString, "test.parquet") + makeParquetFileAllTypes(path, dictionaryEnabled = false, n = 10000) + spark.read.parquet(path.toString).createOrReplaceTempView("t1"); + checkSparkAnswerAndOperator( + spark.sql("SELECT array_contains(array(_2, _3, _4), _2) FROM t1")) + checkSparkAnswerAndOperator( + spark.sql("SELECT array_contains((CASE WHEN _2 =_3 THEN array(_4) END), _4) FROM t1")); + } + } } From 9320aedc8df2e8f7e5acb42ecdc44f33dff5d592 Mon Sep 17 00:00:00 2001 From: Kristin Cowalcijk Date: Fri, 3 Jan 2025 14:33:48 +0800 Subject: [PATCH 49/83] feat: Add a `spark.comet.exec.memoryPool` configuration for experimenting with various datafusion memory pool setups. (#1021) --- .../scala/org/apache/comet/CometConf.scala | 9 + docs/source/user-guide/configs.md | 1 + native/core/src/execution/jni_api.rs | 211 ++++++++++++++++-- .../org/apache/comet/CometExecIterator.scala | 29 ++- .../main/scala/org/apache/comet/Native.scala | 3 + 5 files changed, 231 insertions(+), 22 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 8815ac4eb8..2fff0a04c3 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -467,6 +467,15 @@ object CometConf extends ShimCometConf { .booleanConf .createWithDefault(false) + val COMET_EXEC_MEMORY_POOL_TYPE: ConfigEntry[String] = conf("spark.comet.exec.memoryPool") + .doc( + "The type of memory pool to be used for Comet native execution. " + + "Available memory pool types are 'greedy', 'fair_spill', 'greedy_task_shared', " + + "'fair_spill_task_shared', 'greedy_global' and 'fair_spill_global', By default, " + + "this config is 'greedy_task_shared'.") + .stringConf + .createWithDefault("greedy_task_shared") + val COMET_SCAN_PREFETCH_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.scan.preFetch.enabled") .doc("Whether to enable pre-fetching feature of CometScan.") diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 7881f07632..ecea70254a 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -48,6 +48,7 @@ Comet provides the following configuration settings. | 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. | 0.7 | +| spark.comet.exec.memoryPool | The type of memory pool to be used for Comet native execution. Available memory pool types are 'greedy', 'fair_spill', 'greedy_task_shared', 'fair_spill_task_shared', 'greedy_global' and 'fair_spill_global', By default, this config is 'greedy_task_shared'. | greedy_task_shared | | 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.compression.codec | The codec of Comet native shuffle used to compress shuffle data. Only zstd is supported. Compression can be disabled by setting spark.shuffle.compress=false. | zstd | diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index 09caf5e279..b1190d9059 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -24,6 +24,9 @@ use datafusion::{ physical_plan::{display::DisplayableExecutionPlan, SendableRecordBatchStream}, prelude::{SessionConfig, SessionContext}, }; +use datafusion_execution::memory_pool::{ + FairSpillPool, GreedyMemoryPool, MemoryPool, TrackConsumersPool, +}; use futures::poll; use jni::{ errors::Result as JNIResult, @@ -51,20 +54,26 @@ use datafusion_comet_proto::spark_operator::Operator; use datafusion_common::ScalarValue; use datafusion_execution::runtime_env::RuntimeEnvBuilder; use futures::stream::StreamExt; +use jni::sys::JNI_FALSE; use jni::{ objects::GlobalRef, sys::{jboolean, jdouble, jintArray, jobjectArray, jstring}, }; +use std::num::NonZeroUsize; +use std::sync::Mutex; use tokio::runtime::Runtime; use crate::execution::operators::ScanExec; use crate::execution::spark_plan::SparkPlan; use log::info; +use once_cell::sync::{Lazy, OnceCell}; /// Comet native execution context. Kept alive across JNI calls. struct ExecutionContext { /// The id of the execution context. pub id: i64, + /// Task attempt id + pub task_attempt_id: i64, /// The deserialized Spark plan pub spark_plan: Operator, /// The DataFusion root operator converted from the `spark_plan` @@ -89,6 +98,51 @@ struct ExecutionContext { pub explain_native: bool, /// Map of metrics name -> jstring object to cache jni_NewStringUTF calls. pub metrics_jstrings: HashMap>, + /// Memory pool config + pub memory_pool_config: MemoryPoolConfig, +} + +#[derive(PartialEq, Eq)] +enum MemoryPoolType { + Unified, + Greedy, + FairSpill, + GreedyTaskShared, + FairSpillTaskShared, + GreedyGlobal, + FairSpillGlobal, +} + +struct MemoryPoolConfig { + pool_type: MemoryPoolType, + pool_size: usize, +} + +impl MemoryPoolConfig { + fn new(pool_type: MemoryPoolType, pool_size: usize) -> Self { + Self { + pool_type, + pool_size, + } + } +} + +/// The per-task memory pools keyed by task attempt id. +static TASK_SHARED_MEMORY_POOLS: Lazy>> = + Lazy::new(|| Mutex::new(HashMap::new())); + +struct PerTaskMemoryPool { + memory_pool: Arc, + num_plans: usize, +} + +impl PerTaskMemoryPool { + fn new(memory_pool: Arc) -> Self { + Self { + memory_pool, + num_plans: 0, + } + } } /// Accept serialized query plan and return the address of the native query plan. @@ -105,8 +159,11 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( comet_task_memory_manager_obj: JObject, batch_size: jint, use_unified_memory_manager: jboolean, + memory_pool_type: jstring, memory_limit: jlong, + memory_limit_per_task: jlong, memory_fraction: jdouble, + task_attempt_id: jlong, debug_native: jboolean, explain_native: jboolean, worker_threads: jint, @@ -145,21 +202,27 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( let task_memory_manager = Arc::new(jni_new_global_ref!(env, comet_task_memory_manager_obj)?); + let memory_pool_type = env.get_string(&JString::from_raw(memory_pool_type))?.into(); + let memory_pool_config = parse_memory_pool_config( + use_unified_memory_manager != JNI_FALSE, + memory_pool_type, + memory_limit, + memory_limit_per_task, + memory_fraction, + )?; + let memory_pool = + create_memory_pool(&memory_pool_config, task_memory_manager, task_attempt_id); + // We need to keep the session context alive. Some session state like temporary // dictionaries are stored in session context. If it is dropped, the temporary // dictionaries will be dropped as well. - let session = prepare_datafusion_session_context( - batch_size as usize, - use_unified_memory_manager == 1, - memory_limit as usize, - memory_fraction, - task_memory_manager, - )?; + let session = prepare_datafusion_session_context(batch_size as usize, memory_pool)?; let plan_creation_time = start.elapsed(); let exec_context = Box::new(ExecutionContext { id, + task_attempt_id, spark_plan, root_op: None, scans: vec![], @@ -172,6 +235,7 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( debug_native: debug_native == 1, explain_native: explain_native == 1, metrics_jstrings: HashMap::new(), + memory_pool_config, }); Ok(Box::into_raw(exec_context) as i64) @@ -181,22 +245,10 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( /// Configure DataFusion session context. fn prepare_datafusion_session_context( batch_size: usize, - use_unified_memory_manager: bool, - memory_limit: usize, - memory_fraction: f64, - comet_task_memory_manager: Arc, + memory_pool: Arc, ) -> CometResult { let mut rt_config = RuntimeEnvBuilder::new().with_disk_manager(DiskManagerConfig::NewOs); - - // Check if we are using unified memory manager integrated with Spark. - if use_unified_memory_manager { - // Set Comet memory pool for native - let memory_pool = CometMemoryPool::new(comet_task_memory_manager); - rt_config = rt_config.with_memory_pool(Arc::new(memory_pool)); - } else { - // Use the memory pool from DF - rt_config = rt_config.with_memory_limit(memory_limit, memory_fraction) - } + rt_config = rt_config.with_memory_pool(memory_pool); // Get Datafusion configuration from Spark Execution context // can be configured in Comet Spark JVM using Spark --conf parameters @@ -224,6 +276,107 @@ fn prepare_datafusion_session_context( Ok(session_ctx) } +fn parse_memory_pool_config( + use_unified_memory_manager: bool, + memory_pool_type: String, + memory_limit: i64, + memory_limit_per_task: i64, + memory_fraction: f64, +) -> CometResult { + let memory_pool_config = if use_unified_memory_manager { + MemoryPoolConfig::new(MemoryPoolType::Unified, 0) + } else { + // Use the memory pool from DF + let pool_size = (memory_limit as f64 * memory_fraction) as usize; + let pool_size_per_task = (memory_limit_per_task as f64 * memory_fraction) as usize; + match memory_pool_type.as_str() { + "fair_spill_task_shared" => { + MemoryPoolConfig::new(MemoryPoolType::FairSpillTaskShared, pool_size_per_task) + } + "greedy_task_shared" => { + MemoryPoolConfig::new(MemoryPoolType::GreedyTaskShared, pool_size_per_task) + } + "fair_spill_global" => { + MemoryPoolConfig::new(MemoryPoolType::FairSpillGlobal, pool_size) + } + "greedy_global" => MemoryPoolConfig::new(MemoryPoolType::GreedyGlobal, pool_size), + "fair_spill" => MemoryPoolConfig::new(MemoryPoolType::FairSpill, pool_size_per_task), + "greedy" => MemoryPoolConfig::new(MemoryPoolType::Greedy, pool_size_per_task), + _ => { + return Err(CometError::Config(format!( + "Unsupported memory pool type: {}", + memory_pool_type + ))) + } + } + }; + Ok(memory_pool_config) +} + +fn create_memory_pool( + memory_pool_config: &MemoryPoolConfig, + comet_task_memory_manager: Arc, + task_attempt_id: i64, +) -> Arc { + const NUM_TRACKED_CONSUMERS: usize = 10; + match memory_pool_config.pool_type { + MemoryPoolType::Unified => { + // Set Comet memory pool for native + let memory_pool = CometMemoryPool::new(comet_task_memory_manager); + Arc::new(memory_pool) + } + MemoryPoolType::Greedy => Arc::new(TrackConsumersPool::new( + GreedyMemoryPool::new(memory_pool_config.pool_size), + NonZeroUsize::new(NUM_TRACKED_CONSUMERS).unwrap(), + )), + MemoryPoolType::FairSpill => Arc::new(TrackConsumersPool::new( + FairSpillPool::new(memory_pool_config.pool_size), + NonZeroUsize::new(NUM_TRACKED_CONSUMERS).unwrap(), + )), + MemoryPoolType::GreedyGlobal => { + static GLOBAL_MEMORY_POOL_GREEDY: OnceCell> = OnceCell::new(); + let memory_pool = GLOBAL_MEMORY_POOL_GREEDY.get_or_init(|| { + Arc::new(TrackConsumersPool::new( + GreedyMemoryPool::new(memory_pool_config.pool_size), + NonZeroUsize::new(NUM_TRACKED_CONSUMERS).unwrap(), + )) + }); + Arc::clone(memory_pool) + } + MemoryPoolType::FairSpillGlobal => { + static GLOBAL_MEMORY_POOL_FAIR: OnceCell> = OnceCell::new(); + let memory_pool = GLOBAL_MEMORY_POOL_FAIR.get_or_init(|| { + Arc::new(TrackConsumersPool::new( + FairSpillPool::new(memory_pool_config.pool_size), + NonZeroUsize::new(NUM_TRACKED_CONSUMERS).unwrap(), + )) + }); + Arc::clone(memory_pool) + } + MemoryPoolType::GreedyTaskShared | MemoryPoolType::FairSpillTaskShared => { + let mut memory_pool_map = TASK_SHARED_MEMORY_POOLS.lock().unwrap(); + let per_task_memory_pool = + memory_pool_map.entry(task_attempt_id).or_insert_with(|| { + let pool: Arc = + if memory_pool_config.pool_type == MemoryPoolType::GreedyTaskShared { + Arc::new(TrackConsumersPool::new( + GreedyMemoryPool::new(memory_pool_config.pool_size), + NonZeroUsize::new(NUM_TRACKED_CONSUMERS).unwrap(), + )) + } else { + Arc::new(TrackConsumersPool::new( + FairSpillPool::new(memory_pool_config.pool_size), + NonZeroUsize::new(NUM_TRACKED_CONSUMERS).unwrap(), + )) + }; + PerTaskMemoryPool::new(pool) + }); + per_task_memory_pool.num_plans += 1; + Arc::clone(&per_task_memory_pool.memory_pool) + } + } +} + /// Prepares arrow arrays for output. fn prepare_output( env: &mut JNIEnv, @@ -407,6 +560,22 @@ pub extern "system" fn Java_org_apache_comet_Native_releasePlan( ) { try_unwrap_or_throw(&e, |_| unsafe { let execution_context = get_execution_context(exec_context); + if execution_context.memory_pool_config.pool_type == MemoryPoolType::FairSpillTaskShared + || execution_context.memory_pool_config.pool_type == MemoryPoolType::GreedyTaskShared + { + // Decrement the number of native plans using the per-task shared memory pool, and + // remove the memory pool if the released native plan is the last native plan using it. + let task_attempt_id = execution_context.task_attempt_id; + let mut memory_pool_map = TASK_SHARED_MEMORY_POOLS.lock().unwrap(); + if let Some(per_task_memory_pool) = memory_pool_map.get_mut(&task_attempt_id) { + per_task_memory_pool.num_plans -= 1; + if per_task_memory_pool.num_plans == 0 { + // Drop the memory pool from the per-task memory pool map if there are no + // more native plans using it. + memory_pool_map.remove(&task_attempt_id); + } + } + } let _: Box = Box::from_raw(execution_context); Ok(()) }) diff --git a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala index 04d9306951..0b90a91c74 100644 --- a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala +++ b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala @@ -23,7 +23,7 @@ import org.apache.spark._ import org.apache.spark.sql.comet.CometMetricNode import org.apache.spark.sql.vectorized._ -import org.apache.comet.CometConf.{COMET_BATCH_SIZE, COMET_BLOCKING_THREADS, COMET_DEBUG_ENABLED, COMET_EXEC_MEMORY_FRACTION, COMET_EXPLAIN_NATIVE_ENABLED, COMET_WORKER_THREADS} +import org.apache.comet.CometConf.{COMET_BATCH_SIZE, COMET_BLOCKING_THREADS, COMET_DEBUG_ENABLED, COMET_EXEC_MEMORY_FRACTION, COMET_EXEC_MEMORY_POOL_TYPE, COMET_EXPLAIN_NATIVE_ENABLED, COMET_WORKER_THREADS} import org.apache.comet.vector.NativeUtil /** @@ -72,8 +72,11 @@ class CometExecIterator( new CometTaskMemoryManager(id), batchSize = COMET_BATCH_SIZE.get(), use_unified_memory_manager = conf.getBoolean("spark.memory.offHeap.enabled", false), + memory_pool_type = COMET_EXEC_MEMORY_POOL_TYPE.get(), memory_limit = CometSparkSessionExtensions.getCometMemoryOverhead(conf), + memory_limit_per_task = getMemoryLimitPerTask(conf), memory_fraction = COMET_EXEC_MEMORY_FRACTION.get(), + task_attempt_id = TaskContext.get().taskAttemptId, debug = COMET_DEBUG_ENABLED.get(), explain = COMET_EXPLAIN_NATIVE_ENABLED.get(), workerThreads = COMET_WORKER_THREADS.get(), @@ -84,6 +87,30 @@ class CometExecIterator( private var currentBatch: ColumnarBatch = null private var closed: Boolean = false + private def getMemoryLimitPerTask(conf: SparkConf): Long = { + val numCores = numDriverOrExecutorCores(conf).toFloat + val maxMemory = CometSparkSessionExtensions.getCometMemoryOverhead(conf) + val coresPerTask = conf.get("spark.task.cpus", "1").toFloat + // example 16GB maxMemory * 16 cores with 4 cores per task results + // in memory_limit_per_task = 16 GB * 4 / 16 = 16 GB / 4 = 4GB + (maxMemory.toFloat * coresPerTask / numCores).toLong + } + + private def numDriverOrExecutorCores(conf: SparkConf): Int = { + def convertToInt(threads: String): Int = { + if (threads == "*") Runtime.getRuntime.availableProcessors() else threads.toInt + } + val LOCAL_N_REGEX = """local\[([0-9]+|\*)\]""".r + val LOCAL_N_FAILURES_REGEX = """local\[([0-9]+|\*)\s*,\s*([0-9]+)\]""".r + val master = conf.get("spark.master") + master match { + case "local" => 1 + case LOCAL_N_REGEX(threads) => convertToInt(threads) + case LOCAL_N_FAILURES_REGEX(threads, _) => convertToInt(threads) + case _ => conf.get("spark.executor.cores", "1").toInt + } + } + def getNextBatch(): Option[ColumnarBatch] = { assert(partitionIndex >= 0 && partitionIndex < numParts) diff --git a/spark/src/main/scala/org/apache/comet/Native.scala b/spark/src/main/scala/org/apache/comet/Native.scala index 083c0f2b50..5fd84989ba 100644 --- a/spark/src/main/scala/org/apache/comet/Native.scala +++ b/spark/src/main/scala/org/apache/comet/Native.scala @@ -52,8 +52,11 @@ class Native extends NativeBase { taskMemoryManager: CometTaskMemoryManager, batchSize: Int, use_unified_memory_manager: Boolean, + memory_pool_type: String, memory_limit: Long, + memory_limit_per_task: Long, memory_fraction: Double, + task_attempt_id: Long, debug: Boolean, explain: Boolean, workerThreads: Int, From 2e0f00a486e9547a58c08e61dd6a5f30b82e4e94 Mon Sep 17 00:00:00 2001 From: Oleks V Date: Fri, 3 Jan 2025 10:39:58 -0800 Subject: [PATCH 50/83] feat: Reenable tests for filtered SMJ anti join (#1211) * feat: reenable filtered SMJ Anti join tests * feat: reenable filtered SMJ Anti join tests * feat: reenable filtered SMJ Anti join tests * feat: reenable filtered SMJ Anti join tests * Add CoalesceBatchesExec around SMJ with join filter * adding `CoalesceBatches` * adding `CoalesceBatches` * adding `CoalesceBatches` * feat: reenable filtered SMJ Anti join tests * feat: reenable filtered SMJ Anti join tests --------- Co-authored-by: Andy Grove --- native/core/src/execution/planner.rs | 48 ++++++++++++++----- .../comet/CometSparkSessionExtensions.scala | 2 +- .../apache/comet/serde/QueryPlanSerde.scala | 6 +-- .../apache/comet/exec/CometJoinSuite.scala | 11 ++--- 4 files changed, 43 insertions(+), 24 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index c40e2e73f9..da452c2f15 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -70,6 +70,7 @@ use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctio use crate::execution::shuffle::CompressionCodec; use crate::execution::spark_plan::SparkPlan; +use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_comet_proto::{ spark_expression::{ self, agg_expr::ExprStruct as AggExprStruct, expr::ExprStruct, literal::Value, AggExpr, @@ -1183,17 +1184,42 @@ impl PhysicalPlanner { false, )?); - Ok(( - scans, - Arc::new(SparkPlan::new( - spark_plan.plan_id, - join, - vec![ - Arc::clone(&join_params.left), - Arc::clone(&join_params.right), - ], - )), - )) + if join.filter.is_some() { + // SMJ with join filter produces lots of tiny batches + let coalesce_batches: Arc = + Arc::new(CoalesceBatchesExec::new( + Arc::::clone(&join), + self.session_ctx + .state() + .config_options() + .execution + .batch_size, + )); + Ok(( + scans, + Arc::new(SparkPlan::new_with_additional( + spark_plan.plan_id, + coalesce_batches, + vec![ + Arc::clone(&join_params.left), + Arc::clone(&join_params.right), + ], + vec![join], + )), + )) + } else { + Ok(( + scans, + Arc::new(SparkPlan::new( + spark_plan.plan_id, + join, + vec![ + Arc::clone(&join_params.left), + Arc::clone(&join_params.right), + ], + )), + )) + } } OpStruct::HashJoin(join) => { let (join_params, scans) = self.parse_join_parameters( diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index 8bff6b5fbd..52a0d5e180 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -567,7 +567,7 @@ class CometSparkSessionExtensions case op: SortMergeJoinExec if CometConf.COMET_EXEC_SORT_MERGE_JOIN_ENABLED.get(conf) && - op.children.forall(isCometNative(_)) => + op.children.forall(isCometNative) => val newOp = transform1(op) newOp match { case Some(nativeOp) => 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 dc081b196b..7ed3725bed 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -2859,11 +2859,7 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim case RightOuter => JoinType.RightOuter case FullOuter => JoinType.FullOuter case LeftSemi => JoinType.LeftSemi - // TODO: DF SMJ with join condition fails TPCH q21 - case LeftAnti if condition.isEmpty => JoinType.LeftAnti - case LeftAnti => - withInfo(join, "LeftAnti SMJ join with condition is not supported") - return None + case LeftAnti => JoinType.LeftAnti case _ => // Spark doesn't support other join types withInfo(op, s"Unsupported join type ${join.joinType}") 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 ad1aef4a8f..d756da1515 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala @@ -216,7 +216,7 @@ class CometJoinSuite extends CometTestBase { v.toDouble, v.toString, v % 2 == 0, - v.toString().getBytes, + v.toString.getBytes, Decimal(v)) withParquetTable((0 until 10).map(i => manyTypes(i, i % 5)), "tbl_a") { @@ -294,6 +294,7 @@ class CometJoinSuite extends CometTestBase { test("SortMergeJoin without join filter") { withSQLConf( + CometConf.COMET_EXEC_SORT_MERGE_JOIN_ENABLED.key -> "true", SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { withParquetTable((0 until 10).map(i => (i, i % 5)), "tbl_a") { @@ -338,9 +339,9 @@ class CometJoinSuite extends CometTestBase { } } - // https://github.com/apache/datafusion-comet/issues/398 - ignore("SortMergeJoin with join filter") { + test("SortMergeJoin with join filter") { withSQLConf( + CometConf.COMET_EXEC_SORT_MERGE_JOIN_ENABLED.key -> "true", CometConf.COMET_EXEC_SORT_MERGE_JOIN_WITH_JOIN_FILTER_ENABLED.key -> "true", SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { @@ -391,9 +392,6 @@ class CometJoinSuite extends CometTestBase { "AND tbl_a._2 >= tbl_b._1") checkSparkAnswerAndOperator(df9) - // TODO: Enable these tests after fixing the issue: - // https://github.com/apache/datafusion-comet/issues/861 - /* val df10 = sql( "SELECT * FROM tbl_a LEFT ANTI JOIN tbl_b ON tbl_a._2 = tbl_b._1 " + "AND tbl_a._2 >= tbl_b._1") @@ -403,7 +401,6 @@ class CometJoinSuite extends CometTestBase { "SELECT * FROM tbl_b LEFT ANTI JOIN tbl_a ON tbl_a._2 = tbl_b._1 " + "AND tbl_a._2 >= tbl_b._1") checkSparkAnswerAndOperator(df11) - */ } } } From 4333dce35090911ab3f0a5ae7d0f06e8991bb6c7 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 3 Jan 2025 11:12:14 -0800 Subject: [PATCH 51/83] chore: Add safety check to CometBuffer (#1050) * chore: Add safety check to CometBuffer * Add CometColumnarToRowExec * fix * fix * more * Update plan stability results * fix * fix * fix * Revert "fix" This reverts commit 9bad173c7751f105bf3ded2ebc2fed0737d1b909. * Revert "Revert "fix"" This reverts commit d527ad1a365d3aff64200ceba6d11cf376f3919f. * fix BucketedReadWithoutHiveSupportSuite * fix SparkPlanSuite --- .../apache/comet/parquet/ColumnReader.java | 22 -- .../comet/parquet/ConstantColumnReader.java | 4 +- .../comet/parquet/MetadataColumnReader.java | 9 +- .../comet/parquet/RowIndexColumnReader.java | 2 +- .../apache/comet/vector/CometPlainVector.java | 16 ++ dev/diffs/3.4.3.diff | 38 +++- dev/diffs/3.5.1.diff | 40 +++- dev/diffs/4.0.0-preview1.diff | 34 ++- native/core/benches/parquet_read.rs | 2 +- native/core/src/common/buffer.rs | 50 +++-- native/core/src/execution/operators/copy.rs | 5 +- native/core/src/parquet/mod.rs | 2 +- native/core/src/parquet/mutable_vector.rs | 14 +- native/core/src/parquet/read/column.rs | 5 +- .../org/apache/comet/CometExecIterator.scala | 10 + .../comet/CometSparkSessionExtensions.scala | 13 ++ .../sql/comet/CometColumnarToRowExec.scala | 209 ++++++++++++++++++ .../q1/explain.txt | 8 +- .../q1/simplified.txt | 4 +- .../q10/explain.txt | 24 +- .../q10/simplified.txt | 12 +- .../q11/explain.txt | 12 +- .../q11/simplified.txt | 6 +- .../q12/explain.txt | 8 +- .../q12/simplified.txt | 4 +- .../q13/explain.txt | 8 +- .../q13/simplified.txt | 4 +- .../q14a/explain.txt | 16 +- .../q14a/simplified.txt | 8 +- .../q14b/explain.txt | 28 +-- .../q14b/simplified.txt | 14 +- .../q15/explain.txt | 8 +- .../q15/simplified.txt | 4 +- .../q16/explain.txt | 4 +- .../q16/simplified.txt | 2 +- .../q17/explain.txt | 12 +- .../q17/simplified.txt | 6 +- .../q18/explain.txt | 8 +- .../q18/simplified.txt | 4 +- .../q19/explain.txt | 4 +- .../q19/simplified.txt | 2 +- .../q2/explain.txt | 4 +- .../q2/simplified.txt | 2 +- .../q20/explain.txt | 8 +- .../q20/simplified.txt | 4 +- .../q21/explain.txt | 8 +- .../q21/simplified.txt | 4 +- .../q22/explain.txt | 8 +- .../q22/simplified.txt | 4 +- .../q23a/explain.txt | 20 +- .../q23a/simplified.txt | 10 +- .../q23b/explain.txt | 20 +- .../q23b/simplified.txt | 10 +- .../q24a/explain.txt | 12 +- .../q24a/simplified.txt | 6 +- .../q24b/explain.txt | 12 +- .../q24b/simplified.txt | 6 +- .../q25/explain.txt | 12 +- .../q25/simplified.txt | 6 +- .../q26/explain.txt | 8 +- .../q26/simplified.txt | 4 +- .../q27/explain.txt | 8 +- .../q27/simplified.txt | 4 +- .../q28/explain.txt | 24 +- .../q28/simplified.txt | 12 +- .../q29/explain.txt | 16 +- .../q29/simplified.txt | 8 +- .../q3/explain.txt | 4 +- .../q3/simplified.txt | 2 +- .../q30/explain.txt | 8 +- .../q30/simplified.txt | 4 +- .../q31/explain.txt | 16 +- .../q31/simplified.txt | 8 +- .../q32/explain.txt | 8 +- .../q32/simplified.txt | 4 +- .../q33/explain.txt | 8 +- .../q33/simplified.txt | 4 +- .../q34/explain.txt | 8 +- .../q34/simplified.txt | 4 +- .../q35/explain.txt | 24 +- .../q35/simplified.txt | 12 +- .../q36/explain.txt | 8 +- .../q36/simplified.txt | 4 +- .../q37/explain.txt | 8 +- .../q37/simplified.txt | 4 +- .../q38/explain.txt | 8 +- .../q38/simplified.txt | 4 +- .../q39a/explain.txt | 12 +- .../q39a/simplified.txt | 6 +- .../q39b/explain.txt | 12 +- .../q39b/simplified.txt | 6 +- .../q4/explain.txt | 12 +- .../q4/simplified.txt | 6 +- .../q40/explain.txt | 8 +- .../q40/simplified.txt | 4 +- .../q41/explain.txt | 4 +- .../q41/simplified.txt | 2 +- .../q42/explain.txt | 4 +- .../q42/simplified.txt | 2 +- .../q43/explain.txt | 4 +- .../q43/simplified.txt | 2 +- .../q44/explain.txt | 16 +- .../q44/simplified.txt | 8 +- .../q45/explain.txt | 12 +- .../q45/simplified.txt | 6 +- .../q46/explain.txt | 8 +- .../q46/simplified.txt | 4 +- .../q47/explain.txt | 16 +- .../q47/simplified.txt | 8 +- .../q48/explain.txt | 8 +- .../q48/simplified.txt | 4 +- .../q49/explain.txt | 16 +- .../q49/simplified.txt | 8 +- .../q5/explain.txt | 16 +- .../q5/simplified.txt | 8 +- .../q50/explain.txt | 8 +- .../q50/simplified.txt | 4 +- .../q51/explain.txt | 12 +- .../q51/simplified.txt | 6 +- .../q52/explain.txt | 4 +- .../q52/simplified.txt | 2 +- .../q53/explain.txt | 8 +- .../q53/simplified.txt | 4 +- .../q54/explain.txt | 20 +- .../q54/simplified.txt | 10 +- .../q55/explain.txt | 4 +- .../q55/simplified.txt | 2 +- .../q56/explain.txt | 8 +- .../q56/simplified.txt | 4 +- .../q57/explain.txt | 16 +- .../q57/simplified.txt | 8 +- .../q58/explain.txt | 12 +- .../q58/simplified.txt | 6 +- .../q59/explain.txt | 4 +- .../q59/simplified.txt | 2 +- .../q6/explain.txt | 12 +- .../q6/simplified.txt | 6 +- .../q60/explain.txt | 8 +- .../q60/simplified.txt | 4 +- .../q61/explain.txt | 12 +- .../q61/simplified.txt | 6 +- .../q62/explain.txt | 4 +- .../q62/simplified.txt | 2 +- .../q63/explain.txt | 8 +- .../q63/simplified.txt | 4 +- .../q64/explain.txt | 12 +- .../q64/simplified.txt | 6 +- .../q65/explain.txt | 8 +- .../q65/simplified.txt | 4 +- .../q66/explain.txt | 8 +- .../q66/simplified.txt | 4 +- .../q67/explain.txt | 8 +- .../q67/simplified.txt | 4 +- .../q68/explain.txt | 8 +- .../q68/simplified.txt | 4 +- .../q69/explain.txt | 24 +- .../q69/simplified.txt | 12 +- .../q7/explain.txt | 8 +- .../q7/simplified.txt | 4 +- .../q70/explain.txt | 16 +- .../q70/simplified.txt | 8 +- .../q71/explain.txt | 8 +- .../q71/simplified.txt | 4 +- .../q72/explain.txt | 8 +- .../q72/simplified.txt | 4 +- .../q73/explain.txt | 8 +- .../q73/simplified.txt | 4 +- .../q74/explain.txt | 12 +- .../q74/simplified.txt | 6 +- .../q75/explain.txt | 12 +- .../q75/simplified.txt | 6 +- .../q76/explain.txt | 4 +- .../q76/simplified.txt | 2 +- .../q77/explain.txt | 20 +- .../q77/simplified.txt | 10 +- .../q78/explain.txt | 8 +- .../q78/simplified.txt | 4 +- .../q79/explain.txt | 8 +- .../q79/simplified.txt | 4 +- .../q8/explain.txt | 8 +- .../q8/simplified.txt | 4 +- .../q80/explain.txt | 16 +- .../q80/simplified.txt | 8 +- .../q81/explain.txt | 8 +- .../q81/simplified.txt | 4 +- .../q82/explain.txt | 8 +- .../q82/simplified.txt | 4 +- .../q83/explain.txt | 8 +- .../q83/simplified.txt | 4 +- .../q84/explain.txt | 4 +- .../q84/simplified.txt | 2 +- .../q85/explain.txt | 8 +- .../q85/simplified.txt | 4 +- .../q86/explain.txt | 8 +- .../q86/simplified.txt | 4 +- .../q87/explain.txt | 16 +- .../q87/simplified.txt | 8 +- .../q88/explain.txt | 32 +-- .../q88/simplified.txt | 16 +- .../q89/explain.txt | 8 +- .../q89/simplified.txt | 4 +- .../q9/explain.txt | 24 +- .../q9/simplified.txt | 12 +- .../q90/explain.txt | 8 +- .../q90/simplified.txt | 4 +- .../q91/explain.txt | 8 +- .../q91/simplified.txt | 4 +- .../q92/explain.txt | 8 +- .../q92/simplified.txt | 4 +- .../q93/explain.txt | 4 +- .../q93/simplified.txt | 2 +- .../q94/explain.txt | 4 +- .../q94/simplified.txt | 2 +- .../q95/explain.txt | 4 +- .../q95/simplified.txt | 2 +- .../q96/explain.txt | 4 +- .../q96/simplified.txt | 2 +- .../q97/explain.txt | 8 +- .../q97/simplified.txt | 4 +- .../q98/explain.txt | 12 +- .../q98/simplified.txt | 6 +- .../q99/explain.txt | 4 +- .../q99/simplified.txt | 2 +- .../q1/explain.txt | 20 +- .../q1/simplified.txt | 10 +- .../q10/explain.txt | 24 +- .../q10/simplified.txt | 12 +- .../q11/explain.txt | 24 +- .../q11/simplified.txt | 12 +- .../q12/explain.txt | 8 +- .../q12/simplified.txt | 4 +- .../q13/explain.txt | 8 +- .../q13/simplified.txt | 4 +- .../q14a/explain.txt | 24 +- .../q14a/simplified.txt | 12 +- .../q14b/explain.txt | 32 +-- .../q14b/simplified.txt | 16 +- .../q15/explain.txt | 8 +- .../q15/simplified.txt | 4 +- .../q16/explain.txt | 4 +- .../q16/simplified.txt | 2 +- .../q17/explain.txt | 12 +- .../q17/simplified.txt | 6 +- .../q18/explain.txt | 8 +- .../q18/simplified.txt | 4 +- .../q19/explain.txt | 4 +- .../q19/simplified.txt | 2 +- .../q2/explain.txt | 16 +- .../q2/simplified.txt | 8 +- .../q20/explain.txt | 8 +- .../q20/simplified.txt | 4 +- .../q21/explain.txt | 8 +- .../q21/simplified.txt | 4 +- .../q22/explain.txt | 8 +- .../q22/simplified.txt | 4 +- .../q23a/explain.txt | 28 +-- .../q23a/simplified.txt | 14 +- .../q23b/explain.txt | 32 +-- .../q23b/simplified.txt | 16 +- .../q24a/explain.txt | 12 +- .../q24a/simplified.txt | 6 +- .../q24b/explain.txt | 12 +- .../q24b/simplified.txt | 6 +- .../q25/explain.txt | 12 +- .../q25/simplified.txt | 6 +- .../q26/explain.txt | 8 +- .../q26/simplified.txt | 4 +- .../q27/explain.txt | 8 +- .../q27/simplified.txt | 4 +- .../q28/explain.txt | 24 +- .../q28/simplified.txt | 12 +- .../q29/explain.txt | 16 +- .../q29/simplified.txt | 8 +- .../q3/explain.txt | 4 +- .../q3/simplified.txt | 2 +- .../q30/explain.txt | 20 +- .../q30/simplified.txt | 10 +- .../q31/explain.txt | 40 ++-- .../q31/simplified.txt | 20 +- .../q32/explain.txt | 12 +- .../q32/simplified.txt | 6 +- .../q33/explain.txt | 16 +- .../q33/simplified.txt | 8 +- .../q34/explain.txt | 8 +- .../q34/simplified.txt | 4 +- .../q35/explain.txt | 24 +- .../q35/simplified.txt | 12 +- .../q36/explain.txt | 8 +- .../q36/simplified.txt | 4 +- .../q37/explain.txt | 8 +- .../q37/simplified.txt | 4 +- .../q38/explain.txt | 8 +- .../q38/simplified.txt | 4 +- .../q39a/explain.txt | 20 +- .../q39a/simplified.txt | 10 +- .../q39b/explain.txt | 20 +- .../q39b/simplified.txt | 10 +- .../q4/explain.txt | 32 +-- .../q4/simplified.txt | 16 +- .../q40/explain.txt | 8 +- .../q40/simplified.txt | 4 +- .../q41/explain.txt | 4 +- .../q41/simplified.txt | 2 +- .../q42/explain.txt | 4 +- .../q42/simplified.txt | 2 +- .../q43/explain.txt | 4 +- .../q43/simplified.txt | 2 +- .../q44/explain.txt | 12 +- .../q44/simplified.txt | 6 +- .../q45/explain.txt | 12 +- .../q45/simplified.txt | 6 +- .../q46/explain.txt | 16 +- .../q46/simplified.txt | 8 +- .../q47/explain.txt | 8 +- .../q47/simplified.txt | 4 +- .../q48/explain.txt | 8 +- .../q48/simplified.txt | 4 +- .../q49/explain.txt | 16 +- .../q49/simplified.txt | 8 +- .../q5/explain.txt | 16 +- .../q5/simplified.txt | 8 +- .../q50/explain.txt | 8 +- .../q50/simplified.txt | 4 +- .../q51/explain.txt | 12 +- .../q51/simplified.txt | 6 +- .../q52/explain.txt | 4 +- .../q52/simplified.txt | 2 +- .../q53/explain.txt | 8 +- .../q53/simplified.txt | 4 +- .../q54/explain.txt | 20 +- .../q54/simplified.txt | 10 +- .../q55/explain.txt | 4 +- .../q55/simplified.txt | 2 +- .../q56/explain.txt | 16 +- .../q56/simplified.txt | 8 +- .../q57/explain.txt | 8 +- .../q57/simplified.txt | 4 +- .../q58/explain.txt | 20 +- .../q58/simplified.txt | 10 +- .../q59/explain.txt | 20 +- .../q59/simplified.txt | 10 +- .../q6/explain.txt | 20 +- .../q6/simplified.txt | 10 +- .../q60/explain.txt | 16 +- .../q60/simplified.txt | 8 +- .../q61/explain.txt | 12 +- .../q61/simplified.txt | 6 +- .../q62/explain.txt | 4 +- .../q62/simplified.txt | 2 +- .../q63/explain.txt | 8 +- .../q63/simplified.txt | 4 +- .../q64/explain.txt | 60 ++--- .../q64/simplified.txt | 30 +-- .../q65/explain.txt | 20 +- .../q65/simplified.txt | 10 +- .../q66/explain.txt | 12 +- .../q66/simplified.txt | 6 +- .../q67/explain.txt | 8 +- .../q67/simplified.txt | 4 +- .../q68/explain.txt | 16 +- .../q68/simplified.txt | 8 +- .../q69/explain.txt | 24 +- .../q69/simplified.txt | 12 +- .../q7/explain.txt | 8 +- .../q7/simplified.txt | 4 +- .../q70/explain.txt | 16 +- .../q70/simplified.txt | 8 +- .../q71/explain.txt | 12 +- .../q71/simplified.txt | 6 +- .../q72/explain.txt | 8 +- .../q72/simplified.txt | 4 +- .../q73/explain.txt | 8 +- .../q73/simplified.txt | 4 +- .../q74/explain.txt | 24 +- .../q74/simplified.txt | 12 +- .../q75/explain.txt | 16 +- .../q75/simplified.txt | 8 +- .../q76/explain.txt | 4 +- .../q76/simplified.txt | 2 +- .../q77/explain.txt | 28 +-- .../q77/simplified.txt | 14 +- .../q78/explain.txt | 16 +- .../q78/simplified.txt | 8 +- .../q79/explain.txt | 12 +- .../q79/simplified.txt | 6 +- .../q8/explain.txt | 8 +- .../q8/simplified.txt | 4 +- .../q80/explain.txt | 16 +- .../q80/simplified.txt | 8 +- .../q81/explain.txt | 20 +- .../q81/simplified.txt | 10 +- .../q82/explain.txt | 8 +- .../q82/simplified.txt | 4 +- .../q83.ansi/explain.txt | 16 +- .../q83.ansi/simplified.txt | 8 +- .../q84/explain.txt | 4 +- .../q84/simplified.txt | 2 +- .../q85/explain.txt | 8 +- .../q85/simplified.txt | 4 +- .../q86/explain.txt | 8 +- .../q86/simplified.txt | 4 +- .../q87/explain.txt | 16 +- .../q87/simplified.txt | 8 +- .../q88/explain.txt | 32 +-- .../q88/simplified.txt | 16 +- .../q89/explain.txt | 8 +- .../q89/simplified.txt | 4 +- .../q9/explain.txt | 24 +- .../q9/simplified.txt | 12 +- .../q90/explain.txt | 8 +- .../q90/simplified.txt | 4 +- .../q91/explain.txt | 12 +- .../q91/simplified.txt | 6 +- .../q92/explain.txt | 12 +- .../q92/simplified.txt | 6 +- .../q93/explain.txt | 4 +- .../q93/simplified.txt | 2 +- .../q94/explain.txt | 4 +- .../q94/simplified.txt | 2 +- .../q95/explain.txt | 4 +- .../q95/simplified.txt | 2 +- .../q96/explain.txt | 4 +- .../q96/simplified.txt | 2 +- .../q97/explain.txt | 8 +- .../q97/simplified.txt | 4 +- .../q98/explain.txt | 12 +- .../q98/simplified.txt | 6 +- .../q99/explain.txt | 4 +- .../q99/simplified.txt | 2 +- .../approved-plans-v1_4/q1/explain.txt | 8 +- .../approved-plans-v1_4/q1/simplified.txt | 4 +- .../approved-plans-v1_4/q10/explain.txt | 24 +- .../approved-plans-v1_4/q10/simplified.txt | 12 +- .../approved-plans-v1_4/q11/explain.txt | 12 +- .../approved-plans-v1_4/q11/simplified.txt | 6 +- .../approved-plans-v1_4/q12/explain.txt | 8 +- .../approved-plans-v1_4/q12/simplified.txt | 4 +- .../approved-plans-v1_4/q13/explain.txt | 8 +- .../approved-plans-v1_4/q13/simplified.txt | 4 +- .../approved-plans-v1_4/q14a/explain.txt | 16 +- .../approved-plans-v1_4/q14a/simplified.txt | 8 +- .../approved-plans-v1_4/q14b/explain.txt | 28 +-- .../approved-plans-v1_4/q14b/simplified.txt | 14 +- .../approved-plans-v1_4/q15/explain.txt | 8 +- .../approved-plans-v1_4/q15/simplified.txt | 4 +- .../approved-plans-v1_4/q16/explain.txt | 4 +- .../approved-plans-v1_4/q16/simplified.txt | 2 +- .../approved-plans-v1_4/q17/explain.txt | 12 +- .../approved-plans-v1_4/q17/simplified.txt | 6 +- .../approved-plans-v1_4/q18/explain.txt | 8 +- .../approved-plans-v1_4/q18/simplified.txt | 4 +- .../approved-plans-v1_4/q19/explain.txt | 4 +- .../approved-plans-v1_4/q19/simplified.txt | 2 +- .../approved-plans-v1_4/q2/explain.txt | 4 +- .../approved-plans-v1_4/q2/simplified.txt | 2 +- .../approved-plans-v1_4/q20/explain.txt | 8 +- .../approved-plans-v1_4/q20/simplified.txt | 4 +- .../approved-plans-v1_4/q21/explain.txt | 8 +- .../approved-plans-v1_4/q21/simplified.txt | 4 +- .../approved-plans-v1_4/q22/explain.txt | 8 +- .../approved-plans-v1_4/q22/simplified.txt | 4 +- .../approved-plans-v1_4/q23a/explain.txt | 20 +- .../approved-plans-v1_4/q23a/simplified.txt | 10 +- .../approved-plans-v1_4/q23b/explain.txt | 20 +- .../approved-plans-v1_4/q23b/simplified.txt | 10 +- .../approved-plans-v1_4/q24a/explain.txt | 12 +- .../approved-plans-v1_4/q24a/simplified.txt | 6 +- .../approved-plans-v1_4/q24b/explain.txt | 12 +- .../approved-plans-v1_4/q24b/simplified.txt | 6 +- .../approved-plans-v1_4/q25/explain.txt | 12 +- .../approved-plans-v1_4/q25/simplified.txt | 6 +- .../approved-plans-v1_4/q26/explain.txt | 8 +- .../approved-plans-v1_4/q26/simplified.txt | 4 +- .../approved-plans-v1_4/q27/explain.txt | 8 +- .../approved-plans-v1_4/q27/simplified.txt | 4 +- .../approved-plans-v1_4/q28/explain.txt | 24 +- .../approved-plans-v1_4/q28/simplified.txt | 12 +- .../approved-plans-v1_4/q29/explain.txt | 16 +- .../approved-plans-v1_4/q29/simplified.txt | 8 +- .../approved-plans-v1_4/q3/explain.txt | 4 +- .../approved-plans-v1_4/q3/simplified.txt | 2 +- .../approved-plans-v1_4/q30/explain.txt | 8 +- .../approved-plans-v1_4/q30/simplified.txt | 4 +- .../approved-plans-v1_4/q31/explain.txt | 16 +- .../approved-plans-v1_4/q31/simplified.txt | 8 +- .../approved-plans-v1_4/q32/explain.txt | 8 +- .../approved-plans-v1_4/q32/simplified.txt | 4 +- .../approved-plans-v1_4/q33/explain.txt | 8 +- .../approved-plans-v1_4/q33/simplified.txt | 4 +- .../approved-plans-v1_4/q34/explain.txt | 8 +- .../approved-plans-v1_4/q34/simplified.txt | 4 +- .../approved-plans-v1_4/q35/explain.txt | 24 +- .../approved-plans-v1_4/q35/simplified.txt | 12 +- .../approved-plans-v1_4/q36/explain.txt | 8 +- .../approved-plans-v1_4/q36/simplified.txt | 4 +- .../approved-plans-v1_4/q37/explain.txt | 8 +- .../approved-plans-v1_4/q37/simplified.txt | 4 +- .../approved-plans-v1_4/q38/explain.txt | 8 +- .../approved-plans-v1_4/q38/simplified.txt | 4 +- .../approved-plans-v1_4/q39a/explain.txt | 12 +- .../approved-plans-v1_4/q39a/simplified.txt | 6 +- .../approved-plans-v1_4/q39b/explain.txt | 12 +- .../approved-plans-v1_4/q39b/simplified.txt | 6 +- .../approved-plans-v1_4/q4/explain.txt | 12 +- .../approved-plans-v1_4/q4/simplified.txt | 6 +- .../approved-plans-v1_4/q40/explain.txt | 8 +- .../approved-plans-v1_4/q40/simplified.txt | 4 +- .../approved-plans-v1_4/q41/explain.txt | 4 +- .../approved-plans-v1_4/q41/simplified.txt | 2 +- .../approved-plans-v1_4/q42/explain.txt | 4 +- .../approved-plans-v1_4/q42/simplified.txt | 2 +- .../approved-plans-v1_4/q43/explain.txt | 4 +- .../approved-plans-v1_4/q43/simplified.txt | 2 +- .../approved-plans-v1_4/q44/explain.txt | 16 +- .../approved-plans-v1_4/q44/simplified.txt | 8 +- .../approved-plans-v1_4/q45/explain.txt | 12 +- .../approved-plans-v1_4/q45/simplified.txt | 6 +- .../approved-plans-v1_4/q46/explain.txt | 8 +- .../approved-plans-v1_4/q46/simplified.txt | 4 +- .../approved-plans-v1_4/q47/explain.txt | 16 +- .../approved-plans-v1_4/q47/simplified.txt | 8 +- .../approved-plans-v1_4/q48/explain.txt | 8 +- .../approved-plans-v1_4/q48/simplified.txt | 4 +- .../approved-plans-v1_4/q49/explain.txt | 16 +- .../approved-plans-v1_4/q49/simplified.txt | 8 +- .../approved-plans-v1_4/q5/explain.txt | 16 +- .../approved-plans-v1_4/q5/simplified.txt | 8 +- .../approved-plans-v1_4/q50/explain.txt | 8 +- .../approved-plans-v1_4/q50/simplified.txt | 4 +- .../approved-plans-v1_4/q51/explain.txt | 12 +- .../approved-plans-v1_4/q51/simplified.txt | 6 +- .../approved-plans-v1_4/q52/explain.txt | 4 +- .../approved-plans-v1_4/q52/simplified.txt | 2 +- .../approved-plans-v1_4/q53/explain.txt | 8 +- .../approved-plans-v1_4/q53/simplified.txt | 4 +- .../approved-plans-v1_4/q54/explain.txt | 20 +- .../approved-plans-v1_4/q54/simplified.txt | 10 +- .../approved-plans-v1_4/q55/explain.txt | 4 +- .../approved-plans-v1_4/q55/simplified.txt | 2 +- .../approved-plans-v1_4/q56/explain.txt | 8 +- .../approved-plans-v1_4/q56/simplified.txt | 4 +- .../approved-plans-v1_4/q57/explain.txt | 16 +- .../approved-plans-v1_4/q57/simplified.txt | 8 +- .../approved-plans-v1_4/q58/explain.txt | 12 +- .../approved-plans-v1_4/q58/simplified.txt | 6 +- .../approved-plans-v1_4/q59/explain.txt | 4 +- .../approved-plans-v1_4/q59/simplified.txt | 2 +- .../approved-plans-v1_4/q6/explain.txt | 12 +- .../approved-plans-v1_4/q6/simplified.txt | 6 +- .../approved-plans-v1_4/q60/explain.txt | 8 +- .../approved-plans-v1_4/q60/simplified.txt | 4 +- .../approved-plans-v1_4/q61/explain.txt | 12 +- .../approved-plans-v1_4/q61/simplified.txt | 6 +- .../approved-plans-v1_4/q62/explain.txt | 4 +- .../approved-plans-v1_4/q62/simplified.txt | 2 +- .../approved-plans-v1_4/q63/explain.txt | 8 +- .../approved-plans-v1_4/q63/simplified.txt | 4 +- .../approved-plans-v1_4/q64/explain.txt | 12 +- .../approved-plans-v1_4/q64/simplified.txt | 6 +- .../approved-plans-v1_4/q65/explain.txt | 8 +- .../approved-plans-v1_4/q65/simplified.txt | 4 +- .../approved-plans-v1_4/q66/explain.txt | 8 +- .../approved-plans-v1_4/q66/simplified.txt | 4 +- .../approved-plans-v1_4/q67/explain.txt | 8 +- .../approved-plans-v1_4/q67/simplified.txt | 4 +- .../approved-plans-v1_4/q68/explain.txt | 8 +- .../approved-plans-v1_4/q68/simplified.txt | 4 +- .../approved-plans-v1_4/q69/explain.txt | 24 +- .../approved-plans-v1_4/q69/simplified.txt | 12 +- .../approved-plans-v1_4/q7/explain.txt | 8 +- .../approved-plans-v1_4/q7/simplified.txt | 4 +- .../approved-plans-v1_4/q70/explain.txt | 16 +- .../approved-plans-v1_4/q70/simplified.txt | 8 +- .../approved-plans-v1_4/q71/explain.txt | 8 +- .../approved-plans-v1_4/q71/simplified.txt | 4 +- .../approved-plans-v1_4/q72/explain.txt | 8 +- .../approved-plans-v1_4/q72/simplified.txt | 4 +- .../approved-plans-v1_4/q73/explain.txt | 8 +- .../approved-plans-v1_4/q73/simplified.txt | 4 +- .../approved-plans-v1_4/q74/explain.txt | 12 +- .../approved-plans-v1_4/q74/simplified.txt | 6 +- .../approved-plans-v1_4/q75/explain.txt | 12 +- .../approved-plans-v1_4/q75/simplified.txt | 6 +- .../approved-plans-v1_4/q76/explain.txt | 4 +- .../approved-plans-v1_4/q76/simplified.txt | 2 +- .../approved-plans-v1_4/q77/explain.txt | 20 +- .../approved-plans-v1_4/q77/simplified.txt | 10 +- .../approved-plans-v1_4/q78/explain.txt | 8 +- .../approved-plans-v1_4/q78/simplified.txt | 4 +- .../approved-plans-v1_4/q79/explain.txt | 8 +- .../approved-plans-v1_4/q79/simplified.txt | 4 +- .../approved-plans-v1_4/q8/explain.txt | 8 +- .../approved-plans-v1_4/q8/simplified.txt | 4 +- .../approved-plans-v1_4/q80/explain.txt | 16 +- .../approved-plans-v1_4/q80/simplified.txt | 8 +- .../approved-plans-v1_4/q81/explain.txt | 8 +- .../approved-plans-v1_4/q81/simplified.txt | 4 +- .../approved-plans-v1_4/q82/explain.txt | 8 +- .../approved-plans-v1_4/q82/simplified.txt | 4 +- .../approved-plans-v1_4/q83/explain.txt | 8 +- .../approved-plans-v1_4/q83/simplified.txt | 4 +- .../approved-plans-v1_4/q84/explain.txt | 4 +- .../approved-plans-v1_4/q84/simplified.txt | 2 +- .../approved-plans-v1_4/q85/explain.txt | 8 +- .../approved-plans-v1_4/q85/simplified.txt | 4 +- .../approved-plans-v1_4/q86/explain.txt | 8 +- .../approved-plans-v1_4/q86/simplified.txt | 4 +- .../approved-plans-v1_4/q87/explain.txt | 16 +- .../approved-plans-v1_4/q87/simplified.txt | 8 +- .../approved-plans-v1_4/q88/explain.txt | 32 +-- .../approved-plans-v1_4/q88/simplified.txt | 16 +- .../approved-plans-v1_4/q89/explain.txt | 8 +- .../approved-plans-v1_4/q89/simplified.txt | 4 +- .../approved-plans-v1_4/q9/explain.txt | 24 +- .../approved-plans-v1_4/q9/simplified.txt | 12 +- .../approved-plans-v1_4/q90/explain.txt | 8 +- .../approved-plans-v1_4/q90/simplified.txt | 4 +- .../approved-plans-v1_4/q91/explain.txt | 8 +- .../approved-plans-v1_4/q91/simplified.txt | 4 +- .../approved-plans-v1_4/q92/explain.txt | 8 +- .../approved-plans-v1_4/q92/simplified.txt | 4 +- .../approved-plans-v1_4/q93/explain.txt | 4 +- .../approved-plans-v1_4/q93/simplified.txt | 2 +- .../approved-plans-v1_4/q94/explain.txt | 4 +- .../approved-plans-v1_4/q94/simplified.txt | 2 +- .../approved-plans-v1_4/q95/explain.txt | 4 +- .../approved-plans-v1_4/q95/simplified.txt | 2 +- .../approved-plans-v1_4/q96/explain.txt | 4 +- .../approved-plans-v1_4/q96/simplified.txt | 2 +- .../approved-plans-v1_4/q97/explain.txt | 8 +- .../approved-plans-v1_4/q97/simplified.txt | 4 +- .../approved-plans-v1_4/q98/explain.txt | 12 +- .../approved-plans-v1_4/q98/simplified.txt | 6 +- .../approved-plans-v1_4/q99/explain.txt | 4 +- .../approved-plans-v1_4/q99/simplified.txt | 2 +- .../q10a/explain.txt | 8 +- .../q10a/simplified.txt | 4 +- .../q11/explain.txt | 12 +- .../q11/simplified.txt | 6 +- .../q12/explain.txt | 8 +- .../q12/simplified.txt | 4 +- .../q14/explain.txt | 28 +-- .../q14/simplified.txt | 14 +- .../q14a/explain.txt | 20 +- .../q14a/simplified.txt | 10 +- .../q18a/explain.txt | 8 +- .../q18a/simplified.txt | 4 +- .../q20/explain.txt | 8 +- .../q20/simplified.txt | 4 +- .../q22/explain.txt | 12 +- .../q22/simplified.txt | 6 +- .../q22a/explain.txt | 8 +- .../q22a/simplified.txt | 4 +- .../q24/explain.txt | 16 +- .../q24/simplified.txt | 8 +- .../q27a/explain.txt | 8 +- .../q27a/simplified.txt | 4 +- .../q34/explain.txt | 8 +- .../q34/simplified.txt | 4 +- .../q35/explain.txt | 24 +- .../q35/simplified.txt | 12 +- .../q35a/explain.txt | 8 +- .../q35a/simplified.txt | 4 +- .../q36a/explain.txt | 8 +- .../q36a/simplified.txt | 4 +- .../q47/explain.txt | 16 +- .../q47/simplified.txt | 8 +- .../q49/explain.txt | 16 +- .../q49/simplified.txt | 8 +- .../q51a/explain.txt | 20 +- .../q51a/simplified.txt | 10 +- .../q57/explain.txt | 16 +- .../q57/simplified.txt | 8 +- .../q5a/explain.txt | 16 +- .../q5a/simplified.txt | 8 +- .../q6/explain.txt | 12 +- .../q6/simplified.txt | 6 +- .../q64/explain.txt | 12 +- .../q64/simplified.txt | 6 +- .../q67a/explain.txt | 8 +- .../q67a/simplified.txt | 4 +- .../q70a/explain.txt | 16 +- .../q70a/simplified.txt | 8 +- .../q72/explain.txt | 8 +- .../q72/simplified.txt | 4 +- .../q74/explain.txt | 12 +- .../q74/simplified.txt | 6 +- .../q75/explain.txt | 12 +- .../q75/simplified.txt | 6 +- .../q77a/explain.txt | 20 +- .../q77a/simplified.txt | 10 +- .../q78/explain.txt | 8 +- .../q78/simplified.txt | 4 +- .../q80a/explain.txt | 16 +- .../q80a/simplified.txt | 8 +- .../q86a/explain.txt | 8 +- .../q86a/simplified.txt | 4 +- .../q98/explain.txt | 12 +- .../q98/simplified.txt | 6 +- .../q10a/explain.txt | 8 +- .../q10a/simplified.txt | 4 +- .../q11/explain.txt | 24 +- .../q11/simplified.txt | 12 +- .../q12/explain.txt | 8 +- .../q12/simplified.txt | 4 +- .../q14/explain.txt | 32 +-- .../q14/simplified.txt | 16 +- .../q14a/explain.txt | 28 +-- .../q14a/simplified.txt | 14 +- .../q18a/explain.txt | 24 +- .../q18a/simplified.txt | 12 +- .../q20/explain.txt | 8 +- .../q20/simplified.txt | 4 +- .../q22/explain.txt | 12 +- .../q22/simplified.txt | 6 +- .../q22a/explain.txt | 8 +- .../q22a/simplified.txt | 4 +- .../q24/explain.txt | 16 +- .../q24/simplified.txt | 8 +- .../q27a/explain.txt | 16 +- .../q27a/simplified.txt | 8 +- .../q34/explain.txt | 8 +- .../q34/simplified.txt | 4 +- .../q35/explain.txt | 24 +- .../q35/simplified.txt | 12 +- .../q35a/explain.txt | 8 +- .../q35a/simplified.txt | 4 +- .../q36a/explain.txt | 8 +- .../q36a/simplified.txt | 4 +- .../q47/explain.txt | 8 +- .../q47/simplified.txt | 4 +- .../q49/explain.txt | 16 +- .../q49/simplified.txt | 8 +- .../q51a/explain.txt | 12 +- .../q51a/simplified.txt | 6 +- .../q57/explain.txt | 8 +- .../q57/simplified.txt | 4 +- .../q5a/explain.txt | 16 +- .../q5a/simplified.txt | 8 +- .../q6/explain.txt | 20 +- .../q6/simplified.txt | 10 +- .../q64/explain.txt | 60 ++--- .../q64/simplified.txt | 30 +-- .../q67a/explain.txt | 8 +- .../q67a/simplified.txt | 4 +- .../q70a/explain.txt | 16 +- .../q70a/simplified.txt | 8 +- .../q72/explain.txt | 8 +- .../q72/simplified.txt | 4 +- .../q74/explain.txt | 24 +- .../q74/simplified.txt | 12 +- .../q75/explain.txt | 16 +- .../q75/simplified.txt | 8 +- .../q77a/explain.txt | 28 +-- .../q77a/simplified.txt | 14 +- .../q78/explain.txt | 16 +- .../q78/simplified.txt | 8 +- .../q80a/explain.txt | 16 +- .../q80a/simplified.txt | 8 +- .../q86a/explain.txt | 8 +- .../q86a/simplified.txt | 4 +- .../q98/explain.txt | 12 +- .../q98/simplified.txt | 6 +- .../approved-plans-v2_7/q10a/explain.txt | 8 +- .../approved-plans-v2_7/q10a/simplified.txt | 4 +- .../approved-plans-v2_7/q11/explain.txt | 12 +- .../approved-plans-v2_7/q11/simplified.txt | 6 +- .../approved-plans-v2_7/q12/explain.txt | 8 +- .../approved-plans-v2_7/q12/simplified.txt | 4 +- .../approved-plans-v2_7/q14/explain.txt | 28 +-- .../approved-plans-v2_7/q14/simplified.txt | 14 +- .../approved-plans-v2_7/q14a/explain.txt | 20 +- .../approved-plans-v2_7/q14a/simplified.txt | 10 +- .../approved-plans-v2_7/q18a/explain.txt | 8 +- .../approved-plans-v2_7/q18a/simplified.txt | 4 +- .../approved-plans-v2_7/q20/explain.txt | 8 +- .../approved-plans-v2_7/q20/simplified.txt | 4 +- .../approved-plans-v2_7/q22/explain.txt | 12 +- .../approved-plans-v2_7/q22/simplified.txt | 6 +- .../approved-plans-v2_7/q22a/explain.txt | 8 +- .../approved-plans-v2_7/q22a/simplified.txt | 4 +- .../approved-plans-v2_7/q24/explain.txt | 16 +- .../approved-plans-v2_7/q24/simplified.txt | 8 +- .../approved-plans-v2_7/q27a/explain.txt | 8 +- .../approved-plans-v2_7/q27a/simplified.txt | 4 +- .../approved-plans-v2_7/q34/explain.txt | 8 +- .../approved-plans-v2_7/q34/simplified.txt | 4 +- .../approved-plans-v2_7/q35/explain.txt | 24 +- .../approved-plans-v2_7/q35/simplified.txt | 12 +- .../approved-plans-v2_7/q35a/explain.txt | 8 +- .../approved-plans-v2_7/q35a/simplified.txt | 4 +- .../approved-plans-v2_7/q36a/explain.txt | 8 +- .../approved-plans-v2_7/q36a/simplified.txt | 4 +- .../approved-plans-v2_7/q47/explain.txt | 16 +- .../approved-plans-v2_7/q47/simplified.txt | 8 +- .../approved-plans-v2_7/q49/explain.txt | 16 +- .../approved-plans-v2_7/q49/simplified.txt | 8 +- .../approved-plans-v2_7/q51a/explain.txt | 20 +- .../approved-plans-v2_7/q51a/simplified.txt | 10 +- .../approved-plans-v2_7/q57/explain.txt | 16 +- .../approved-plans-v2_7/q57/simplified.txt | 8 +- .../approved-plans-v2_7/q5a/explain.txt | 16 +- .../approved-plans-v2_7/q5a/simplified.txt | 8 +- .../approved-plans-v2_7/q6/explain.txt | 12 +- .../approved-plans-v2_7/q6/simplified.txt | 6 +- .../approved-plans-v2_7/q64/explain.txt | 12 +- .../approved-plans-v2_7/q64/simplified.txt | 6 +- .../approved-plans-v2_7/q67a/explain.txt | 8 +- .../approved-plans-v2_7/q67a/simplified.txt | 4 +- .../approved-plans-v2_7/q70a/explain.txt | 16 +- .../approved-plans-v2_7/q70a/simplified.txt | 8 +- .../approved-plans-v2_7/q72/explain.txt | 8 +- .../approved-plans-v2_7/q72/simplified.txt | 4 +- .../approved-plans-v2_7/q74/explain.txt | 12 +- .../approved-plans-v2_7/q74/simplified.txt | 6 +- .../approved-plans-v2_7/q75/explain.txt | 12 +- .../approved-plans-v2_7/q75/simplified.txt | 6 +- .../approved-plans-v2_7/q77a/explain.txt | 20 +- .../approved-plans-v2_7/q77a/simplified.txt | 10 +- .../approved-plans-v2_7/q78/explain.txt | 8 +- .../approved-plans-v2_7/q78/simplified.txt | 4 +- .../approved-plans-v2_7/q80a/explain.txt | 16 +- .../approved-plans-v2_7/q80a/simplified.txt | 8 +- .../approved-plans-v2_7/q86a/explain.txt | 8 +- .../approved-plans-v2_7/q86a/simplified.txt | 4 +- .../approved-plans-v2_7/q98/explain.txt | 12 +- .../approved-plans-v2_7/q98/simplified.txt | 6 +- .../apache/comet/CometExpressionSuite.scala | 6 +- .../apache/comet/exec/CometExecSuite.scala | 4 +- .../org/apache/spark/sql/CometTestBase.scala | 3 +- 830 files changed, 4101 insertions(+), 3761 deletions(-) create mode 100644 spark/src/main/scala/org/apache/spark/sql/comet/CometColumnarToRowExec.scala 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 1cc42f62ed..1927868a1c 100644 --- a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java @@ -172,28 +172,6 @@ public void close() { /** Returns a decoded {@link CometDecodedVector Comet vector}. */ public CometDecodedVector loadVector() { - // Only re-use Comet vector iff: - // 1. if we're not using dictionary encoding, since with dictionary encoding, the native - // side may fallback to plain encoding and the underlying memory address for the vector - // will change as result. - // 2. if the column type is of fixed width, in other words, string/binary are not supported - // since the native side may resize the vector and therefore change memory address. - // 3. if the last loaded vector contains null values: if values of last vector are all not - // null, Arrow C data API will skip loading the native validity buffer, therefore we - // should not re-use the vector in that case. - // 4. if the last loaded vector doesn't contain any null value, but the current vector also - // are all not null, which means we can also re-use the loaded vector. - // 5. if the new number of value is the same or smaller - if ((hadNull || currentNumNulls == 0) - && currentVector != null - && dictionary == null - && currentVector.isFixedLength() - && currentVector.numValues() >= currentNumValues) { - currentVector.setNumNulls(currentNumNulls); - currentVector.setNumValues(currentNumValues); - return currentVector; - } - LOG.debug("Reloading vector"); // Close the previous vector first to release struct memory allocated to import Arrow array & diff --git a/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java b/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java index 8de2376f94..5fd348eebb 100644 --- a/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java @@ -53,13 +53,13 @@ public ConstantColumnReader( public ConstantColumnReader( DataType type, ColumnDescriptor descriptor, Object value, boolean useDecimal128) { - super(type, descriptor, useDecimal128); + super(type, descriptor, useDecimal128, true); this.value = value; } ConstantColumnReader( DataType type, ColumnDescriptor descriptor, int batchSize, boolean useDecimal128) { - super(type, descriptor, useDecimal128); + super(type, descriptor, useDecimal128, true); this.batchSize = batchSize; initNative(); } 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 13b90e256e..2820c42f89 100644 --- a/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java @@ -40,9 +40,14 @@ public class MetadataColumnReader extends AbstractColumnReader { private ArrowArray array = null; private ArrowSchema schema = null; - public MetadataColumnReader(DataType type, ColumnDescriptor descriptor, boolean useDecimal128) { + private boolean isConstant; + + public MetadataColumnReader( + DataType type, ColumnDescriptor descriptor, boolean useDecimal128, boolean isConstant) { // TODO: should we handle legacy dates & timestamps for metadata columns? super(type, descriptor, useDecimal128, false); + + this.isConstant = isConstant; } @Override @@ -62,7 +67,7 @@ public void readBatch(int total) { Native.currentBatch(nativeHandle, arrayAddr, schemaAddr); FieldVector fieldVector = Data.importVector(allocator, array, schema, null); - vector = new CometPlainVector(fieldVector, useDecimal128); + vector = new CometPlainVector(fieldVector, useDecimal128, false, isConstant); } vector.setNumValues(total); diff --git a/common/src/main/java/org/apache/comet/parquet/RowIndexColumnReader.java b/common/src/main/java/org/apache/comet/parquet/RowIndexColumnReader.java index 8448318dbd..46e6ee67f5 100644 --- a/common/src/main/java/org/apache/comet/parquet/RowIndexColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/RowIndexColumnReader.java @@ -33,7 +33,7 @@ public class RowIndexColumnReader extends MetadataColumnReader { private long offset; public RowIndexColumnReader(StructField field, int batchSize, long[] indices) { - super(field.dataType(), TypeUtil.convertToParquet(field), false); + super(field.dataType(), TypeUtil.convertToParquet(field), false, false); this.indices = indices; setBatchSize(batchSize); } diff --git a/common/src/main/java/org/apache/comet/vector/CometPlainVector.java b/common/src/main/java/org/apache/comet/vector/CometPlainVector.java index 65cc876bda..f3803d53a9 100644 --- a/common/src/main/java/org/apache/comet/vector/CometPlainVector.java +++ b/common/src/main/java/org/apache/comet/vector/CometPlainVector.java @@ -38,11 +38,18 @@ public class CometPlainVector extends CometDecodedVector { private byte booleanByteCache; private int booleanByteCacheIndex = -1; + private boolean isReused; + public CometPlainVector(ValueVector vector, boolean useDecimal128) { this(vector, useDecimal128, false); } public CometPlainVector(ValueVector vector, boolean useDecimal128, boolean isUuid) { + this(vector, useDecimal128, isUuid, false); + } + + public CometPlainVector( + ValueVector vector, boolean useDecimal128, boolean isUuid, boolean isReused) { super(vector, vector.getField(), useDecimal128, isUuid); // NullType doesn't have data buffer. if (vector instanceof NullVector) { @@ -52,6 +59,15 @@ public CometPlainVector(ValueVector vector, boolean useDecimal128, boolean isUui } isBaseFixedWidthVector = valueVector instanceof BaseFixedWidthVector; + this.isReused = isReused; + } + + public boolean isReused() { + return isReused; + } + + public void setReused(boolean isReused) { + this.isReused = isReused; } @Override diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 12f739848c..8be1cdcaa7 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -1,5 +1,5 @@ diff --git a/pom.xml b/pom.xml -index d3544881af1..bf0e2b53c70 100644 +index d3544881af1..26ab186c65d 100644 --- a/pom.xml +++ b/pom.xml @@ -148,6 +148,8 @@ @@ -38,7 +38,7 @@ index d3544881af1..bf0e2b53c70 100644 diff --git a/sql/core/pom.xml b/sql/core/pom.xml -index b386d135da1..854aec17c2d 100644 +index b386d135da1..46449e3f3f1 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -77,6 +77,10 @@ @@ -1284,6 +1284,27 @@ index 47679ed7865..9ffbaecb98e 100644 }.length == hashAggCount) assert(collectWithSubqueries(plan) { case s: SortAggregateExec => s }.length == sortAggCount) } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala +index b14f4a405f6..88815fd078f 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala +@@ -23,6 +23,7 @@ import org.apache.spark.sql.QueryTest + import org.apache.spark.sql.catalyst.InternalRow + import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} + import org.apache.spark.sql.catalyst.plans.logical.Deduplicate ++import org.apache.spark.sql.comet.CometColumnarToRowExec + import org.apache.spark.sql.execution.datasources.v2.BatchScanExec + import org.apache.spark.sql.internal.SQLConf + import org.apache.spark.sql.test.SharedSparkSession +@@ -131,7 +132,7 @@ class SparkPlanSuite extends QueryTest with SharedSparkSession { + spark.range(1).write.parquet(path.getAbsolutePath) + val df = spark.read.parquet(path.getAbsolutePath) + val columnarToRowExec = +- df.queryExecution.executedPlan.collectFirst { case p: ColumnarToRowExec => p }.get ++ df.queryExecution.executedPlan.collectFirst { case p: CometColumnarToRowExec => p }.get + try { + spark.range(1).foreach { _ => + columnarToRowExec.canonicalized diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index ac710c32296..baae214c6ee 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -2281,7 +2302,7 @@ index d083cac48ff..3c11bcde807 100644 import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -index 266bb343526..a426d8396be 100644 +index 266bb343526..c3e3d155813 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -24,10 +24,11 @@ import org.apache.spark.sql.catalyst.catalog.BucketSpec @@ -2331,7 +2352,7 @@ index 266bb343526..a426d8396be 100644 val bucketColumnType = bucketedDataFrame.schema.apply(bucketColumnIndex).dataType val rowsWithInvalidBuckets = fileScan.execute().filter(row => { -@@ -451,28 +461,44 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -451,28 +461,49 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val joinOperator = if (joined.sqlContext.conf.adaptiveExecutionEnabled) { val executedPlan = joined.queryExecution.executedPlan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan @@ -2357,6 +2378,11 @@ index 266bb343526..a426d8396be 100644 + case s: SortMergeJoinExec => s + case o => fail(s"expected SortMergeJoinExec, but found\n$o") + } ++ case CometColumnarToRowExec(child) => ++ child.asInstanceOf[CometSortMergeJoinExec].originalPlan match { ++ case s: SortMergeJoinExec => s ++ case o => fail(s"expected SortMergeJoinExec, but found\n$o") ++ } + case o => fail(s"expected SortMergeJoinExec, but found\n$o") + } } @@ -2384,7 +2410,7 @@ index 266bb343526..a426d8396be 100644 s"expected sort in the right child to be $sortRight but found\n${joinOperator.right}") // check the output partitioning -@@ -835,11 +861,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -835,11 +866,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") val scanDF = spark.table("bucketed_table").select("j") @@ -2398,7 +2424,7 @@ index 266bb343526..a426d8396be 100644 checkAnswer(aggDF, df1.groupBy("j").agg(max("k"))) } } -@@ -1026,15 +1052,23 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -1026,15 +1057,23 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti expectedNumShuffles: Int, expectedCoalescedNumBuckets: Option[Int]): Unit = { val plan = sql(query).queryExecution.executedPlan diff --git a/dev/diffs/3.5.1.diff b/dev/diffs/3.5.1.diff index 64d6681bcb..5e3433c682 100644 --- a/dev/diffs/3.5.1.diff +++ b/dev/diffs/3.5.1.diff @@ -1,5 +1,5 @@ diff --git a/pom.xml b/pom.xml -index 0f504dbee85..f6019da888a 100644 +index 0f504dbee85..430ec217e59 100644 --- a/pom.xml +++ b/pom.xml @@ -152,6 +152,8 @@ @@ -38,7 +38,7 @@ index 0f504dbee85..f6019da888a 100644 diff --git a/sql/core/pom.xml b/sql/core/pom.xml -index c46ab7b8fce..d8b99c2c115 100644 +index c46ab7b8fce..13357e8c7a6 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -77,6 +77,10 @@ @@ -1309,8 +1309,29 @@ index 47679ed7865..9ffbaecb98e 100644 }.length == hashAggCount) assert(collectWithSubqueries(plan) { case s: SortAggregateExec => s }.length == sortAggCount) } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala +index b14f4a405f6..88815fd078f 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala +@@ -23,6 +23,7 @@ import org.apache.spark.sql.QueryTest + import org.apache.spark.sql.catalyst.InternalRow + import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} + import org.apache.spark.sql.catalyst.plans.logical.Deduplicate ++import org.apache.spark.sql.comet.CometColumnarToRowExec + import org.apache.spark.sql.execution.datasources.v2.BatchScanExec + import org.apache.spark.sql.internal.SQLConf + import org.apache.spark.sql.test.SharedSparkSession +@@ -131,7 +132,7 @@ class SparkPlanSuite extends QueryTest with SharedSparkSession { + spark.range(1).write.parquet(path.getAbsolutePath) + val df = spark.read.parquet(path.getAbsolutePath) + val columnarToRowExec = +- df.queryExecution.executedPlan.collectFirst { case p: ColumnarToRowExec => p }.get ++ df.queryExecution.executedPlan.collectFirst { case p: CometColumnarToRowExec => p }.get + try { + spark.range(1).foreach { _ => + columnarToRowExec.canonicalized diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala -index 5a413c77754..c52f4b3818c 100644 +index 5a413c77754..a6f97dccb67 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution @@ -2270,7 +2291,7 @@ index d083cac48ff..3c11bcde807 100644 import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -index 746f289c393..1a2f1f7e3fd 100644 +index 746f289c393..0c99d028163 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -25,10 +25,11 @@ import org.apache.spark.sql.catalyst.expressions @@ -2320,7 +2341,7 @@ index 746f289c393..1a2f1f7e3fd 100644 val bucketColumnType = bucketedDataFrame.schema.apply(bucketColumnIndex).dataType val rowsWithInvalidBuckets = fileScan.execute().filter(row => { -@@ -452,28 +462,44 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -452,28 +462,49 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val joinOperator = if (joined.sqlContext.conf.adaptiveExecutionEnabled) { val executedPlan = joined.queryExecution.executedPlan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan @@ -2346,6 +2367,11 @@ index 746f289c393..1a2f1f7e3fd 100644 + case s: SortMergeJoinExec => s + case o => fail(s"expected SortMergeJoinExec, but found\n$o") + } ++ case CometColumnarToRowExec(child) => ++ child.asInstanceOf[CometSortMergeJoinExec].originalPlan match { ++ case s: SortMergeJoinExec => s ++ case o => fail(s"expected SortMergeJoinExec, but found\n$o") ++ } + case o => fail(s"expected SortMergeJoinExec, but found\n$o") + } } @@ -2373,7 +2399,7 @@ index 746f289c393..1a2f1f7e3fd 100644 s"expected sort in the right child to be $sortRight but found\n${joinOperator.right}") // check the output partitioning -@@ -836,11 +862,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -836,11 +867,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") val scanDF = spark.table("bucketed_table").select("j") @@ -2387,7 +2413,7 @@ index 746f289c393..1a2f1f7e3fd 100644 checkAnswer(aggDF, df1.groupBy("j").agg(max("k"))) } } -@@ -1029,15 +1055,21 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -1029,15 +1060,21 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti Seq(true, false).foreach { aqeEnabled => withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled.toString) { val plan = sql(query).queryExecution.executedPlan diff --git a/dev/diffs/4.0.0-preview1.diff b/dev/diffs/4.0.0-preview1.diff index db62ed607e..5f9bf9d801 100644 --- a/dev/diffs/4.0.0-preview1.diff +++ b/dev/diffs/4.0.0-preview1.diff @@ -1438,6 +1438,27 @@ index 47679ed7865..9ffbaecb98e 100644 }.length == hashAggCount) assert(collectWithSubqueries(plan) { case s: SortAggregateExec => s }.length == sortAggCount) } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala +index 966f4e74712..a715193d96d 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala +@@ -23,6 +23,7 @@ import org.apache.spark.sql.QueryTest + import org.apache.spark.sql.catalyst.InternalRow + import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} + import org.apache.spark.sql.catalyst.plans.logical.Deduplicate ++import org.apache.spark.sql.comet.CometColumnarToRowExec + import org.apache.spark.sql.execution.datasources.v2.BatchScanExec + import org.apache.spark.sql.internal.SQLConf + import org.apache.spark.sql.test.SharedSparkSession +@@ -134,7 +135,7 @@ class SparkPlanSuite extends QueryTest with SharedSparkSession { + spark.range(1).write.parquet(path.getAbsolutePath) + val df = spark.read.parquet(path.getAbsolutePath) + val columnarToRowExec = +- df.queryExecution.executedPlan.collectFirst { case p: ColumnarToRowExec => p }.get ++ df.queryExecution.executedPlan.collectFirst { case p: CometColumnarToRowExec => p }.get + try { + spark.range(1).foreach { _ => + columnarToRowExec.canonicalized diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index 3aaf61ffba4..4130ece2283 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -2562,7 +2583,7 @@ index 6ff07449c0c..9f95cff99e5 100644 import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -index 3573bafe482..a21767840a2 100644 +index 3573bafe482..11d387110ea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -25,10 +25,11 @@ import org.apache.spark.sql.catalyst.expressions @@ -2612,7 +2633,7 @@ index 3573bafe482..a21767840a2 100644 val bucketColumnType = bucketedDataFrame.schema.apply(bucketColumnIndex).dataType val rowsWithInvalidBuckets = fileScan.execute().filter(row => { -@@ -452,28 +462,44 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -452,28 +462,49 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val joinOperator = if (joined.sparkSession.sessionState.conf.adaptiveExecutionEnabled) { val executedPlan = joined.queryExecution.executedPlan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan @@ -2638,6 +2659,11 @@ index 3573bafe482..a21767840a2 100644 + case s: SortMergeJoinExec => s + case o => fail(s"expected SortMergeJoinExec, but found\n$o") + } ++ case CometColumnarToRowExec(child) => ++ child.asInstanceOf[CometSortMergeJoinExec].originalPlan match { ++ case s: SortMergeJoinExec => s ++ case o => fail(s"expected SortMergeJoinExec, but found\n$o") ++ } + case o => fail(s"expected SortMergeJoinExec, but found\n$o") + } } @@ -2665,7 +2691,7 @@ index 3573bafe482..a21767840a2 100644 s"expected sort in the right child to be $sortRight but found\n${joinOperator.right}") // check the output partitioning -@@ -836,11 +862,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -836,11 +867,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") val scanDF = spark.table("bucketed_table").select("j") @@ -2679,7 +2705,7 @@ index 3573bafe482..a21767840a2 100644 checkAnswer(aggDF, df1.groupBy("j").agg(max("k"))) } } -@@ -1029,15 +1055,21 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -1029,15 +1060,21 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti Seq(true, false).foreach { aqeEnabled => withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled.toString) { val plan = sql(query).queryExecution.executedPlan diff --git a/native/core/benches/parquet_read.rs b/native/core/benches/parquet_read.rs index ae511ade51..06f0a1ede1 100644 --- a/native/core/benches/parquet_read.rs +++ b/native/core/benches/parquet_read.rs @@ -212,6 +212,6 @@ impl Iterator for TestColumnReader { } self.total_num_values_read += total; - Some(self.inner.current_batch()) + Some(self.inner.current_batch().unwrap()) } } diff --git a/native/core/src/common/buffer.rs b/native/core/src/common/buffer.rs index f24038a955..291082d104 100644 --- a/native/core/src/common/buffer.rs +++ b/native/core/src/common/buffer.rs @@ -16,6 +16,7 @@ // under the License. use crate::common::bit; +use crate::execution::operators::ExecutionError; use arrow::buffer::Buffer as ArrowBuffer; use std::{ alloc::{handle_alloc_error, Layout}, @@ -43,6 +44,8 @@ pub struct CometBuffer { capacity: usize, /// Whether this buffer owns the data it points to. owned: bool, + /// The allocation instance for this buffer. + allocation: Arc, } unsafe impl Sync for CometBuffer {} @@ -63,6 +66,7 @@ impl CometBuffer { len: aligned_capacity, capacity: aligned_capacity, owned: true, + allocation: Arc::new(CometBufferAllocation::new()), } } } @@ -84,6 +88,7 @@ impl CometBuffer { len, capacity, owned: false, + allocation: Arc::new(CometBufferAllocation::new()), } } @@ -163,11 +168,28 @@ impl CometBuffer { /// because of the iterator-style pattern, the content of the original mutable buffer will only /// be updated once upstream operators fully consumed the previous output batch. For breaking /// operators, they are responsible for copying content out of the buffers. - pub unsafe fn to_arrow(&self) -> ArrowBuffer { + pub unsafe fn to_arrow(&self) -> Result { let ptr = NonNull::new_unchecked(self.data.as_ptr()); - // Uses a dummy `Arc::new(0)` as `Allocation` to ensure the memory region pointed by - // `ptr` won't be freed when the returned `ArrowBuffer` goes out of scope. - ArrowBuffer::from_custom_allocation(ptr, self.len, Arc::new(0)) + self.check_reference()?; + Ok(ArrowBuffer::from_custom_allocation( + ptr, + self.len, + Arc::::clone(&self.allocation), + )) + } + + /// Checks if this buffer is exclusively owned by Comet. If not, an error is returned. + /// We run this check when we want to update the buffer. If the buffer is also shared by + /// other components, e.g. one DataFusion operator stores the buffer, Comet cannot safely + /// modify the buffer. + pub fn check_reference(&self) -> Result<(), ExecutionError> { + if Arc::strong_count(&self.allocation) > 1 { + Err(ExecutionError::GeneralError( + "Error on modifying a buffer which is not exclusively owned by Comet".to_string(), + )) + } else { + Ok(()) + } } /// Resets this buffer by filling all bytes with zeros. @@ -242,13 +264,6 @@ impl PartialEq for CometBuffer { } } -impl From<&ArrowBuffer> for CometBuffer { - fn from(value: &ArrowBuffer) -> Self { - assert_eq!(value.len(), value.capacity()); - CometBuffer::from_ptr(value.as_ptr(), value.len(), value.capacity()) - } -} - impl std::ops::Deref for CometBuffer { type Target = [u8]; @@ -264,6 +279,15 @@ impl std::ops::DerefMut for CometBuffer { } } +#[derive(Debug)] +struct CometBufferAllocation {} + +impl CometBufferAllocation { + fn new() -> Self { + Self {} + } +} + #[cfg(test)] mod tests { use super::*; @@ -319,7 +343,7 @@ mod tests { assert_eq!(b"aaaa bbbb cccc dddd", &buf.as_slice()[0..str.len()]); unsafe { - let immutable_buf: ArrowBuffer = buf.to_arrow(); + let immutable_buf: ArrowBuffer = buf.to_arrow().unwrap(); assert_eq!(64, immutable_buf.len()); assert_eq!(str, &immutable_buf.as_slice()[0..str.len()]); } @@ -335,7 +359,7 @@ mod tests { assert_eq!(b"hello comet", &buf.as_slice()[0..11]); unsafe { - let arrow_buf2 = buf.to_arrow(); + let arrow_buf2 = buf.to_arrow().unwrap(); assert_eq!(arrow_buf, arrow_buf2); } } diff --git a/native/core/src/execution/operators/copy.rs b/native/core/src/execution/operators/copy.rs index cec00eb28c..80042635b3 100644 --- a/native/core/src/execution/operators/copy.rs +++ b/native/core/src/execution/operators/copy.rs @@ -260,7 +260,10 @@ fn copy_array(array: &dyn Array) -> ArrayRef { /// is a dictionary array, we will cast the dictionary array to primitive type /// (i.e., unpack the dictionary array) and copy the primitive array. If the input /// array is a primitive array, we simply copy the array. -fn copy_or_unpack_array(array: &Arc, mode: &CopyMode) -> Result { +pub(crate) fn copy_or_unpack_array( + array: &Arc, + mode: &CopyMode, +) -> Result { match array.data_type() { DataType::Dictionary(_, value_type) => { let options = CastOptions::default(); diff --git a/native/core/src/parquet/mod.rs b/native/core/src/parquet/mod.rs index d2a6f4804b..6bea31f441 100644 --- a/native/core/src/parquet/mod.rs +++ b/native/core/src/parquet/mod.rs @@ -541,7 +541,7 @@ pub extern "system" fn Java_org_apache_comet_parquet_Native_currentBatch( try_unwrap_or_throw(&e, |_env| { let ctx = get_context(handle)?; let reader = &mut ctx.column_reader; - let data = reader.current_batch(); + let data = reader.current_batch()?; data.move_to_spark(array_addr, schema_addr) .map_err(|e| e.into()) }) diff --git a/native/core/src/parquet/mutable_vector.rs b/native/core/src/parquet/mutable_vector.rs index d19ea32fad..a84966c13c 100644 --- a/native/core/src/parquet/mutable_vector.rs +++ b/native/core/src/parquet/mutable_vector.rs @@ -18,6 +18,7 @@ use arrow::{array::ArrayData, datatypes::DataType as ArrowDataType}; use crate::common::{bit, CometBuffer}; +use crate::execution::operators::ExecutionError; const DEFAULT_ARRAY_LEN: usize = 4; @@ -184,7 +185,7 @@ impl ParquetMutableVector { /// This method is highly unsafe since it calls `CometBuffer::to_arrow` which leaks raw /// pointer to the memory region that are tracked by `CometBuffer`. Please see comments on /// `to_arrow` buffer to understand the motivation. - pub fn get_array_data(&mut self) -> ArrayData { + pub fn get_array_data(&mut self) -> Result { unsafe { let data_type = if let Some(d) = &self.dictionary { ArrowDataType::Dictionary( @@ -196,20 +197,19 @@ impl ParquetMutableVector { }; let mut builder = ArrayData::builder(data_type) .len(self.num_values) - .add_buffer(self.value_buffer.to_arrow()) - .null_bit_buffer(Some(self.validity_buffer.to_arrow())) + .add_buffer(self.value_buffer.to_arrow()?) + .null_bit_buffer(Some(self.validity_buffer.to_arrow()?)) .null_count(self.num_nulls); if Self::is_binary_type(&self.arrow_type) && self.dictionary.is_none() { let child = &mut self.children[0]; - builder = builder.add_buffer(child.value_buffer.to_arrow()); + builder = builder.add_buffer(child.value_buffer.to_arrow()?); } if let Some(d) = &mut self.dictionary { - builder = builder.add_child_data(d.get_array_data()); + builder = builder.add_child_data(d.get_array_data()?); } - - builder.build_unchecked() + Ok(builder.build_unchecked()) } } diff --git a/native/core/src/parquet/read/column.rs b/native/core/src/parquet/read/column.rs index 05a0bf7b5b..dc7797a2b3 100644 --- a/native/core/src/parquet/read/column.rs +++ b/native/core/src/parquet/read/column.rs @@ -39,6 +39,7 @@ use super::{ }; use crate::common::{bit, bit::log2}; +use crate::execution::operators::ExecutionError; /// Maximum number of decimal digits an i32 can represent const DECIMAL_MAX_INT_DIGITS: i32 = 9; @@ -601,7 +602,7 @@ impl ColumnReader { } #[inline] - pub fn current_batch(&mut self) -> ArrayData { + pub fn current_batch(&mut self) -> Result { make_func_mut!(self, current_batch) } @@ -684,7 +685,7 @@ impl TypedColumnReader { /// Note: the caller must make sure the returned Arrow vector is fully consumed before calling /// `read_batch` again. #[inline] - pub fn current_batch(&mut self) -> ArrayData { + pub fn current_batch(&mut self) -> Result { self.vector.get_array_data() } diff --git a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala index 0b90a91c74..76da6c2d9f 100644 --- a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala +++ b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala @@ -84,6 +84,7 @@ class CometExecIterator( } private var nextBatch: Option[ColumnarBatch] = None + private var prevBatch: ColumnarBatch = null private var currentBatch: ColumnarBatch = null private var closed: Boolean = false @@ -129,6 +130,14 @@ class CometExecIterator( return true } + // Close previous batch if any. + // This is to guarantee safety at the native side before we overwrite the buffer memory + // shared across batches in the native side. + if (prevBatch != null) { + prevBatch.close() + prevBatch = null + } + nextBatch = getNextBatch() if (nextBatch.isEmpty) { @@ -151,6 +160,7 @@ class CometExecIterator( } currentBatch = nextBatch.get + prevBatch = currentBatch nextBatch = None currentBatch } diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index 52a0d5e180..4f5a57e9b4 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -1085,6 +1085,17 @@ class CometSparkSessionExtensions override def apply(plan: SparkPlan): SparkPlan = { val eliminatedPlan = plan transformUp { case ColumnarToRowExec(sparkToColumnar: CometSparkToColumnarExec) => sparkToColumnar.child + case c @ ColumnarToRowExec(child) if child.exists(_.isInstanceOf[CometPlan]) => + val op = CometColumnarToRowExec(child) + if (c.logicalLink.isEmpty) { + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) + } else { + c.logicalLink.foreach(op.setLogicalLink) + } + op + case CometColumnarToRowExec(sparkToColumnar: CometSparkToColumnarExec) => + sparkToColumnar.child case CometSparkToColumnarExec(child: CometSparkToColumnarExec) => child // Spark adds `RowToColumnar` under Comet columnar shuffle. But it's redundant as the // shuffle takes row-based input. @@ -1101,6 +1112,8 @@ class CometSparkSessionExtensions eliminatedPlan match { case ColumnarToRowExec(child: CometCollectLimitExec) => child + case CometColumnarToRowExec(child: CometCollectLimitExec) => + child case other => other } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometColumnarToRowExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometColumnarToRowExec.scala new file mode 100644 index 0000000000..18d95a473c --- /dev/null +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometColumnarToRowExec.scala @@ -0,0 +1,209 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.comet + +import scala.collection.JavaConverters._ + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.catalyst.expressions.codegen.Block._ +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.execution.{CodegenSupport, ColumnarToRowTransition, SparkPlan} +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.execution.vectorized.{ConstantColumnVector, WritableColumnVector} +import org.apache.spark.sql.types._ +import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} +import org.apache.spark.util.Utils + +import org.apache.comet.vector.CometPlainVector + +/** + * Copied from Spark `ColumnarToRowExec`. Comet needs the fix for SPARK-50235 but cannot wait for + * the fix to be released in Spark versions. We copy the implementation here to apply the fix. + */ +case class CometColumnarToRowExec(child: SparkPlan) + extends ColumnarToRowTransition + with CodegenSupport { + // supportsColumnar requires to be only called on driver side, see also SPARK-37779. + assert(Utils.isInRunningSparkTask || child.supportsColumnar) + + override def output: Seq[Attribute] = child.output + + override def outputPartitioning: Partitioning = child.outputPartitioning + + override def outputOrdering: Seq[SortOrder] = child.outputOrdering + + // `ColumnarToRowExec` processes the input RDD directly, which is kind of a leaf node in the + // codegen stage and needs to do the limit check. + protected override def canCheckLimitNotReached: Boolean = true + + override lazy val metrics: Map[String, SQLMetric] = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "numInputBatches" -> SQLMetrics.createMetric(sparkContext, "number of input batches")) + + override def doExecute(): RDD[InternalRow] = { + val numOutputRows = longMetric("numOutputRows") + val numInputBatches = longMetric("numInputBatches") + // This avoids calling `output` in the RDD closure, so that we don't need to include the entire + // plan (this) in the closure. + val localOutput = this.output + child.executeColumnar().mapPartitionsInternal { batches => + val toUnsafe = UnsafeProjection.create(localOutput, localOutput) + batches.flatMap { batch => + numInputBatches += 1 + numOutputRows += batch.numRows() + batch.rowIterator().asScala.map(toUnsafe) + } + } + } + + /** + * Generate [[ColumnVector]] expressions for our parent to consume as rows. This is called once + * per [[ColumnVector]] in the batch. + */ + private def genCodeColumnVector( + ctx: CodegenContext, + columnVar: String, + ordinal: String, + dataType: DataType, + nullable: Boolean): ExprCode = { + val javaType = CodeGenerator.javaType(dataType) + val value = CodeGenerator.getValueFromVector(columnVar, dataType, ordinal) + val isNullVar = if (nullable) { + JavaCode.isNullVariable(ctx.freshName("isNull")) + } else { + FalseLiteral + } + val valueVar = ctx.freshName("value") + val str = s"columnVector[$columnVar, $ordinal, ${dataType.simpleString}]" + val code = code"${ctx.registerComment(str)}" + (if (nullable) { + code""" + boolean $isNullVar = $columnVar.isNullAt($ordinal); + $javaType $valueVar = $isNullVar ? ${CodeGenerator.defaultValue(dataType)} : ($value); + """ + } else { + code"$javaType $valueVar = $value;" + }) + ExprCode(code, isNullVar, JavaCode.variable(valueVar, dataType)) + } + + /** + * Produce code to process the input iterator as [[ColumnarBatch]]es. This produces an + * [[org.apache.spark.sql.catalyst.expressions.UnsafeRow]] for each row in each batch. + */ + override protected def doProduce(ctx: CodegenContext): String = { + // PhysicalRDD always just has one input + val input = ctx.addMutableState("scala.collection.Iterator", "input", v => s"$v = inputs[0];") + + // metrics + val numOutputRows = metricTerm(ctx, "numOutputRows") + val numInputBatches = metricTerm(ctx, "numInputBatches") + + val columnarBatchClz = classOf[ColumnarBatch].getName + val batch = ctx.addMutableState(columnarBatchClz, "batch") + + val idx = ctx.addMutableState(CodeGenerator.JAVA_INT, "batchIdx") // init as batchIdx = 0 + val columnVectorClzs = + child.vectorTypes.getOrElse(Seq.fill(output.indices.size)(classOf[ColumnVector].getName)) + val (colVars, columnAssigns) = columnVectorClzs.zipWithIndex.map { + case (columnVectorClz, i) => + val name = ctx.addMutableState(columnVectorClz, s"colInstance$i") + (name, s"$name = ($columnVectorClz) $batch.column($i);") + }.unzip + + val nextBatch = ctx.freshName("nextBatch") + val nextBatchFuncName = ctx.addNewFunction( + nextBatch, + s""" + |private void $nextBatch() throws java.io.IOException { + | if ($input.hasNext()) { + | $batch = ($columnarBatchClz)$input.next(); + | $numInputBatches.add(1); + | $numOutputRows.add($batch.numRows()); + | $idx = 0; + | ${columnAssigns.mkString("", "\n", "\n")} + | } + |}""".stripMargin) + + ctx.currentVars = null + val rowidx = ctx.freshName("rowIdx") + val columnsBatchInput = (output zip colVars).map { case (attr, colVar) => + genCodeColumnVector(ctx, colVar, rowidx, attr.dataType, attr.nullable) + } + val localIdx = ctx.freshName("localIdx") + val localEnd = ctx.freshName("localEnd") + val numRows = ctx.freshName("numRows") + val shouldStop = if (parent.needStopCheck) { + s"if (shouldStop()) { $idx = $rowidx + 1; return; }" + } else { + "// shouldStop check is eliminated" + } + + val writableColumnVectorClz = classOf[WritableColumnVector].getName + val constantColumnVectorClz = classOf[ConstantColumnVector].getName + val cometPlainColumnVectorClz = classOf[CometPlainVector].getName + + // scalastyle:off line.size.limit + s""" + |if ($batch == null) { + | $nextBatchFuncName(); + |} + |while ($limitNotReachedCond $batch != null) { + | int $numRows = $batch.numRows(); + | int $localEnd = $numRows - $idx; + | for (int $localIdx = 0; $localIdx < $localEnd; $localIdx++) { + | int $rowidx = $idx + $localIdx; + | ${consume(ctx, columnsBatchInput).trim} + | $shouldStop + | } + | $idx = $numRows; + | + | // Comet fix for SPARK-50235 + | for (int i = 0; i < ${colVars.length}; i++) { + | if (!($batch.column(i) instanceof $writableColumnVectorClz || $batch.column(i) instanceof $constantColumnVectorClz || $batch.column(i) instanceof $cometPlainColumnVectorClz)) { + | $batch.column(i).close(); + | } else if ($batch.column(i) instanceof $cometPlainColumnVectorClz) { + | $cometPlainColumnVectorClz cometPlainColumnVector = ($cometPlainColumnVectorClz) $batch.column(i); + | if (!cometPlainColumnVector.isReused()) { + | cometPlainColumnVector.close(); + | } + | } + | } + | + | $batch = null; + | $nextBatchFuncName(); + |} + |// Comet fix for SPARK-50235: clean up resources + |if ($batch != null) { + | $batch.close(); + |} + """.stripMargin + // scalastyle:on line.size.limit + } + + override def inputRDDs(): Seq[RDD[InternalRow]] = { + Seq(child.executeColumnar().asInstanceOf[RDD[InternalRow]]) // Hack because of type erasure + } + + override protected def withNewChildInternal(newChild: SparkPlan): CometColumnarToRowExec = + copy(child = newChild) +} diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/explain.txt index ccca281513..0844332b3b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (40) +* CometColumnarToRow (40) +- CometTakeOrderedAndProject (39) +- CometProject (38) +- CometBroadcastHashJoin (37) @@ -224,14 +224,14 @@ Arguments: [c_customer_id#27], [c_customer_id#27] Input [1]: [c_customer_id#27] Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#27 ASC NULLS FIRST], output=[c_customer_id#27]), [c_customer_id#27], 100, [c_customer_id#27 ASC NULLS FIRST], [c_customer_id#27] -(40) ColumnarToRow [codegen id : 1] +(40) CometColumnarToRow [codegen id : 1] Input [1]: [c_customer_id#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (45) -+- * ColumnarToRow (44) ++- * CometColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(44) ColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/simplified.txt index 5aa8750a76..3ca476a1f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [c_customer_id] CometProject [c_customer_id] @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/explain.txt index e32e86f270..7544b94314 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/explain.txt @@ -11,7 +11,7 @@ TakeOrderedAndProject (45) : : +- * Filter (27) : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * ColumnarToRow (12) + : : : :- * CometColumnarToRow (12) : : : : +- CometBroadcastHashJoin (11) : : : : :- CometFilter (2) : : : : : +- CometScan parquet spark_catalog.default.customer (1) @@ -24,24 +24,24 @@ TakeOrderedAndProject (45) : : : : +- CometFilter (5) : : : : +- CometScan parquet spark_catalog.default.date_dim (4) : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) + : : : +- * CometColumnarToRow (17) : : : +- CometProject (16) : : : +- CometBroadcastHashJoin (15) : : : :- CometScan parquet spark_catalog.default.web_sales (13) : : : +- ReusedExchange (14) : : +- BroadcastExchange (25) - : : +- * ColumnarToRow (24) + : : +- * CometColumnarToRow (24) : : +- CometProject (23) : : +- CometBroadcastHashJoin (22) : : :- CometScan parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (33) - : +- * ColumnarToRow (32) + : +- * CometColumnarToRow (32) : +- CometProject (31) : +- CometFilter (30) : +- CometScan parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (39) - +- * ColumnarToRow (38) + +- * CometColumnarToRow (38) +- CometFilter (37) +- CometScan parquet spark_catalog.default.customer_demographics (36) @@ -101,7 +101,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) ColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 5] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan parquet spark_catalog.default.web_sales @@ -123,7 +123,7 @@ Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#12] (18) BroadcastExchange @@ -155,7 +155,7 @@ Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [1]: [cs_ship_customer_sk#16] (25) BroadcastExchange @@ -191,7 +191,7 @@ Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana Input [2]: [ca_address_sk#20, ca_county#21] Arguments: [ca_address_sk#20], [ca_address_sk#20] -(32) ColumnarToRow [codegen id : 3] +(32) CometColumnarToRow [codegen id : 3] Input [1]: [ca_address_sk#20] (33) BroadcastExchange @@ -219,7 +219,7 @@ ReadSchema: struct Input [2]: [d_date_sk#14, d_year#15] Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#14)) -(72) ColumnarToRow [codegen id : 1] +(72) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] (73) BroadcastExchange @@ -425,7 +425,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#31 BroadcastExchange (77) -+- * ColumnarToRow (76) ++- * CometColumnarToRow (76) +- CometFilter (75) +- CometScan parquet spark_catalog.default.date_dim (74) @@ -441,7 +441,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 2002)) AND isnotnull(d_date_sk#32)) -(76) ColumnarToRow [codegen id : 1] +(76) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] (77) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/simplified.txt index e509c8c3c1..8896bc14be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [customer_preferred_cust_flag] CometProject [customer_preferred_cust_flag] @@ -24,7 +24,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -47,7 +47,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/explain.txt index 18abad247b..7b8864d54f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (22) +- * Project (21) +- Window (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometSort (18) +- CometExchange (17) +- CometHashAggregate (16) @@ -109,7 +109,7 @@ Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometNativeShuff Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6], [i_class#9 ASC NULLS FIRST] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] (20) Window @@ -128,7 +128,7 @@ Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_ite Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (27) -+- * ColumnarToRow (26) ++- * CometColumnarToRow (26) +- CometProject (25) +- CometFilter (24) +- CometScan parquet spark_catalog.default.date_dim (23) @@ -149,7 +149,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (27) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/simplified.txt index e81a507139..c4c49fe339 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/simplified.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] CometExchange [i_class] #1 @@ -20,7 +20,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/explain.txt index e3c357b407..774d652f36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (33) +* CometColumnarToRow (33) +- CometHashAggregate (32) +- CometExchange (31) +- CometHashAggregate (30) @@ -188,14 +188,14 @@ Input [7]: [sum#23, count#24, sum#25, count#26, sum#27, count#28, sum#29] Keys: [] Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))] -(33) ColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [codegen id : 1] Input [4]: [avg(ss_quantity)#30, avg(ss_ext_sales_price)#31, avg(ss_ext_wholesale_cost)#32, sum(ss_ext_wholesale_cost)#33] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (38) -+- * ColumnarToRow (37) ++- * CometColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometScan parquet spark_catalog.default.date_dim (34) @@ -216,7 +216,7 @@ Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#16, d_year#17] Arguments: [d_date_sk#16], [d_date_sk#16] -(37) ColumnarToRow [codegen id : 1] +(37) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/simplified.txt index 46ed9b6365..c4d3b744d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),sum,count,sum,count,sum,count,sum,avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost))] CometExchange #1 @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/explain.txt index bd6aa9273c..7d4428d5b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (102) +* CometColumnarToRow (102) +- CometTakeOrderedAndProject (101) +- CometHashAggregate (100) +- CometExchange (99) @@ -569,13 +569,13 @@ Functions [2]: [sum(sales#49), sum(number_sales#50)] Input [6]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum(sales)#99, sum(number_sales)#100] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#91 ASC NULLS FIRST,i_brand_id#92 ASC NULLS FIRST,i_class_id#93 ASC NULLS FIRST,i_category_id#94 ASC NULLS FIRST], output=[channel#91,i_brand_id#92,i_class_id#93,i_category_id#94,sum(sales)#99,sum(number_sales)#100]), [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum(sales)#99, sum(number_sales)#100], 100, [channel#91 ASC NULLS FIRST, i_brand_id#92 ASC NULLS FIRST, i_class_id#93 ASC NULLS FIRST, i_category_id#94 ASC NULLS FIRST], [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum(sales)#99, sum(number_sales)#100] -(102) ColumnarToRow [codegen id : 1] +(102) CometColumnarToRow [codegen id : 1] Input [6]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum(sales)#99, sum(number_sales)#100] ===== Subqueries ===== Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#51, [id=#52] -* ColumnarToRow (119) +* CometColumnarToRow (119) +- CometHashAggregate (118) +- CometExchange (117) +- CometHashAggregate (116) @@ -670,7 +670,7 @@ Input [2]: [sum#122, count#123] Keys: [] Functions [1]: [avg((cast(quantity#106 as decimal(10,0)) * list_price#107))] -(119) ColumnarToRow [codegen id : 1] +(119) CometColumnarToRow [codegen id : 1] Input [1]: [average_sales#124] Subquery:2 Hosting operator id = 103 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#12 @@ -681,7 +681,7 @@ Subquery:4 Hosting operator id = 111 Hosting Expression = ws_sold_date_sk#117 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (124) -+- * ColumnarToRow (123) ++- * CometColumnarToRow (123) +- CometProject (122) +- CometFilter (121) +- CometScan parquet spark_catalog.default.date_dim (120) @@ -702,7 +702,7 @@ Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2 Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(123) ColumnarToRow [codegen id : 1] +(123) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] (124) BroadcastExchange @@ -711,7 +711,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (129) -+- * ColumnarToRow (128) ++- * CometColumnarToRow (128) +- CometProject (127) +- CometFilter (126) +- CometScan parquet spark_catalog.default.date_dim (125) @@ -732,7 +732,7 @@ Condition : (((isnotnull(d_year#125) AND (d_year#125 >= 1999)) AND (d_year#125 < Input [2]: [d_date_sk#26, d_year#125] Arguments: [d_date_sk#26], [d_date_sk#26] -(128) ColumnarToRow [codegen id : 1] +(128) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] (129) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/simplified.txt index 852af8ad82..ec0b46d611 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales),spark_grouping_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)] @@ -11,7 +11,7 @@ WholeStageCodegen (1) CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] Subquery #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #15 @@ -45,7 +45,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -69,7 +69,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/explain.txt index 4126fed5c3..a1c22982cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (85) +* CometColumnarToRow (85) +- CometTakeOrderedAndProject (84) +- CometBroadcastHashJoin (83) :- CometFilter (64) @@ -480,13 +480,13 @@ Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#62, i_c Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#50,i_brand_id#40,i_class_id#41,i_category_id#42,sales#51,number_sales#52,channel#72,i_brand_id#62,i_class_id#63,i_category_id#64,sales#73,number_sales#74]), [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74], 100, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -(85) ColumnarToRow [codegen id : 1] +(85) CometColumnarToRow [codegen id : 1] Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] ===== Subqueries ===== Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* ColumnarToRow (102) +* CometColumnarToRow (102) +- CometHashAggregate (101) +- CometExchange (100) +- CometHashAggregate (99) @@ -581,7 +581,7 @@ Input [2]: [sum#96, count#97] Keys: [] Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] -(102) ColumnarToRow [codegen id : 1] +(102) CometColumnarToRow [codegen id : 1] Input [1]: [average_sales#98] Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 @@ -592,7 +592,7 @@ Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (107) -+- * ColumnarToRow (106) ++- * CometColumnarToRow (106) +- CometProject (105) +- CometFilter (104) +- CometScan parquet spark_catalog.default.date_dim (103) @@ -613,7 +613,7 @@ Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subq Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(106) ColumnarToRow [codegen id : 1] +(106) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] (107) BroadcastExchange @@ -621,7 +621,7 @@ Input [1]: [d_date_sk#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* ColumnarToRow (111) +* CometColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometScan parquet spark_catalog.default.date_dim (108) @@ -642,12 +642,12 @@ Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] Arguments: [d_week_seq#99], [d_week_seq#99] -(111) ColumnarToRow [codegen id : 1] +(111) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#99] Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (116) -+- * ColumnarToRow (115) ++- * CometColumnarToRow (115) +- CometProject (114) +- CometFilter (113) +- CometScan parquet spark_catalog.default.date_dim (112) @@ -668,7 +668,7 @@ Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1999)) AND (d_year#103 < Input [2]: [d_date_sk#26, d_year#103] Arguments: [d_date_sk#26], [d_date_sk#26] -(115) ColumnarToRow [codegen id : 1] +(115) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] (116) BroadcastExchange @@ -685,7 +685,7 @@ Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquer Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 BroadcastExchange (121) -+- * ColumnarToRow (120) ++- * CometColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan parquet spark_catalog.default.date_dim (117) @@ -706,7 +706,7 @@ Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = Subquery scalar-subq Input [2]: [d_date_sk#65, d_week_seq#66] Arguments: [d_date_sk#65], [d_date_sk#65] -(120) ColumnarToRow [codegen id : 1] +(120) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#65] (121) BroadcastExchange @@ -714,7 +714,7 @@ Input [1]: [d_date_sk#65] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#67, [id=#68] -* ColumnarToRow (125) +* CometColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometScan parquet spark_catalog.default.date_dim (122) @@ -735,7 +735,7 @@ Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] Arguments: [d_week_seq#104], [d_week_seq#104] -(125) ColumnarToRow [codegen id : 1] +(125) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#104] Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/simplified.txt index bb4be4d3ae..befc6dadbb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/simplified.txt @@ -1,12 +1,12 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] CometBroadcastHashJoin [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Subquery #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #14 @@ -40,13 +40,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_week_seq] Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_week_seq,d_year,d_moy,d_dom] @@ -71,7 +71,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -132,13 +132,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_week_seq] Subquery #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_week_seq,d_year,d_moy,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/explain.txt index 037c526267..3ee326ad53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (23) +* CometColumnarToRow (23) +- CometTakeOrderedAndProject (22) +- CometHashAggregate (21) +- CometExchange (20) @@ -130,14 +130,14 @@ Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] Input [2]: [ca_zip#9, sum(cs_sales_price)#14] Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_zip#9 ASC NULLS FIRST], output=[ca_zip#9,sum(cs_sales_price)#14]), [ca_zip#9, sum(cs_sales_price)#14], 100, [ca_zip#9 ASC NULLS FIRST], [ca_zip#9, sum(cs_sales_price)#14] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarToRow [codegen id : 1] Input [2]: [ca_zip#9, sum(cs_sales_price)#14] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * ColumnarToRow (27) ++- * CometColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan parquet spark_catalog.default.date_dim (24) @@ -158,7 +158,7 @@ Condition : ((((isnotnull(d_qoy#12) AND isnotnull(d_year#11)) AND (d_qoy#12 = 2) Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(27) ColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/simplified.txt index d417f1c682..425f20e10c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [ca_zip,sum(cs_sales_price)] CometHashAggregate [ca_zip,sum(cs_sales_price),sum,sum(UnscaledValue(cs_sales_price))] @@ -16,7 +16,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/explain.txt index acd12b2771..5eeca31ae8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/explain.txt @@ -3,7 +3,7 @@ +- Exchange (39) +- * HashAggregate (38) +- * HashAggregate (37) - +- * ColumnarToRow (36) + +- * CometColumnarToRow (36) +- CometHashAggregate (35) +- CometProject (34) +- CometBroadcastHashJoin (33) @@ -203,7 +203,7 @@ Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [3]: [cs_order_number#5, sum#20, sum#21] (37) HashAggregate [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/simplified.txt index f054ee03e9..c51841720d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/simplified.txt @@ -5,7 +5,7 @@ WholeStageCodegen (2) WholeStageCodegen (1) HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [cs_order_number,sum,sum,cs_ext_ship_cost,cs_net_profit] CometProject [cs_order_number,cs_ext_ship_cost,cs_net_profit] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/explain.txt index 496ec2f1a1..08c00a0bc4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (42) +* CometColumnarToRow (42) +- CometTakeOrderedAndProject (41) +- CometHashAggregate (40) +- CometExchange (39) @@ -239,14 +239,14 @@ Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_qu Input [15]: [i_item_id#27, i_item_desc#28, s_state#25, store_sales_quantitycount#47, store_sales_quantityave#48, store_sales_quantitystdev#49, store_sales_quantitycov#50, as_store_returns_quantitycount#51, as_store_returns_quantityave#52, as_store_returns_quantitystdev#53, store_returns_quantitycov#54, catalog_sales_quantitycount#55, catalog_sales_quantityave#56, catalog_sales_quantitystdev#57, catalog_sales_quantitycov#58] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#27 ASC NULLS FIRST,i_item_desc#28 ASC NULLS FIRST,s_state#25 ASC NULLS FIRST], output=[i_item_id#27,i_item_desc#28,s_state#25,store_sales_quantitycount#47,store_sales_quantityave#48,store_sales_quantitystdev#49,store_sales_quantitycov#50,as_store_returns_quantitycount#51,as_store_returns_quantityave#52,as_store_returns_quantitystdev#53,store_returns_quantitycov#54,catalog_sales_quantitycount#55,catalog_sales_quantityave#56,catalog_sales_quantitystdev#57,catalog_sales_quantitycov#58]), [i_item_id#27, i_item_desc#28, s_state#25, store_sales_quantitycount#47, store_sales_quantityave#48, store_sales_quantitystdev#49, store_sales_quantitycov#50, as_store_returns_quantitycount#51, as_store_returns_quantityave#52, as_store_returns_quantitystdev#53, store_returns_quantitycov#54, catalog_sales_quantitycount#55, catalog_sales_quantityave#56, catalog_sales_quantitystdev#57, catalog_sales_quantitycov#58], 100, [i_item_id#27 ASC NULLS FIRST, i_item_desc#28 ASC NULLS FIRST, s_state#25 ASC NULLS FIRST], [i_item_id#27, i_item_desc#28, s_state#25, store_sales_quantitycount#47, store_sales_quantityave#48, store_sales_quantitystdev#49, store_sales_quantitycov#50, as_store_returns_quantitycount#51, as_store_returns_quantityave#52, as_store_returns_quantitystdev#53, store_returns_quantitycov#54, catalog_sales_quantitycount#55, catalog_sales_quantityave#56, catalog_sales_quantitystdev#57, catalog_sales_quantitycov#58] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarToRow [codegen id : 1] Input [15]: [i_item_id#27, i_item_desc#28, s_state#25, store_sales_quantitycount#47, store_sales_quantityave#48, store_sales_quantitystdev#49, store_sales_quantitycov#50, as_store_returns_quantitycount#51, as_store_returns_quantityave#52, as_store_returns_quantitystdev#53, store_returns_quantitycov#54, catalog_sales_quantitycount#55, catalog_sales_quantityave#56, catalog_sales_quantitystdev#57, catalog_sales_quantitycov#58] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (47) -+- * ColumnarToRow (46) ++- * CometColumnarToRow (46) +- CometProject (45) +- CometFilter (44) +- CometScan parquet spark_catalog.default.date_dim (43) @@ -267,7 +267,7 @@ Condition : ((isnotnull(d_quarter_name#20) AND (d_quarter_name#20 = 2001Q1)) AND Input [2]: [d_date_sk#19, d_quarter_name#20] Arguments: [d_date_sk#19], [d_date_sk#19] -(46) ColumnarToRow [codegen id : 1] +(46) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] (47) BroadcastExchange @@ -276,7 +276,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -297,7 +297,7 @@ Condition : (d_quarter_name#22 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk Input [2]: [d_date_sk#21, d_quarter_name#22] Arguments: [d_date_sk#21], [d_date_sk#21] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#21] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/simplified.txt index a3adaf1724..97000b91df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] CometHashAggregate [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count(ss_quantity),avg(ss_quantity),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(sr_return_quantity),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cs_quantity),stddev_samp(cast(cs_quantity as double))] @@ -24,7 +24,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_quarter_name] @@ -35,7 +35,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_quarter_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/explain.txt index c41a1ef7d4..2d08a0303c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (41) +* CometColumnarToRow (41) +- CometTakeOrderedAndProject (40) +- CometHashAggregate (39) +- CometExchange (38) @@ -232,14 +232,14 @@ Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_pric Input [11]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, agg1#48, agg2#49, agg3#50, agg4#51, agg5#52, agg6#53, agg7#54] Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#30 ASC NULLS FIRST,ca_state#31 ASC NULLS FIRST,ca_county#32 ASC NULLS FIRST,i_item_id#29 ASC NULLS FIRST], output=[i_item_id#29,ca_country#30,ca_state#31,ca_county#32,agg1#48,agg2#49,agg3#50,agg4#51,agg5#52,agg6#53,agg7#54]), [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, agg1#48, agg2#49, agg3#50, agg4#51, agg5#52, agg6#53, agg7#54], 100, [ca_country#30 ASC NULLS FIRST, ca_state#31 ASC NULLS FIRST, ca_county#32 ASC NULLS FIRST, i_item_id#29 ASC NULLS FIRST], [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, agg1#48, agg2#49, agg3#50, agg4#51, agg5#52, agg6#53, agg7#54] -(41) ColumnarToRow [codegen id : 1] +(41) CometColumnarToRow [codegen id : 1] Input [11]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, agg1#48, agg2#49, agg3#50, agg4#51, agg5#52, agg6#53, agg7#54] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (46) -+- * ColumnarToRow (45) ++- * CometColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometScan parquet spark_catalog.default.date_dim (42) @@ -260,7 +260,7 @@ Condition : ((isnotnull(d_year#26) AND (d_year#26 = 1998)) AND isnotnull(d_date_ Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(45) ColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/simplified.txt index e26491c76a..eaf02960fe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2)))] @@ -23,7 +23,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/explain.txt index b91551e576..c1bf0f14a0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (34) +* CometColumnarToRow (34) +- CometTakeOrderedAndProject (33) +- CometHashAggregate (32) +- CometExchange (31) @@ -193,6 +193,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] Input [5]: [brand_id#22, brand#23, i_manufact_id#12, i_manufact#13, ext_price#24] Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#24 DESC NULLS LAST,brand#23 ASC NULLS FIRST,brand_id#22 ASC NULLS FIRST,i_manufact_id#12 ASC NULLS FIRST,i_manufact#13 ASC NULLS FIRST], output=[brand_id#22,brand#23,i_manufact_id#12,i_manufact#13,ext_price#24]), [brand_id#22, brand#23, i_manufact_id#12, i_manufact#13, ext_price#24], 100, [ext_price#24 DESC NULLS LAST, brand#23 ASC NULLS FIRST, brand_id#22 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#13 ASC NULLS FIRST], [brand_id#22, brand#23, i_manufact_id#12, i_manufact#13, ext_price#24] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [5]: [brand_id#22, brand#23, i_manufact_id#12, i_manufact#13, ext_price#24] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/simplified.txt index 46b0d650a5..bfbf7fe94a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [brand_id,brand,i_manufact_id,i_manufact,ext_price] CometHashAggregate [brand_id,brand,i_manufact_id,i_manufact,ext_price,i_brand,i_brand_id,sum,sum(UnscaledValue(ss_ext_sales_price))] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/explain.txt index d51ae75262..30e7e4ae49 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (33) +* CometColumnarToRow (33) +- CometSort (32) +- CometColumnarExchange (31) +- CometProject (30) @@ -183,6 +183,6 @@ Arguments: rangepartitioning(d_week_seq1#28 ASC NULLS FIRST, 5), ENSURE_REQUIREM Input [8]: [d_week_seq1#28, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68] Arguments: [d_week_seq1#28, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68], [d_week_seq1#28 ASC NULLS FIRST] -(33) ColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [codegen id : 1] Input [8]: [d_week_seq1#28, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt index ed772bade0..38cb758b73 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] CometColumnarExchange [d_week_seq1] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/explain.txt index 909acbf7f8..6a32155103 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (22) +- * Project (21) +- Window (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometSort (18) +- CometExchange (17) +- CometHashAggregate (16) @@ -109,7 +109,7 @@ Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometNativeShuff Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6], [i_class#9 ASC NULLS FIRST] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] (20) Window @@ -128,7 +128,7 @@ Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_ite Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (27) -+- * ColumnarToRow (26) ++- * CometColumnarToRow (26) +- CometProject (25) +- CometFilter (24) +- CometScan parquet spark_catalog.default.date_dim (23) @@ -149,7 +149,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (27) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/simplified.txt index 9a906085b8..147d91b586 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/simplified.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] CometExchange [i_class] #1 @@ -20,7 +20,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/explain.txt index 31a5bef868..0340ef91c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (24) +* CometColumnarToRow (24) +- CometTakeOrderedAndProject (23) +- CometFilter (22) +- CometHashAggregate (21) @@ -135,14 +135,14 @@ Condition : (CASE WHEN (inv_before#15 > 0) THEN (knownfloatingpointnormalized(no Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#15, inv_after#16] Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_warehouse_name#7 ASC NULLS FIRST,i_item_id#9 ASC NULLS FIRST], output=[w_warehouse_name#7,i_item_id#9,inv_before#15,inv_after#16]), [w_warehouse_name#7, i_item_id#9, inv_before#15, inv_after#16], 100, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST], [w_warehouse_name#7, i_item_id#9, inv_before#15, inv_after#16] -(24) ColumnarToRow [codegen id : 1] +(24) CometColumnarToRow [codegen id : 1] Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#15, inv_after#16] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (28) -+- * ColumnarToRow (27) ++- * CometColumnarToRow (27) +- CometFilter (26) +- CometScan parquet spark_catalog.default.date_dim (25) @@ -158,7 +158,7 @@ ReadSchema: struct Input [2]: [d_date_sk#11, d_date#12] Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-10)) AND (d_date#12 <= 2000-04-10)) AND isnotnull(d_date_sk#11)) -(27) ColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#11, d_date#12] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/simplified.txt index 6e483f6049..6ff97b53ac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] CometFilter [w_warehouse_name,i_item_id,inv_before,inv_after] @@ -17,7 +17,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/explain.txt index 526dadb3c8..759689b1b8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (24) +* CometColumnarToRow (24) +- CometTakeOrderedAndProject (23) +- CometHashAggregate (22) +- CometExchange (21) @@ -135,14 +135,14 @@ Functions [1]: [avg(inv_quantity_on_hand#3)] Input [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#21] Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#21 ASC NULLS FIRST,i_product_name#14 ASC NULLS FIRST,i_brand#15 ASC NULLS FIRST,i_class#16 ASC NULLS FIRST,i_category#17 ASC NULLS FIRST], output=[i_product_name#14,i_brand#15,i_class#16,i_category#17,qoh#21]), [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#21], 100, [qoh#21 ASC NULLS FIRST, i_product_name#14 ASC NULLS FIRST, i_brand#15 ASC NULLS FIRST, i_class#16 ASC NULLS FIRST, i_category#17 ASC NULLS FIRST], [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#21] -(24) ColumnarToRow [codegen id : 1] +(24) CometColumnarToRow [codegen id : 1] Input [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (29) -+- * ColumnarToRow (28) ++- * CometColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan parquet spark_catalog.default.date_dim (25) @@ -163,7 +163,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_mont Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/simplified.txt index ed997b5d50..f9d4ce909d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,spark_grouping_id,sum,count,avg(inv_quantity_on_hand)] @@ -17,7 +17,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/explain.txt index 0879422474..6a67b0f6d3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (67) +* CometColumnarToRow (67) +- CometHashAggregate (66) +- CometExchange (65) +- CometHashAggregate (64) @@ -370,14 +370,14 @@ Input [2]: [sum#48, isEmpty#49] Keys: [] Functions [1]: [sum(sales#32)] -(67) ColumnarToRow [codegen id : 1] +(67) CometColumnarToRow [codegen id : 1] Input [1]: [sum(sales)#50] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (72) -+- * ColumnarToRow (71) ++- * CometColumnarToRow (71) +- CometProject (70) +- CometFilter (69) +- CometScan parquet spark_catalog.default.date_dim (68) @@ -398,7 +398,7 @@ Condition : ((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2 Input [3]: [d_date_sk#29, d_year#30, d_moy#31] Arguments: [d_date_sk#29], [d_date_sk#29] -(71) ColumnarToRow [codegen id : 1] +(71) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#29] (72) BroadcastExchange @@ -407,7 +407,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 2 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (77) -+- * ColumnarToRow (76) ++- * CometColumnarToRow (76) +- CometProject (75) +- CometFilter (74) +- CometScan parquet spark_catalog.default.date_dim (73) @@ -428,7 +428,7 @@ Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(76) ColumnarToRow [codegen id : 1] +(76) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] (77) BroadcastExchange @@ -436,7 +436,7 @@ Input [2]: [d_date_sk#10, d_date#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:3 Hosting operator id = 36 Hosting Expression = Subquery scalar-subquery#27, [id=#28] -* ColumnarToRow (95) +* CometColumnarToRow (95) +- CometHashAggregate (94) +- CometExchange (93) +- CometHashAggregate (92) @@ -536,12 +536,12 @@ Input [1]: [max#62] Keys: [] Functions [1]: [max(csales#61)] -(95) ColumnarToRow [codegen id : 1] +(95) CometColumnarToRow [codegen id : 1] Input [1]: [tpcds_cmax#63] Subquery:4 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#54 IN dynamicpruning#55 BroadcastExchange (100) -+- * ColumnarToRow (99) ++- * CometColumnarToRow (99) +- CometProject (98) +- CometFilter (97) +- CometScan parquet spark_catalog.default.date_dim (96) @@ -562,7 +562,7 @@ Condition : (d_year#58 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#57)) Input [2]: [d_date_sk#57, d_year#58] Arguments: [d_date_sk#57], [d_date_sk#57] -(99) ColumnarToRow [codegen id : 1] +(99) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#57] (100) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/simplified.txt index 7606f105f3..4ecd0dfcc5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [sum(sales),sum,isEmpty,sum(sales)] CometExchange #1 @@ -17,7 +17,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -37,7 +37,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_year] @@ -54,7 +54,7 @@ WholeStageCodegen (1) CometFilter [c_customer_sk,ssales] Subquery #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [tpcds_cmax,max,max(csales)] CometExchange #11 @@ -71,7 +71,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/explain.txt index b9d5b25fd4..8ed54d7afa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (87) +* CometColumnarToRow (87) +- CometTakeOrderedAndProject (86) +- CometUnion (85) :- CometHashAggregate (62) @@ -479,14 +479,14 @@ Child 1 Input [3]: [c_last_name#52, c_first_name#51, sales#57] Input [3]: [c_last_name#31, c_first_name#30, sales#56] Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#31 ASC NULLS FIRST,c_first_name#30 ASC NULLS FIRST,sales#56 ASC NULLS FIRST], output=[c_last_name#31,c_first_name#30,sales#56]), [c_last_name#31, c_first_name#30, sales#56], 100, [c_last_name#31 ASC NULLS FIRST, c_first_name#30 ASC NULLS FIRST, sales#56 ASC NULLS FIRST], [c_last_name#31, c_first_name#30, sales#56] -(87) ColumnarToRow [codegen id : 1] +(87) CometColumnarToRow [codegen id : 1] Input [3]: [c_last_name#31, c_first_name#30, sales#56] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (92) -+- * ColumnarToRow (91) ++- * CometColumnarToRow (91) +- CometProject (90) +- CometFilter (89) +- CometScan parquet spark_catalog.default.date_dim (88) @@ -507,7 +507,7 @@ Condition : ((((isnotnull(d_year#33) AND isnotnull(d_moy#34)) AND (d_year#33 = 2 Input [3]: [d_date_sk#32, d_year#33, d_moy#34] Arguments: [d_date_sk#32], [d_date_sk#32] -(91) ColumnarToRow [codegen id : 1] +(91) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#32] (92) BroadcastExchange @@ -516,7 +516,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (97) -+- * ColumnarToRow (96) ++- * CometColumnarToRow (96) +- CometProject (95) +- CometFilter (94) +- CometScan parquet spark_catalog.default.date_dim (93) @@ -537,7 +537,7 @@ Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(96) ColumnarToRow [codegen id : 1] +(96) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] (97) BroadcastExchange @@ -545,7 +545,7 @@ Input [2]: [d_date_sk#10, d_date#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] Subquery:3 Hosting operator id = 37 Hosting Expression = Subquery scalar-subquery#27, [id=#28] -* ColumnarToRow (115) +* CometColumnarToRow (115) +- CometHashAggregate (114) +- CometExchange (113) +- CometHashAggregate (112) @@ -645,12 +645,12 @@ Input [1]: [max#69] Keys: [] Functions [1]: [max(csales#68)] -(115) ColumnarToRow [codegen id : 1] +(115) CometColumnarToRow [codegen id : 1] Input [1]: [tpcds_cmax#70] Subquery:4 Hosting operator id = 98 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#62 BroadcastExchange (120) -+- * ColumnarToRow (119) ++- * CometColumnarToRow (119) +- CometProject (118) +- CometFilter (117) +- CometScan parquet spark_catalog.default.date_dim (116) @@ -671,7 +671,7 @@ Condition : (d_year#65 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#64)) Input [2]: [d_date_sk#64, d_year#65] Arguments: [d_date_sk#64], [d_date_sk#64] -(119) ColumnarToRow [codegen id : 1] +(119) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#64] (120) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt index d503bb68af..9493bffd25 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [c_last_name,c_first_name,sales] CometUnion [c_last_name,c_first_name,sales] @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -40,7 +40,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_year] @@ -57,7 +57,7 @@ WholeStageCodegen (1) CometFilter [c_customer_sk,ssales] Subquery #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [tpcds_cmax,max,max(csales)] CometExchange #11 @@ -74,7 +74,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/explain.txt index 9dbf4af839..d7b76b4dde 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/explain.txt @@ -8,7 +8,7 @@ +- * HashAggregate (36) +- * Project (35) +- * BroadcastHashJoin Inner BuildRight (34) - :- * ColumnarToRow (29) + :- * CometColumnarToRow (29) : +- CometProject (28) : +- CometBroadcastHashJoin (27) : :- CometProject (23) @@ -38,7 +38,7 @@ : +- CometFilter (25) : +- CometScan parquet spark_catalog.default.customer (24) +- BroadcastExchange (33) - +- * ColumnarToRow (32) + +- * CometColumnarToRow (32) +- CometFilter (31) +- CometScan parquet spark_catalog.default.customer_address (30) @@ -174,7 +174,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Arguments: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24], [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] -(29) ColumnarToRow [codegen id : 2] +(29) CometColumnarToRow [codegen id : 2] Input [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] (30) CometScan parquet spark_catalog.default.customer_address @@ -188,7 +188,7 @@ ReadSchema: struct Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] (33) BroadcastExchange @@ -256,7 +256,7 @@ Subquery:1 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquer +- * HashAggregate (64) +- * Project (63) +- * BroadcastHashJoin Inner BuildRight (62) - :- * ColumnarToRow (60) + :- * CometColumnarToRow (60) : +- CometProject (59) : +- CometBroadcastHashJoin (58) : :- CometProject (56) @@ -348,7 +348,7 @@ Arguments: [ss_customer_sk#41], [c_customer_sk#57], Inner, BuildRight Input [14]: [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_customer_sk#57, c_first_name#58, c_last_name#59, c_birth_country#60] Arguments: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60], [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60] -(60) ColumnarToRow [codegen id : 2] +(60) CometColumnarToRow [codegen id : 2] Input [12]: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60] (61) ReusedExchange [Reuses operator id: 33] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt index bd14d82504..cd048032d9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt @@ -14,7 +14,7 @@ WholeStageCodegen (4) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] @@ -47,7 +47,7 @@ WholeStageCodegen (4) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] @@ -80,7 +80,7 @@ WholeStageCodegen (4) InputAdapter BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/explain.txt index f27ae40199..9730c4bd08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/explain.txt @@ -8,7 +8,7 @@ +- * HashAggregate (36) +- * Project (35) +- * BroadcastHashJoin Inner BuildRight (34) - :- * ColumnarToRow (29) + :- * CometColumnarToRow (29) : +- CometProject (28) : +- CometBroadcastHashJoin (27) : :- CometProject (23) @@ -38,7 +38,7 @@ : +- CometFilter (25) : +- CometScan parquet spark_catalog.default.customer (24) +- BroadcastExchange (33) - +- * ColumnarToRow (32) + +- * CometColumnarToRow (32) +- CometFilter (31) +- CometScan parquet spark_catalog.default.customer_address (30) @@ -174,7 +174,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Arguments: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24], [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] -(29) ColumnarToRow [codegen id : 2] +(29) CometColumnarToRow [codegen id : 2] Input [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] (30) CometScan parquet spark_catalog.default.customer_address @@ -188,7 +188,7 @@ ReadSchema: struct Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] (33) BroadcastExchange @@ -256,7 +256,7 @@ Subquery:1 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquer +- * HashAggregate (64) +- * Project (63) +- * BroadcastHashJoin Inner BuildRight (62) - :- * ColumnarToRow (60) + :- * CometColumnarToRow (60) : +- CometProject (59) : +- CometBroadcastHashJoin (58) : :- CometProject (56) @@ -348,7 +348,7 @@ Arguments: [ss_customer_sk#41], [c_customer_sk#57], Inner, BuildRight Input [14]: [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_customer_sk#57, c_first_name#58, c_last_name#59, c_birth_country#60] Arguments: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60], [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60] -(60) ColumnarToRow [codegen id : 2] +(60) CometColumnarToRow [codegen id : 2] Input [12]: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60] (61) ReusedExchange [Reuses operator id: 33] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt index bd14d82504..cd048032d9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt @@ -14,7 +14,7 @@ WholeStageCodegen (4) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] @@ -47,7 +47,7 @@ WholeStageCodegen (4) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] @@ -80,7 +80,7 @@ WholeStageCodegen (4) InputAdapter BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/explain.txt index 6c78335481..62c54c9f56 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (42) +* CometColumnarToRow (42) +- CometTakeOrderedAndProject (41) +- CometHashAggregate (40) +- CometExchange (39) @@ -239,14 +239,14 @@ Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_lo Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#35, store_returns_loss#36, catalog_sales_profit#37] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#30 ASC NULLS FIRST,i_item_desc#31 ASC NULLS FIRST,s_store_id#27 ASC NULLS FIRST,s_store_name#28 ASC NULLS FIRST], output=[i_item_id#30,i_item_desc#31,s_store_id#27,s_store_name#28,store_sales_profit#35,store_returns_loss#36,catalog_sales_profit#37]), [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#35, store_returns_loss#36, catalog_sales_profit#37], 100, [i_item_id#30 ASC NULLS FIRST, i_item_desc#31 ASC NULLS FIRST, s_store_id#27 ASC NULLS FIRST, s_store_name#28 ASC NULLS FIRST], [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#35, store_returns_loss#36, catalog_sales_profit#37] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#35, store_returns_loss#36, catalog_sales_profit#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (47) -+- * ColumnarToRow (46) ++- * CometColumnarToRow (46) +- CometProject (45) +- CometFilter (44) +- CometScan parquet spark_catalog.default.date_dim (43) @@ -267,7 +267,7 @@ Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4) Input [3]: [d_date_sk#19, d_year#20, d_moy#21] Arguments: [d_date_sk#19], [d_date_sk#19] -(46) ColumnarToRow [codegen id : 1] +(46) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] (47) BroadcastExchange @@ -276,7 +276,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -297,7 +297,7 @@ Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= Input [3]: [d_date_sk#22, d_year#23, d_moy#24] Arguments: [d_date_sk#22], [d_date_sk#22] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt index 13cb05053c..e64e8bc840 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit,sum,sum,sum,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit))] @@ -24,7 +24,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -35,7 +35,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/explain.txt index b93df3398e..525313e3f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (30) +* CometColumnarToRow (30) +- CometTakeOrderedAndProject (29) +- CometHashAggregate (28) +- CometExchange (27) @@ -169,14 +169,14 @@ Functions [4]: [avg(cs_quantity#4), avg(UnscaledValue(cs_list_price#5)), avg(Uns Input [5]: [i_item_id#17, agg1#29, agg2#30, agg3#31, agg4#32] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#17 ASC NULLS FIRST], output=[i_item_id#17,agg1#29,agg2#30,agg3#31,agg4#32]), [i_item_id#17, agg1#29, agg2#30, agg3#31, agg4#32], 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#29, agg2#30, agg3#31, agg4#32] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [5]: [i_item_id#17, agg1#29, agg2#30, agg3#31, agg4#32] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (35) -+- * ColumnarToRow (34) ++- * CometColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan parquet spark_catalog.default.date_dim (31) @@ -197,7 +197,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/simplified.txt index f643ff7b04..16f8d0848e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count,avg(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price))] @@ -18,7 +18,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/explain.txt index 4d1b12626a..28bc38292b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (30) +* CometColumnarToRow (30) +- CometTakeOrderedAndProject (29) +- CometHashAggregate (28) +- CometExchange (27) @@ -169,14 +169,14 @@ Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(Uns Input [7]: [i_item_id#20, s_state#21, g_state#31, agg1#32, agg2#33, agg3#34, agg4#35] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#20 ASC NULLS FIRST,s_state#21 ASC NULLS FIRST], output=[i_item_id#20,s_state#21,g_state#31,agg1#32,agg2#33,agg3#34,agg4#35]), [i_item_id#20, s_state#21, g_state#31, agg1#32, agg2#33, agg3#34, agg4#35], 100, [i_item_id#20 ASC NULLS FIRST, s_state#21 ASC NULLS FIRST], [i_item_id#20, s_state#21, g_state#31, agg1#32, agg2#33, agg3#34, agg4#35] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#20, s_state#21, g_state#31, agg1#32, agg2#33, agg3#34, agg4#35] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (35) -+- * ColumnarToRow (34) ++- * CometColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan parquet spark_catalog.default.date_dim (31) @@ -197,7 +197,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/simplified.txt index 121f1be7ce..e65b02d4d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] CometHashAggregate [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price))] @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/explain.txt index 8f0671c8f0..2e36b987c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/explain.txt @@ -8,7 +8,7 @@ : : : : : +- Exchange (9) : : : : : +- * HashAggregate (8) : : : : : +- * HashAggregate (7) -: : : : : +- * ColumnarToRow (6) +: : : : : +- * CometColumnarToRow (6) : : : : : +- CometExchange (5) : : : : : +- CometHashAggregate (4) : : : : : +- CometProject (3) @@ -19,7 +19,7 @@ : : : : +- Exchange (19) : : : : +- * HashAggregate (18) : : : : +- * HashAggregate (17) -: : : : +- * ColumnarToRow (16) +: : : : +- * CometColumnarToRow (16) : : : : +- CometExchange (15) : : : : +- CometHashAggregate (14) : : : : +- CometProject (13) @@ -30,7 +30,7 @@ : : : +- Exchange (31) : : : +- * HashAggregate (30) : : : +- * HashAggregate (29) -: : : +- * ColumnarToRow (28) +: : : +- * CometColumnarToRow (28) : : : +- CometExchange (27) : : : +- CometHashAggregate (26) : : : +- CometProject (25) @@ -41,7 +41,7 @@ : : +- Exchange (43) : : +- * HashAggregate (42) : : +- * HashAggregate (41) -: : +- * ColumnarToRow (40) +: : +- * CometColumnarToRow (40) : : +- CometExchange (39) : : +- CometHashAggregate (38) : : +- CometProject (37) @@ -52,7 +52,7 @@ : +- Exchange (55) : +- * HashAggregate (54) : +- * HashAggregate (53) -: +- * ColumnarToRow (52) +: +- * CometColumnarToRow (52) : +- CometExchange (51) : +- CometHashAggregate (50) : +- CometProject (49) @@ -63,7 +63,7 @@ +- Exchange (67) +- * HashAggregate (66) +- * HashAggregate (65) - +- * ColumnarToRow (64) + +- * CometColumnarToRow (64) +- CometExchange (63) +- CometHashAggregate (62) +- CometProject (61) @@ -95,7 +95,7 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_li Input [4]: [ss_list_price#3, sum#6, count#7, count#8] Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(6) ColumnarToRow [codegen id : 1] +(6) CometColumnarToRow [codegen id : 1] Input [4]: [ss_list_price#3, sum#6, count#7, count#8] (7) HashAggregate [codegen id : 1] @@ -147,7 +147,7 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#18)), partial_count(ss_l Input [4]: [ss_list_price#18, sum#21, count#22, count#23] Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(16) ColumnarToRow [codegen id : 2] +(16) CometColumnarToRow [codegen id : 2] Input [4]: [ss_list_price#18, sum#21, count#22, count#23] (17) HashAggregate [codegen id : 2] @@ -207,7 +207,7 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#33)), partial_count(ss_l Input [4]: [ss_list_price#33, sum#36, count#37, count#38] Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(28) ColumnarToRow [codegen id : 4] +(28) CometColumnarToRow [codegen id : 4] Input [4]: [ss_list_price#33, sum#36, count#37, count#38] (29) HashAggregate [codegen id : 4] @@ -267,7 +267,7 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#48)), partial_count(ss_l Input [4]: [ss_list_price#48, sum#51, count#52, count#53] Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] -(40) ColumnarToRow [codegen id : 6] +(40) CometColumnarToRow [codegen id : 6] Input [4]: [ss_list_price#48, sum#51, count#52, count#53] (41) HashAggregate [codegen id : 6] @@ -327,7 +327,7 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#63)), partial_count(ss_l Input [4]: [ss_list_price#63, sum#66, count#67, count#68] Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(52) ColumnarToRow [codegen id : 8] +(52) CometColumnarToRow [codegen id : 8] Input [4]: [ss_list_price#63, sum#66, count#67, count#68] (53) HashAggregate [codegen id : 8] @@ -387,7 +387,7 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#78)), partial_count(ss_l Input [4]: [ss_list_price#78, sum#81, count#82, count#83] Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] -(64) ColumnarToRow [codegen id : 10] +(64) CometColumnarToRow [codegen id : 10] Input [4]: [ss_list_price#78, sum#81, count#82, count#83] (65) HashAggregate [codegen id : 10] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/simplified.txt index 4a547c4e04..04eaf6673f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/simplified.txt @@ -10,7 +10,7 @@ WholeStageCodegen (12) WholeStageCodegen (1) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [ss_list_price] #2 CometHashAggregate [ss_list_price,sum,count,count] @@ -26,7 +26,7 @@ WholeStageCodegen (12) WholeStageCodegen (2) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [ss_list_price] #5 CometHashAggregate [ss_list_price,sum,count,count] @@ -42,7 +42,7 @@ WholeStageCodegen (12) WholeStageCodegen (4) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [ss_list_price] #8 CometHashAggregate [ss_list_price,sum,count,count] @@ -58,7 +58,7 @@ WholeStageCodegen (12) WholeStageCodegen (6) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [ss_list_price] #11 CometHashAggregate [ss_list_price,sum,count,count] @@ -74,7 +74,7 @@ WholeStageCodegen (12) WholeStageCodegen (8) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [ss_list_price] #14 CometHashAggregate [ss_list_price,sum,count,count] @@ -90,7 +90,7 @@ WholeStageCodegen (12) WholeStageCodegen (10) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [ss_list_price] #17 CometHashAggregate [ss_list_price,sum,count,count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/explain.txt index 71d583785d..b02bf3a4cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (45) +* CometColumnarToRow (45) +- CometTakeOrderedAndProject (44) +- CometHashAggregate (43) +- CometExchange (42) @@ -258,14 +258,14 @@ Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#11), sum(cs_quantity# Input [7]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, store_sales_quantity#36, store_returns_quantity#37, catalog_sales_quantity#38] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#31 ASC NULLS FIRST,i_item_desc#32 ASC NULLS FIRST,s_store_id#28 ASC NULLS FIRST,s_store_name#29 ASC NULLS FIRST], output=[i_item_id#31,i_item_desc#32,s_store_id#28,s_store_name#29,store_sales_quantity#36,store_returns_quantity#37,catalog_sales_quantity#38]), [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, store_sales_quantity#36, store_returns_quantity#37, catalog_sales_quantity#38], 100, [i_item_id#31 ASC NULLS FIRST, i_item_desc#32 ASC NULLS FIRST, s_store_id#28 ASC NULLS FIRST, s_store_name#29 ASC NULLS FIRST], [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, store_sales_quantity#36, store_returns_quantity#37, catalog_sales_quantity#38] -(45) ColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, store_sales_quantity#36, store_returns_quantity#37, catalog_sales_quantity#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (50) -+- * ColumnarToRow (49) ++- * CometColumnarToRow (49) +- CometProject (48) +- CometFilter (47) +- CometScan parquet spark_catalog.default.date_dim (46) @@ -286,7 +286,7 @@ Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9) Input [3]: [d_date_sk#19, d_year#20, d_moy#21] Arguments: [d_date_sk#19], [d_date_sk#19] -(49) ColumnarToRow [codegen id : 1] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] (50) BroadcastExchange @@ -295,7 +295,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (55) -+- * ColumnarToRow (54) ++- * CometColumnarToRow (54) +- CometProject (53) +- CometFilter (52) +- CometScan parquet spark_catalog.default.date_dim (51) @@ -316,7 +316,7 @@ Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= Input [3]: [d_date_sk#22, d_year#23, d_moy#24] Arguments: [d_date_sk#22], [d_date_sk#22] -(54) ColumnarToRow [codegen id : 1] +(54) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] (55) BroadcastExchange @@ -325,7 +325,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 BroadcastExchange (60) -+- * ColumnarToRow (59) ++- * CometColumnarToRow (59) +- CometProject (58) +- CometFilter (57) +- CometScan parquet spark_catalog.default.date_dim (56) @@ -346,7 +346,7 @@ Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(59) ColumnarToRow [codegen id : 1] +(59) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] (60) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/simplified.txt index 9398fcdf5e..9340a4e5f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum,sum,sum,sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity)] @@ -24,7 +24,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -35,7 +35,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -46,7 +46,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/explain.txt index a0796a47af..e80a2eaa78 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Input [4]: [d_year#2, brand_id#12, brand#13, sum_agg#14] Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,sum_agg#14 DESC NULLS LAST,brand_id#12 ASC NULLS FIRST], output=[d_year#2,brand_id#12,brand#13,sum_agg#14]), [d_year#2, brand_id#12, brand#13, sum_agg#14], 100, [d_year#2 ASC NULLS FIRST, sum_agg#14 DESC NULLS LAST, brand_id#12 ASC NULLS FIRST], [d_year#2, brand_id#12, brand#13, sum_agg#14] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [4]: [d_year#2, brand_id#12, brand#13, sum_agg#14] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/simplified.txt index f82fd24bdd..7bc02651ec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [d_year,brand_id,brand,sum_agg] CometHashAggregate [d_year,brand_id,brand,sum_agg,i_brand,i_brand_id,sum,sum(UnscaledValue(ss_ext_sales_price))] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/explain.txt index a634822f58..83d3179285 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (48) +* CometColumnarToRow (48) +- CometTakeOrderedAndProject (47) +- CometProject (46) +- CometBroadcastHashJoin (45) @@ -268,14 +268,14 @@ Arguments: [c_customer_id#29, c_salutation#31, c_first_name#32, c_last_name#33, Input [13]: [c_customer_id#29, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41, ctr_total_return#13] Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#29 ASC NULLS FIRST,c_salutation#31 ASC NULLS FIRST,c_first_name#32 ASC NULLS FIRST,c_last_name#33 ASC NULLS FIRST,c_preferred_cust_flag#34 ASC NULLS FIRST,c_birth_day#35 ASC NULLS FIRST,c_birth_month#36 ASC NULLS FIRST,c_birth_year#37 ASC NULLS FIRST,c_birth_country#38 ASC NULLS FIRST,c_login#39 ASC NULLS FIRST,c_email_address#40 ASC NULLS FIRST,c_last_review_date#41 ASC NULLS FIRST,ctr_total_return#13 ASC NULLS FIRST], output=[c_customer_id#29,c_salutation#31,c_first_name#32,c_last_name#33,c_preferred_cust_flag#34,c_birth_day#35,c_birth_month#36,c_birth_year#37,c_birth_country#38,c_login#39,c_email_address#40,c_last_review_date#41,ctr_total_return#13]), [c_customer_id#29, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41, ctr_total_return#13], 100, [c_customer_id#29 ASC NULLS FIRST, c_salutation#31 ASC NULLS FIRST, c_first_name#32 ASC NULLS FIRST, c_last_name#33 ASC NULLS FIRST, c_preferred_cust_flag#34 ASC NULLS FIRST, c_birth_day#35 ASC NULLS FIRST, c_birth_month#36 ASC NULLS FIRST, c_birth_year#37 ASC NULLS FIRST, c_birth_country#38 ASC NULLS FIRST, c_login#39 ASC NULLS FIRST, c_email_address#40 ASC NULLS FIRST, c_last_review_date#41 ASC NULLS FIRST, ctr_total_return#13 ASC NULLS FIRST], [c_customer_id#29, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41, ctr_total_return#13] -(48) ColumnarToRow [codegen id : 1] +(48) CometColumnarToRow [codegen id : 1] Input [13]: [c_customer_id#29, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41, ctr_total_return#13] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (53) -+- * ColumnarToRow (52) ++- * CometColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometScan parquet spark_catalog.default.date_dim (49) @@ -296,7 +296,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(52) ColumnarToRow [codegen id : 1] +(52) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/simplified.txt index 824cc51163..2a789450e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] @@ -21,7 +21,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/explain.txt index 76a62c8926..6ce03737eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (90) +* CometColumnarToRow (90) +- CometSort (89) +- CometColumnarExchange (88) +- CometProject (87) @@ -504,14 +504,14 @@ Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENT Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9 ASC NULLS FIRST] -(90) ColumnarToRow [codegen id : 1] +(90) CometColumnarToRow [codegen id : 1] Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (94) -+- * ColumnarToRow (93) ++- * CometColumnarToRow (93) +- CometFilter (92) +- CometScan parquet spark_catalog.default.date_dim (91) @@ -527,7 +527,7 @@ ReadSchema: struct Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(93) ColumnarToRow [codegen id : 1] +(93) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] (94) BroadcastExchange @@ -536,7 +536,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (98) -+- * ColumnarToRow (97) ++- * CometColumnarToRow (97) +- CometFilter (96) +- CometScan parquet spark_catalog.default.date_dim (95) @@ -552,7 +552,7 @@ ReadSchema: struct Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) -(97) ColumnarToRow [codegen id : 1] +(97) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] (98) BroadcastExchange @@ -561,7 +561,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:3 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 BroadcastExchange (102) -+- * ColumnarToRow (101) ++- * CometColumnarToRow (101) +- CometFilter (100) +- CometScan parquet spark_catalog.default.date_dim (99) @@ -577,7 +577,7 @@ ReadSchema: struct Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) -(101) ColumnarToRow [codegen id : 1] +(101) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] (102) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt index d6615fe1c1..22635bb9f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] CometColumnarExchange [ca_county] #1 @@ -23,7 +23,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] @@ -46,7 +46,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] @@ -67,7 +67,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #12 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/explain.txt index 03edc9859f..7caa3c6e24 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (30) +* CometColumnarToRow (30) +- CometHashAggregate (29) +- CometExchange (28) +- CometHashAggregate (27) @@ -168,14 +168,14 @@ Input [1]: [sum#17] Keys: [] Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [1]: [excess discount amount#18] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (35) -+- * ColumnarToRow (34) ++- * CometColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan parquet spark_catalog.default.date_dim (31) @@ -196,7 +196,7 @@ Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#1 Input [2]: [d_date_sk#16, d_date#19] Arguments: [d_date_sk#16], [d_date_sk#16] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/simplified.txt index 14b2a34a89..2650006a6c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [excess discount amount,sum,sum(UnscaledValue(cs_ext_discount_amt))] CometExchange #1 @@ -15,7 +15,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/explain.txt index 7a50fe69e9..587c91520a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (61) +* CometColumnarToRow (61) +- CometTakeOrderedAndProject (60) +- CometHashAggregate (59) +- CometExchange (58) @@ -339,14 +339,14 @@ Functions [1]: [sum(total_sales#36)] Input [2]: [i_manufact_id#12, total_sales#41] Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#41 ASC NULLS FIRST], output=[i_manufact_id#12,total_sales#41]), [i_manufact_id#12, total_sales#41], 100, [total_sales#41 ASC NULLS FIRST], [i_manufact_id#12, total_sales#41] -(61) ColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [codegen id : 1] Input [2]: [i_manufact_id#12, total_sales#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (66) -+- * ColumnarToRow (65) ++- * CometColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometScan parquet spark_catalog.default.date_dim (62) @@ -367,7 +367,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (66) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/simplified.txt index 9556aa785b..50c2d64a92 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_manufact_id,total_sales] CometHashAggregate [i_manufact_id,total_sales,sum,isEmpty,sum(total_sales)] @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/explain.txt index 50e3800526..452f25394a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (32) +* CometColumnarToRow (32) +- CometSort (31) +- CometColumnarExchange (30) +- CometProject (29) @@ -179,14 +179,14 @@ Arguments: rangepartitioning(c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] Arguments: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (37) -+- * ColumnarToRow (36) ++- * CometColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometScan parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_do Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt index 779d4cbddb..014786e7fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/explain.txt index c1e19555c9..0730e11302 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/explain.txt @@ -11,7 +11,7 @@ TakeOrderedAndProject (44) : : +- * Filter (27) : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * ColumnarToRow (12) + : : : :- * CometColumnarToRow (12) : : : : +- CometBroadcastHashJoin (11) : : : : :- CometFilter (2) : : : : : +- CometScan parquet spark_catalog.default.customer (1) @@ -24,23 +24,23 @@ TakeOrderedAndProject (44) : : : : +- CometFilter (5) : : : : +- CometScan parquet spark_catalog.default.date_dim (4) : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) + : : : +- * CometColumnarToRow (17) : : : +- CometProject (16) : : : +- CometBroadcastHashJoin (15) : : : :- CometScan parquet spark_catalog.default.web_sales (13) : : : +- ReusedExchange (14) : : +- BroadcastExchange (25) - : : +- * ColumnarToRow (24) + : : +- * CometColumnarToRow (24) : : +- CometProject (23) : : +- CometBroadcastHashJoin (22) : : :- CometScan parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (32) - : +- * ColumnarToRow (31) + : +- * CometColumnarToRow (31) : +- CometFilter (30) : +- CometScan parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (38) - +- * ColumnarToRow (37) + +- * CometColumnarToRow (37) +- CometFilter (36) +- CometScan parquet spark_catalog.default.customer_demographics (35) @@ -100,7 +100,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) ColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 5] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan parquet spark_catalog.default.web_sales @@ -122,7 +122,7 @@ Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#12] (18) BroadcastExchange @@ -154,7 +154,7 @@ Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [1]: [cs_ship_customer_sk#16] (25) BroadcastExchange @@ -186,7 +186,7 @@ ReadSchema: struct Input [2]: [ca_address_sk#20, ca_state#21] Condition : isnotnull(ca_address_sk#20) -(31) ColumnarToRow [codegen id : 3] +(31) CometColumnarToRow [codegen id : 3] Input [2]: [ca_address_sk#20, ca_state#21] (32) BroadcastExchange @@ -214,7 +214,7 @@ ReadSchema: struct= 2000-02-01)) AND (d_date#1 Input [2]: [d_date_sk#10, d_date#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt index 85e86ca8ae..13fcba8592 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] CometHashAggregate [i_item_id,i_item_desc,i_current_price] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/explain.txt index c0e7300dff..8139d06c08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (47) +* CometColumnarToRow (47) +- CometHashAggregate (46) +- CometExchange (45) +- CometHashAggregate (44) @@ -261,14 +261,14 @@ Input [1]: [count#26] Keys: [] Functions [1]: [count(1)] -(47) ColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [1]: [count(1)#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 BroadcastExchange (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/simplified.txt index 8272260f5c..5ce896e6e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [count(1),count,count(1)] CometExchange #1 @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/explain.txt index 32b231d65d..0b404d5868 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (47) +* CometColumnarToRow (47) +- CometSort (46) +- CometColumnarExchange (45) +- CometBroadcastHashJoin (44) @@ -261,14 +261,14 @@ Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC N Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST] -(47) ColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_moy#11] (52) BroadcastExchange @@ -298,7 +298,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 BroadcastExchange (57) -+- * ColumnarToRow (56) ++- * CometColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan parquet spark_catalog.default.date_dim (53) @@ -319,7 +319,7 @@ Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2 Input [3]: [d_date_sk#28, d_year#29, d_moy#30] Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] -(56) ColumnarToRow [codegen id : 1] +(56) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#28, d_moy#30] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt index b789f3a8d2..9df61b1669 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] @@ -52,7 +52,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/explain.txt index 0239d76784..683ec549cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (47) +* CometColumnarToRow (47) +- CometSort (46) +- CometColumnarExchange (45) +- CometBroadcastHashJoin (44) @@ -261,14 +261,14 @@ Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC N Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST] -(47) ColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_moy#11] (52) BroadcastExchange @@ -298,7 +298,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 BroadcastExchange (57) -+- * ColumnarToRow (56) ++- * CometColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan parquet spark_catalog.default.date_dim (53) @@ -319,7 +319,7 @@ Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2 Input [3]: [d_date_sk#28, d_year#29, d_moy#30] Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] -(56) ColumnarToRow [codegen id : 1] +(56) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#28, d_moy#30] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt index b789f3a8d2..9df61b1669 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] @@ -52,7 +52,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/explain.txt index 0fe37749f9..457cfd8a39 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (101) +* CometColumnarToRow (101) +- CometTakeOrderedAndProject (100) +- CometProject (99) +- CometBroadcastHashJoin (98) @@ -575,14 +575,14 @@ Arguments: [customer_id#41, customer_first_name#42, customer_last_name#43, custo Input [7]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47] Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#41 ASC NULLS FIRST,customer_first_name#42 ASC NULLS FIRST,customer_last_name#43 ASC NULLS FIRST,customer_preferred_cust_flag#44 ASC NULLS FIRST,customer_birth_country#45 ASC NULLS FIRST,customer_login#46 ASC NULLS FIRST,customer_email_address#47 ASC NULLS FIRST], output=[customer_id#41,customer_first_name#42,customer_last_name#43,customer_preferred_cust_flag#44,customer_birth_country#45,customer_login#46,customer_email_address#47]), [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47], 100, [customer_id#41 ASC NULLS FIRST, customer_first_name#42 ASC NULLS FIRST, customer_last_name#43 ASC NULLS FIRST, customer_preferred_cust_flag#44 ASC NULLS FIRST, customer_birth_country#45 ASC NULLS FIRST, customer_login#46 ASC NULLS FIRST, customer_email_address#47 ASC NULLS FIRST], [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47] -(101) ColumnarToRow [codegen id : 1] +(101) CometColumnarToRow [codegen id : 1] Input [7]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#15 BroadcastExchange (105) -+- * ColumnarToRow (104) ++- * CometColumnarToRow (104) +- CometFilter (103) +- CometScan parquet spark_catalog.default.date_dim (102) @@ -598,7 +598,7 @@ ReadSchema: struct Input [2]: [d_date_sk#16, d_year#17] Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) -(104) ColumnarToRow [codegen id : 1] +(104) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_year#17] (105) BroadcastExchange @@ -607,7 +607,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 BroadcastExchange (109) -+- * ColumnarToRow (108) ++- * CometColumnarToRow (108) +- CometFilter (107) +- CometScan parquet spark_catalog.default.date_dim (106) @@ -623,7 +623,7 @@ ReadSchema: struct Input [2]: [d_date_sk#37, d_year#38] Condition : ((isnotnull(d_year#38) AND (d_year#38 = 2002)) AND isnotnull(d_date_sk#37)) -(108) ColumnarToRow [codegen id : 1] +(108) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#37, d_year#38] (109) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/simplified.txt index d6651fadcf..19f7ccde3a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] CometProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] @@ -27,7 +27,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -50,7 +50,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/explain.txt index ef66f763d2..10640eb2bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (32) +* CometColumnarToRow (32) +- CometTakeOrderedAndProject (31) +- CometHashAggregate (30) +- CometExchange (29) @@ -179,14 +179,14 @@ Functions [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - Input [4]: [w_state#12, i_item_id#14, sales_before#22, sales_after#23] Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_state#12 ASC NULLS FIRST,i_item_id#14 ASC NULLS FIRST], output=[w_state#12,i_item_id#14,sales_before#22,sales_after#23]), [w_state#12, i_item_id#14, sales_before#22, sales_after#23], 100, [w_state#12 ASC NULLS FIRST, i_item_id#14 ASC NULLS FIRST], [w_state#12, i_item_id#14, sales_before#22, sales_after#23] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [4]: [w_state#12, i_item_id#14, sales_before#22, sales_after#23] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (36) -+- * ColumnarToRow (35) ++- * CometColumnarToRow (35) +- CometFilter (34) +- CometScan parquet spark_catalog.default.date_dim (33) @@ -202,7 +202,7 @@ ReadSchema: struct Input [2]: [d_date_sk#16, d_date#17] Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2000-02-10)) AND (d_date#17 <= 2000-04-10)) AND isnotnull(d_date_sk#16)) -(35) ColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_date#17] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/simplified.txt index eac6138196..6b87d9b8b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] CometHashAggregate [w_state,i_item_id,sales_before,sales_after,sum,isEmpty,sum,isEmpty,sum(CASE WHEN (d_date < 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END)] @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/explain.txt index 48a1c60d16..bb3c74daeb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -103,6 +103,6 @@ Functions: [] Input [1]: [i_product_name#3] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#3 ASC NULLS FIRST], output=[i_product_name#3]), [i_product_name#3], 100, [i_product_name#3 ASC NULLS FIRST], [i_product_name#3] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [1]: [i_product_name#3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt index 007103499b..de12ce9581 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_product_name] CometHashAggregate [i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/explain.txt index b58d82c785..0ad962d86a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Input [4]: [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#12] Arguments: TakeOrderedAndProject(limit=100, orderBy=[sum(ss_ext_sales_price)#12 DESC NULLS LAST,d_year#2 ASC NULLS FIRST,i_category_id#8 ASC NULLS FIRST,i_category#9 ASC NULLS FIRST], output=[d_year#2,i_category_id#8,i_category#9,sum(ss_ext_sales_price)#12]), [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#12], 100, [sum(ss_ext_sales_price)#12 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#9 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#12] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [4]: [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#12] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/simplified.txt index 40528f52d3..7307450677 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [d_year,i_category_id,i_category,sum(ss_ext_sales_price)] CometHashAggregate [d_year,i_category_id,i_category,sum(ss_ext_sales_price),sum,sum(UnscaledValue(ss_ext_sales_price))] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/explain.txt index 2953a5f8f9..10dfceddc2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_s Input [9]: [s_store_name#9, s_store_id#8, sun_sales#18, mon_sales#19, tue_sales#20, wed_sales#21, thu_sales#22, fri_sales#23, sat_sales#24] Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#9 ASC NULLS FIRST,s_store_id#8 ASC NULLS FIRST,sun_sales#18 ASC NULLS FIRST,mon_sales#19 ASC NULLS FIRST,tue_sales#20 ASC NULLS FIRST,wed_sales#21 ASC NULLS FIRST,thu_sales#22 ASC NULLS FIRST,fri_sales#23 ASC NULLS FIRST,sat_sales#24 ASC NULLS FIRST], output=[s_store_name#9,s_store_id#8,sun_sales#18,mon_sales#19,tue_sales#20,wed_sales#21,thu_sales#22,fri_sales#23,sat_sales#24]), [s_store_name#9, s_store_id#8, sun_sales#18, mon_sales#19, tue_sales#20, wed_sales#21, thu_sales#22, fri_sales#23, sat_sales#24], 100, [s_store_name#9 ASC NULLS FIRST, s_store_id#8 ASC NULLS FIRST, sun_sales#18 ASC NULLS FIRST, mon_sales#19 ASC NULLS FIRST, tue_sales#20 ASC NULLS FIRST, wed_sales#21 ASC NULLS FIRST, thu_sales#22 ASC NULLS FIRST, fri_sales#23 ASC NULLS FIRST, sat_sales#24 ASC NULLS FIRST], [s_store_name#9, s_store_id#8, sun_sales#18, mon_sales#19, tue_sales#20, wed_sales#21, thu_sales#22, fri_sales#23, sat_sales#24] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [9]: [s_store_name#9, s_store_id#8, sun_sales#18, mon_sales#19, tue_sales#20, wed_sales#21, thu_sales#22, fri_sales#23, sat_sales#24] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/simplified.txt index 447131b776..08b394e62e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] CometHashAggregate [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/explain.txt index 337a09591b..c58b239c51 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/explain.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject (42) : : : +- * Sort (12) : : : +- Exchange (11) : : : +- WindowGroupLimit (10) - : : : +- * ColumnarToRow (9) + : : : +- * CometColumnarToRow (9) : : : +- CometSort (8) : : : +- CometFilter (7) : : : +- CometHashAggregate (6) @@ -31,13 +31,13 @@ TakeOrderedAndProject (42) : : +- * Sort (25) : : +- Exchange (24) : : +- WindowGroupLimit (23) - : : +- * ColumnarToRow (22) + : : +- * CometColumnarToRow (22) : : +- CometSort (21) : : +- CometFilter (20) : : +- CometHashAggregate (19) : : +- ReusedExchange (18) : +- BroadcastExchange (36) - : +- * ColumnarToRow (35) + : +- * CometColumnarToRow (35) : +- CometFilter (34) : +- CometScan parquet spark_catalog.default.item (33) +- ReusedExchange (39) @@ -80,7 +80,7 @@ Condition : (isnotnull(rank_col#8) AND (cast(rank_col#8 as decimal(13,7)) > (0.9 Input [2]: [item_sk#7, rank_col#8] Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] -(9) ColumnarToRow [codegen id : 1] +(9) CometColumnarToRow [codegen id : 1] Input [2]: [item_sk#7, rank_col#8] (10) WindowGroupLimit @@ -131,7 +131,7 @@ Condition : (isnotnull(rank_col#17) AND (cast(rank_col#17 as decimal(13,7)) > (0 Input [2]: [item_sk#16, rank_col#17] Arguments: [item_sk#16, rank_col#17], [rank_col#17 DESC NULLS LAST] -(22) ColumnarToRow [codegen id : 4] +(22) CometColumnarToRow [codegen id : 4] Input [2]: [item_sk#16, rank_col#17] (23) WindowGroupLimit @@ -187,7 +187,7 @@ ReadSchema: struct Input [2]: [i_item_sk#19, i_product_name#20] Condition : isnotnull(i_item_sk#19) -(35) ColumnarToRow [codegen id : 7] +(35) CometColumnarToRow [codegen id : 7] Input [2]: [i_item_sk#19, i_product_name#20] (36) BroadcastExchange @@ -224,7 +224,7 @@ Arguments: 100, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#23, worst_per ===== Subqueries ===== Subquery:1 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* ColumnarToRow (49) +* CometColumnarToRow (49) +- CometHashAggregate (48) +- CometExchange (47) +- CometHashAggregate (46) @@ -262,7 +262,7 @@ Input [3]: [ss_store_sk#26, sum#29, count#30] Keys [1]: [ss_store_sk#26] Functions [1]: [avg(UnscaledValue(ss_net_profit#27))] -(49) ColumnarToRow [codegen id : 1] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [rank_col#31] Subquery:2 Hosting operator id = 20 Hosting Expression = ReusedSubquery Subquery scalar-subquery#9, [id=#10] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt index 58687dc84b..c6901b897c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt @@ -20,13 +20,13 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] Exchange #1 WindowGroupLimit [rank_col] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item_sk,rank_col] CometFilter [item_sk,rank_col] Subquery #1 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [rank_col,ss_store_sk,sum,count,avg(UnscaledValue(ss_net_profit))] CometExchange [ss_store_sk] #3 @@ -54,7 +54,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] Exchange #4 WindowGroupLimit [rank_col] WholeStageCodegen (4) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item_sk,rank_col] CometFilter [item_sk,rank_col] @@ -64,7 +64,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] InputAdapter BroadcastExchange #5 WholeStageCodegen (7) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [i_item_sk,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/explain.txt index 962a51203b..d9213e07f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/explain.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject (36) +- * Project (32) +- * Filter (31) +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (30) - :- * ColumnarToRow (24) + :- * CometColumnarToRow (24) : +- CometProject (23) : +- CometBroadcastHashJoin (22) : :- CometProject (18) @@ -31,7 +31,7 @@ TakeOrderedAndProject (36) : +- CometFilter (20) : +- CometScan parquet spark_catalog.default.item (19) +- BroadcastExchange (29) - +- * ColumnarToRow (28) + +- * CometColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan parquet spark_catalog.default.item (25) @@ -149,7 +149,7 @@ Arguments: [ws_item_sk#2], [i_item_sk#15], Inner, BuildRight Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#11, i_item_sk#15, i_item_id#16] Arguments: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16], [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [4]: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16] (25) CometScan parquet spark_catalog.default.item @@ -167,7 +167,7 @@ Condition : i_item_sk#17 IN (2,3,5,7,11,13,17,19,23,29) Input [2]: [i_item_sk#17, i_item_id#18] Arguments: [i_item_id#18], [i_item_id#18] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [i_item_id#18] (29) BroadcastExchange @@ -214,7 +214,7 @@ Arguments: 100, [ca_zip#11 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (41) -+- * ColumnarToRow (40) ++- * CometColumnarToRow (40) +- CometProject (39) +- CometFilter (38) +- CometScan parquet spark_catalog.default.date_dim (37) @@ -235,7 +235,7 @@ Condition : ((((isnotnull(d_qoy#14) AND isnotnull(d_year#13)) AND (d_qoy#14 = 2) Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] Arguments: [d_date_sk#12], [d_date_sk#12] -(40) ColumnarToRow [codegen id : 1] +(40) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#12] (41) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt index f60fdb18a1..a325fb95d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] Project [ws_sales_price,ca_city,ca_zip] Filter [ca_zip,exists] BroadcastHashJoin [i_item_id,i_item_id] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ca_city,ca_zip,i_item_sk,i_item_id] @@ -23,7 +23,7 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] @@ -44,7 +44,7 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] InputAdapter BroadcastExchange #7 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [i_item_id] CometFilter [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/explain.txt index 7bc10ce88e..406d3a6730 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (38) +* CometColumnarToRow (38) +- CometTakeOrderedAndProject (37) +- CometProject (36) +- CometBroadcastHashJoin (35) @@ -213,14 +213,14 @@ Arguments: [c_last_name#25, c_first_name#24, ca_city#30, bought_city#26, ss_tick Input [7]: [c_last_name#25, c_first_name#24, ca_city#30, bought_city#26, ss_ticket_number#5, amt#27, profit#28] Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#25 ASC NULLS FIRST,c_first_name#24 ASC NULLS FIRST,ca_city#30 ASC NULLS FIRST,bought_city#26 ASC NULLS FIRST,ss_ticket_number#5 ASC NULLS FIRST], output=[c_last_name#25,c_first_name#24,ca_city#30,bought_city#26,ss_ticket_number#5,amt#27,profit#28]), [c_last_name#25, c_first_name#24, ca_city#30, bought_city#26, ss_ticket_number#5, amt#27, profit#28], 100, [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, ca_city#30 ASC NULLS FIRST, bought_city#26 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#25, c_first_name#24, ca_city#30, bought_city#26, ss_ticket_number#5, amt#27, profit#28] -(38) ColumnarToRow [codegen id : 1] +(38) CometColumnarToRow [codegen id : 1] Input [7]: [c_last_name#25, c_first_name#24, ca_city#30, bought_city#26, ss_ticket_number#5, amt#27, profit#28] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (43) -+- * ColumnarToRow (42) ++- * CometColumnarToRow (42) +- CometProject (41) +- CometFilter (40) +- CometScan parquet spark_catalog.default.date_dim (39) @@ -241,7 +241,7 @@ Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (43) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/simplified.txt index e1b53bf4d0..25e38a548e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] CometProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dow] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/explain.txt index 32126320ff..e39a63cc69 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/explain.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject (47) : : +- Window (26) : : +- * Filter (25) : : +- Window (24) - : : +- * ColumnarToRow (23) + : : +- * CometColumnarToRow (23) : : +- CometSort (22) : : +- CometExchange (21) : : +- CometHashAggregate (20) @@ -35,7 +35,7 @@ TakeOrderedAndProject (47) : +- BroadcastExchange (36) : +- * Project (35) : +- Window (34) - : +- * ColumnarToRow (33) + : +- * CometColumnarToRow (33) : +- CometSort (32) : +- CometExchange (31) : +- CometHashAggregate (30) @@ -43,7 +43,7 @@ TakeOrderedAndProject (47) +- BroadcastExchange (44) +- * Project (43) +- Window (42) - +- * ColumnarToRow (41) + +- * CometColumnarToRow (41) +- CometSort (40) +- ReusedExchange (39) @@ -154,7 +154,7 @@ Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_ Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17] Arguments: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17], [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarToRow [codegen id : 1] Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17] (24) Window @@ -193,7 +193,7 @@ Arguments: hashpartitioning(i_category#20, i_brand#21, s_store_name#22, s_compan Input [7]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16] Arguments: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16], [i_category#20 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(33) ColumnarToRow [codegen id : 3] +(33) CometColumnarToRow [codegen id : 3] Input [7]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16] (34) Window @@ -225,7 +225,7 @@ Output [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_ye Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#16] Arguments: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#16], [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] -(41) ColumnarToRow [codegen id : 5] +(41) CometColumnarToRow [codegen id : 5] Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#16] (42) Window @@ -258,7 +258,7 @@ Arguments: 100, [(sum_sales#16 - avg_monthly_sales#19) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (51) -+- * ColumnarToRow (50) ++- * CometColumnarToRow (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -274,7 +274,7 @@ ReadSchema: struct Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (51) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/simplified.txt index cf4556774d..26c6cd226a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/simplified.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 @@ -34,7 +34,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] @@ -51,7 +51,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 @@ -64,7 +64,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/explain.txt index 70c7dc75f5..451f4d9646 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (28) +* CometColumnarToRow (28) +- CometHashAggregate (27) +- CometExchange (26) +- CometHashAggregate (25) @@ -159,14 +159,14 @@ Input [1]: [sum#18] Keys: [] Functions [1]: [sum(ss_quantity#4)] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [sum(ss_quantity)#19] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (33) -+- * ColumnarToRow (32) ++- * CometColumnarToRow (32) +- CometProject (31) +- CometFilter (30) +- CometScan parquet spark_catalog.default.date_dim (29) @@ -187,7 +187,7 @@ Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#16, d_year#17] Arguments: [d_date_sk#16], [d_date_sk#16] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] (33) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/simplified.txt index a46d8d6037..60c611beca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [sum(ss_quantity),sum,sum(ss_quantity)] CometExchange #1 @@ -17,7 +17,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/explain.txt index 2def4544c3..c1371c0e8a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/explain.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject (77) : +- Window (24) : +- * Sort (23) : +- Window (22) - : +- * ColumnarToRow (21) + : +- * CometColumnarToRow (21) : +- CometSort (20) : +- CometExchange (19) : +- CometHashAggregate (18) @@ -35,7 +35,7 @@ TakeOrderedAndProject (77) : +- Window (47) : +- * Sort (46) : +- Window (45) - : +- * ColumnarToRow (44) + : +- * CometColumnarToRow (44) : +- CometSort (43) : +- CometExchange (42) : +- CometHashAggregate (41) @@ -58,7 +58,7 @@ TakeOrderedAndProject (77) +- Window (70) +- * Sort (69) +- Window (68) - +- * ColumnarToRow (67) + +- * CometColumnarToRow (67) +- CometSort (66) +- CometExchange (65) +- CometHashAggregate (64) @@ -172,7 +172,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [3]: [item#22, return_ratio#23, currency_ratio#24] Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) ColumnarToRow [codegen id : 1] +(21) CometColumnarToRow [codegen id : 1] Input [3]: [item#22, return_ratio#23, currency_ratio#24] (22) Window @@ -273,7 +273,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] -(44) ColumnarToRow [codegen id : 4] +(44) CometColumnarToRow [codegen id : 4] Input [3]: [item#47, return_ratio#48, currency_ratio#49] (45) Window @@ -374,7 +374,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] -(67) ColumnarToRow [codegen id : 7] +(67) CometColumnarToRow [codegen id : 7] Input [3]: [item#72, return_ratio#73, currency_ratio#74] (68) Window @@ -425,7 +425,7 @@ Arguments: 100, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, cur Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (82) -+- * ColumnarToRow (81) ++- * CometColumnarToRow (81) +- CometProject (80) +- CometFilter (79) +- CometScan parquet spark_catalog.default.date_dim (78) @@ -446,7 +446,7 @@ Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2 Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(81) ColumnarToRow [codegen id : 1] +(81) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] (82) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/simplified.txt index d98b2b0a66..aaf75c375c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/simplified.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] InputAdapter Window [return_ratio] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item,return_ratio,currency_ratio] CometExchange #2 @@ -35,7 +35,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -57,7 +57,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] InputAdapter Window [return_ratio] WholeStageCodegen (4) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item,return_ratio,currency_ratio] CometExchange #7 @@ -87,7 +87,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] InputAdapter Window [return_ratio] WholeStageCodegen (7) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item,return_ratio,currency_ratio] CometExchange #10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/explain.txt index 02ad2f3575..1b4ffcc441 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/explain.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject (70) +- * Expand (66) +- Union (65) :- * HashAggregate (22) - : +- * ColumnarToRow (21) + : +- * CometColumnarToRow (21) : +- CometExchange (20) : +- CometHashAggregate (19) : +- CometProject (18) @@ -28,7 +28,7 @@ TakeOrderedAndProject (70) : +- CometFilter (15) : +- CometScan parquet spark_catalog.default.store (14) :- * HashAggregate (41) - : +- * ColumnarToRow (40) + : +- * CometColumnarToRow (40) : +- CometExchange (39) : +- CometHashAggregate (38) : +- CometProject (37) @@ -47,7 +47,7 @@ TakeOrderedAndProject (70) : +- CometFilter (34) : +- CometScan parquet spark_catalog.default.catalog_page (33) +- * HashAggregate (64) - +- * ColumnarToRow (63) + +- * CometColumnarToRow (63) +- CometExchange (62) +- CometHashAggregate (61) +- CometProject (60) @@ -168,7 +168,7 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(21) ColumnarToRow [codegen id : 1] +(21) CometColumnarToRow [codegen id : 1] Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] (22) HashAggregate [codegen id : 1] @@ -259,7 +259,7 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#46)), partial_sum(Unscaled Input [5]: [cp_catalog_page_id#62, sum#63, sum#64, sum#65, sum#66] Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(40) ColumnarToRow [codegen id : 2] +(40) CometColumnarToRow [codegen id : 2] Input [5]: [cp_catalog_page_id#62, sum#63, sum#64, sum#65, sum#66] (41) HashAggregate [codegen id : 2] @@ -369,7 +369,7 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#83)), partial_sum(Unscaled Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(63) ColumnarToRow [codegen id : 3] +(63) CometColumnarToRow [codegen id : 3] Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] (64) HashAggregate [codegen id : 3] @@ -411,7 +411,7 @@ Arguments: 100, [channel#118 ASC NULLS FIRST, id#119 ASC NULLS FIRST], [channel# Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (75) -+- * ColumnarToRow (74) ++- * CometColumnarToRow (74) +- CometProject (73) +- CometFilter (72) +- CometScan parquet spark_catalog.default.date_dim (71) @@ -432,7 +432,7 @@ Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#2 Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(74) ColumnarToRow [codegen id : 1] +(74) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] (75) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt index c6f9f60af4..30bdf3edea 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Union WholeStageCodegen (1) HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [s_store_id] #2 CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] @@ -25,7 +25,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] @@ -43,7 +43,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (2) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [cp_catalog_page_id] #6 CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] @@ -66,7 +66,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (3) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [web_site_id] #8 CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/explain.txt index 9b1efc7ada..ded5068bb0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (28) +* CometColumnarToRow (28) +- CometTakeOrderedAndProject (27) +- CometHashAggregate (26) +- CometExchange (25) @@ -160,14 +160,14 @@ Functions [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30 Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#12 ASC NULLS FIRST,s_company_id#13 ASC NULLS FIRST,s_street_number#14 ASC NULLS FIRST,s_street_name#15 ASC NULLS FIRST,s_street_type#16 ASC NULLS FIRST,s_suite_number#17 ASC NULLS FIRST,s_city#18 ASC NULLS FIRST,s_county#19 ASC NULLS FIRST,s_state#20 ASC NULLS FIRST,s_zip#21 ASC NULLS FIRST], output=[s_store_name#12,s_company_id#13,s_street_number#14,s_street_name#15,s_street_type#16,s_suite_number#17,s_city#18,s_county#19,s_state#20,s_zip#21,30 days #31,31 - 60 days #32,61 - 90 days #33,91 - 120 days #34,>120 days #35]), [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35], 100, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRST, s_street_number#14 ASC NULLS FIRST, s_street_name#15 ASC NULLS FIRST, s_street_type#16 ASC NULLS FIRST, s_suite_number#17 ASC NULLS FIRST, s_city#18 ASC NULLS FIRST, s_county#19 ASC NULLS FIRST, s_state#20 ASC NULLS FIRST, s_zip#21 ASC NULLS FIRST], [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 BroadcastExchange (33) -+- * ColumnarToRow (32) ++- * CometColumnarToRow (32) +- CometProject (31) +- CometFilter (30) +- CometScan parquet spark_catalog.default.date_dim (29) @@ -188,7 +188,7 @@ Condition : ((((isnotnull(d_year#24) AND isnotnull(d_moy#25)) AND (d_year#24 = 2 Input [3]: [d_date_sk#23, d_year#24, d_moy#25] Arguments: [d_date_sk#23], [d_date_sk#23] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#23] (33) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/simplified.txt index 5333344640..c0a0b6d39c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum,sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 30) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 60) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 90) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) > 120) THEN 1 ELSE 0 END)] @@ -21,7 +21,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/explain.txt index 8779bd70e7..36d0ef2338 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/explain.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject (40) : +- Exchange (17) : +- * Project (16) : +- Window (15) - : +- * ColumnarToRow (14) + : +- * CometColumnarToRow (14) : +- CometSort (13) : +- CometExchange (12) : +- CometHashAggregate (11) @@ -28,7 +28,7 @@ TakeOrderedAndProject (40) +- Exchange (32) +- * Project (31) +- Window (30) - +- * ColumnarToRow (29) + +- * CometColumnarToRow (29) +- CometSort (28) +- CometExchange (27) +- CometHashAggregate (26) @@ -103,7 +103,7 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeSh Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] Arguments: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] (15) Window @@ -168,7 +168,7 @@ Arguments: hashpartitioning(ss_item_sk#12, 5), ENSURE_REQUIREMENTS, CometNativeS Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] Arguments: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12], [ss_item_sk#12 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] -(29) ColumnarToRow [codegen id : 4] +(29) CometColumnarToRow [codegen id : 4] Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] (30) Window @@ -221,7 +221,7 @@ Arguments: 100, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_s Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (45) -+- * ColumnarToRow (44) ++- * CometColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan parquet spark_catalog.default.date_dim (41) @@ -242,7 +242,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_mont Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(44) ColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/simplified.txt index f972c31c17..bddc3a125a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/simplified.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store InputAdapter Window [_w0,ws_item_sk,d_date] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item_sk,d_date,_w0,ws_item_sk] CometExchange [ws_item_sk] #3 @@ -34,7 +34,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] @@ -53,7 +53,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store InputAdapter Window [_w0,ss_item_sk,d_date] WholeStageCodegen (4) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item_sk,d_date,_w0,ss_item_sk] CometExchange [ss_item_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/explain.txt index 42b974e53f..c7a400159f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Input [4]: [d_year#2, brand_id#12, brand#13, ext_price#14] Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,ext_price#14 DESC NULLS LAST,brand_id#12 ASC NULLS FIRST], output=[d_year#2,brand_id#12,brand#13,ext_price#14]), [d_year#2, brand_id#12, brand#13, ext_price#14], 100, [d_year#2 ASC NULLS FIRST, ext_price#14 DESC NULLS LAST, brand_id#12 ASC NULLS FIRST], [d_year#2, brand_id#12, brand#13, ext_price#14] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [4]: [d_year#2, brand_id#12, brand#13, ext_price#14] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/simplified.txt index af5223b69e..1e7168862a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [d_year,brand_id,brand,ext_price] CometHashAggregate [d_year,brand_id,brand,ext_price,i_brand,i_brand_id,sum,sum(UnscaledValue(ss_ext_sales_price))] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/explain.txt index 9b71fa4009..c1448cf812 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (29) +- * Project (28) +- * Filter (27) +- Window (26) - +- * ColumnarToRow (25) + +- * CometColumnarToRow (25) +- CometSort (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -144,7 +144,7 @@ Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometNativ Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] Arguments: [i_manufact_id#5, sum_sales#20, _w0#21], [i_manufact_id#5 ASC NULLS FIRST] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] (26) Window @@ -167,7 +167,7 @@ Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (34) -+- * ColumnarToRow (33) ++- * CometColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] -(33) ColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#15, d_qoy#17] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/simplified.txt index 363b84b354..96a2eec50e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] InputAdapter Window [_w0,i_manufact_id] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_manufact_id,sum_sales,_w0] CometExchange [i_manufact_id] #1 @@ -27,7 +27,7 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_qoy] CometFilter [d_date_sk,d_month_seq,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/explain.txt index 73422b2923..72ba214250 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (56) +* CometColumnarToRow (56) +- CometTakeOrderedAndProject (55) +- CometHashAggregate (54) +- CometExchange (53) @@ -320,14 +320,14 @@ Functions [1]: [count(1)] Input [3]: [segment#38, num_customers#40, segment_base#41] Arguments: TakeOrderedAndProject(limit=100, orderBy=[segment#38 ASC NULLS FIRST,num_customers#40 ASC NULLS FIRST], output=[segment#38,num_customers#40,segment_base#41]), [segment#38, num_customers#40, segment_base#41], 100, [segment#38 ASC NULLS FIRST, num_customers#40 ASC NULLS FIRST], [segment#38, num_customers#40, segment_base#41] -(56) ColumnarToRow [codegen id : 1] +(56) CometColumnarToRow [codegen id : 1] Input [3]: [segment#38, num_customers#40, segment_base#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (61) -+- * ColumnarToRow (60) ++- * CometColumnarToRow (60) +- CometProject (59) +- CometFilter (58) +- CometScan parquet spark_catalog.default.date_dim (57) @@ -348,7 +348,7 @@ Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12 Input [3]: [d_date_sk#17, d_year#18, d_moy#19] Arguments: [d_date_sk#17], [d_date_sk#17] -(60) ColumnarToRow [codegen id : 1] +(60) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#17] (61) BroadcastExchange @@ -359,7 +359,7 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dy Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 BroadcastExchange (66) -+- * ColumnarToRow (65) ++- * CometColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometScan parquet spark_catalog.default.date_dim (62) @@ -380,7 +380,7 @@ Condition : (((isnotnull(d_month_seq#32) AND (d_month_seq#32 >= Subquery scalar- Input [2]: [d_date_sk#31, d_month_seq#32] Arguments: [d_date_sk#31], [d_date_sk#31] -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#31] (66) BroadcastExchange @@ -388,7 +388,7 @@ Input [1]: [d_date_sk#31] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:4 Hosting operator id = 63 Hosting Expression = Subquery scalar-subquery#33, [id=#34] -* ColumnarToRow (73) +* CometColumnarToRow (73) +- CometHashAggregate (72) +- CometExchange (71) +- CometHashAggregate (70) @@ -426,11 +426,11 @@ Input [1]: [(d_month_seq + 1)#45] Keys [1]: [(d_month_seq + 1)#45] Functions: [] -(73) ColumnarToRow [codegen id : 1] +(73) CometColumnarToRow [codegen id : 1] Input [1]: [(d_month_seq + 1)#45] Subquery:5 Hosting operator id = 63 Hosting Expression = Subquery scalar-subquery#35, [id=#36] -* ColumnarToRow (80) +* CometColumnarToRow (80) +- CometHashAggregate (79) +- CometExchange (78) +- CometHashAggregate (77) @@ -468,7 +468,7 @@ Input [1]: [(d_month_seq + 3)#49] Keys [1]: [(d_month_seq + 3)#49] Functions: [] -(80) ColumnarToRow [codegen id : 1] +(80) CometColumnarToRow [codegen id : 1] Input [1]: [(d_month_seq + 3)#49] Subquery:6 Hosting operator id = 44 Hosting Expression = ReusedSubquery Subquery scalar-subquery#33, [id=#34] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/simplified.txt index 1290186685..e41bbe85b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [segment,num_customers,segment_base] CometHashAggregate [segment,num_customers,segment_base,count,count(1)] @@ -32,7 +32,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -58,13 +58,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] Subquery #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [(d_month_seq + 1)] CometExchange [(d_month_seq + 1)] #10 @@ -74,7 +74,7 @@ WholeStageCodegen (1) CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] Subquery #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [(d_month_seq + 3)] CometExchange [(d_month_seq + 3)] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/explain.txt index 46240a3c02..4549080d86 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Input [3]: [brand_id#12, brand#13, ext_price#14] Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#14 DESC NULLS LAST,brand_id#12 ASC NULLS FIRST], output=[brand_id#12,brand#13,ext_price#14]), [brand_id#12, brand#13, ext_price#14], 100, [ext_price#14 DESC NULLS LAST, brand_id#12 ASC NULLS FIRST], [brand_id#12, brand#13, ext_price#14] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [3]: [brand_id#12, brand#13, ext_price#14] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/simplified.txt index 999c8a6c4c..5a5f7ee21e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [brand_id,brand,ext_price] CometHashAggregate [brand_id,brand,ext_price,i_brand,i_brand_id,sum,sum(UnscaledValue(ss_ext_sales_price))] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/explain.txt index c56eb8287f..e3977ede3f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (61) +* CometColumnarToRow (61) +- CometTakeOrderedAndProject (60) +- CometHashAggregate (59) +- CometExchange (58) @@ -339,14 +339,14 @@ Functions [1]: [sum(total_sales#36)] Input [2]: [i_item_id#12, total_sales#41] Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#41 ASC NULLS FIRST], output=[i_item_id#12,total_sales#41]), [i_item_id#12, total_sales#41], 100, [total_sales#41 ASC NULLS FIRST], [i_item_id#12, total_sales#41] -(61) ColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [codegen id : 1] Input [2]: [i_item_id#12, total_sales#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (66) -+- * ColumnarToRow (65) ++- * CometColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometScan parquet spark_catalog.default.date_dim (62) @@ -367,7 +367,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (66) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/simplified.txt index 2603a2781f..95ff3d709a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,total_sales] CometHashAggregate [i_item_id,total_sales,sum,isEmpty,sum(total_sales)] @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/explain.txt index 78df07e8fb..8f45480fc8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/explain.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject (47) : : +- Window (26) : : +- * Filter (25) : : +- Window (24) - : : +- * ColumnarToRow (23) + : : +- * CometColumnarToRow (23) : : +- CometSort (22) : : +- CometExchange (21) : : +- CometHashAggregate (20) @@ -35,7 +35,7 @@ TakeOrderedAndProject (47) : +- BroadcastExchange (36) : +- * Project (35) : +- Window (34) - : +- * ColumnarToRow (33) + : +- * CometColumnarToRow (33) : +- CometSort (32) : +- CometExchange (31) : +- CometHashAggregate (30) @@ -43,7 +43,7 @@ TakeOrderedAndProject (47) +- BroadcastExchange (44) +- * Project (43) +- Window (42) - +- * ColumnarToRow (41) + +- * CometColumnarToRow (41) +- CometSort (40) +- ReusedExchange (39) @@ -154,7 +154,7 @@ Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQU Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16] Arguments: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16], [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarToRow [codegen id : 1] Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16] (24) Window @@ -193,7 +193,7 @@ Arguments: hashpartitioning(i_category#19, i_brand#20, cc_name#21, 5), ENSURE_RE Input [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] Arguments: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15], [i_category#19 ASC NULLS FIRST, i_brand#20 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] -(33) ColumnarToRow [codegen id : 3] +(33) CometColumnarToRow [codegen id : 3] Input [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] (34) Window @@ -225,7 +225,7 @@ Output [6]: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sal Input [6]: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sales#15] Arguments: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sales#15], [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] -(41) ColumnarToRow [codegen id : 5] +(41) CometColumnarToRow [codegen id : 5] Input [6]: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sales#15] (42) Window @@ -258,7 +258,7 @@ Arguments: 100, [(sum_sales#15 - avg_monthly_sales#18) ASC NULLS FIRST, cc_name# Subquery:1 Hosting operator id = 3 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (51) -+- * ColumnarToRow (50) ++- * CometColumnarToRow (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -274,7 +274,7 @@ ReadSchema: struct Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (51) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/simplified.txt index 39a7ce9d67..10cd515da7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/simplified.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] CometExchange [i_category,i_brand,cc_name] #1 @@ -34,7 +34,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] @@ -51,7 +51,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] CometExchange [i_category,i_brand,cc_name] #8 @@ -64,7 +64,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/explain.txt index cd1e112f69..18736927ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (51) +* CometColumnarToRow (51) +- CometTakeOrderedAndProject (50) +- CometProject (49) +- CometBroadcastHashJoin (48) @@ -280,14 +280,14 @@ Arguments: [item_id#11, ss_item_rev#12, ss_dev#33, cs_item_rev#22, cs_dev#34, ws Input [8]: [item_id#11, ss_item_rev#12, ss_dev#33, cs_item_rev#22, cs_dev#34, ws_item_rev#32, ws_dev#35, average#36] Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_id#11 ASC NULLS FIRST,ss_item_rev#12 ASC NULLS FIRST], output=[item_id#11,ss_item_rev#12,ss_dev#33,cs_item_rev#22,cs_dev#34,ws_item_rev#32,ws_dev#35,average#36]), [item_id#11, ss_item_rev#12, ss_dev#33, cs_item_rev#22, cs_dev#34, ws_item_rev#32, ws_dev#35, average#36], 100, [item_id#11 ASC NULLS FIRST, ss_item_rev#12 ASC NULLS FIRST], [item_id#11, ss_item_rev#12, ss_dev#33, cs_item_rev#22, cs_dev#34, ws_item_rev#32, ws_dev#35, average#36] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [8]: [item_id#11, ss_item_rev#12, ss_dev#33, cs_item_rev#22, cs_dev#34, ws_item_rev#32, ws_dev#35, average#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (61) -+- * ColumnarToRow (60) ++- * CometColumnarToRow (60) +- CometProject (59) +- CometBroadcastHashJoin (58) :- CometFilter (53) @@ -337,7 +337,7 @@ Arguments: [d_date#8], [d_date#9], LeftSemi, BuildRight Input [2]: [d_date_sk#7, d_date#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(60) ColumnarToRow [codegen id : 1] +(60) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (61) BroadcastExchange @@ -345,7 +345,7 @@ Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] Subquery:2 Hosting operator id = 55 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* ColumnarToRow (65) +* CometColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometScan parquet spark_catalog.default.date_dim (62) @@ -366,7 +366,7 @@ Condition : (isnotnull(d_date#40) AND (d_date#40 = 2000-01-03)) Input [2]: [d_date#40, d_week_seq#41] Arguments: [d_week_seq#41], [d_week_seq#41] -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#41] Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/simplified.txt index 36fc8ac573..db48fd9775 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] CometProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometBroadcastHashJoin [d_date_sk,d_date,d_date] @@ -30,7 +30,7 @@ WholeStageCodegen (1) CometFilter [d_date,d_week_seq] Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_date,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/explain.txt index 62311fb338..81f784c134 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (39) +* CometColumnarToRow (39) +- CometTakeOrderedAndProject (38) +- CometProject (37) +- CometBroadcastHashJoin (36) @@ -219,6 +219,6 @@ Arguments: [s_store_name1#26, s_store_id1#28, d_week_seq1#27, (sun_sales1 / sun_ Input [10]: [s_store_name1#26, s_store_id1#28, d_week_seq1#27, (sun_sales1 / sun_sales2)#67, (mon_sales1 / mon_sales2)#68, (tue_sales1 / tue_sales2)#69, (wed_sales1 / wed_sales2)#70, (thu_sales1 / thu_sales2)#71, (fri_sales1 / fri_sales2)#72, (sat_sales1 / sat_sales2)#73] Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name1#26 ASC NULLS FIRST,s_store_id1#28 ASC NULLS FIRST,d_week_seq1#27 ASC NULLS FIRST], output=[s_store_name1#26,s_store_id1#28,d_week_seq1#27,(sun_sales1 / sun_sales2)#67,(mon_sales1 / mon_sales2)#68,(tue_sales1 / tue_sales2)#69,(wed_sales1 / wed_sales2)#70,(thu_sales1 / thu_sales2)#71,(fri_sales1 / fri_sales2)#72,(sat_sales1 / sat_sales2)#73]), [s_store_name1#26, s_store_id1#28, d_week_seq1#27, (sun_sales1 / sun_sales2)#67, (mon_sales1 / mon_sales2)#68, (tue_sales1 / tue_sales2)#69, (wed_sales1 / wed_sales2)#70, (thu_sales1 / thu_sales2)#71, (fri_sales1 / fri_sales2)#72, (sat_sales1 / sat_sales2)#73], 100, [s_store_name1#26 ASC NULLS FIRST, s_store_id1#28 ASC NULLS FIRST, d_week_seq1#27 ASC NULLS FIRST], [s_store_name1#26, s_store_id1#28, d_week_seq1#27, (sun_sales1 / sun_sales2)#67, (mon_sales1 / mon_sales2)#68, (tue_sales1 / tue_sales2)#69, (wed_sales1 / wed_sales2)#70, (thu_sales1 / thu_sales2)#71, (fri_sales1 / fri_sales2)#72, (sat_sales1 / sat_sales2)#73] -(39) ColumnarToRow [codegen id : 1] +(39) CometColumnarToRow [codegen id : 1] Input [10]: [s_store_name1#26, s_store_id1#28, d_week_seq1#27, (sun_sales1 / sun_sales2)#67, (mon_sales1 / mon_sales2)#68, (tue_sales1 / tue_sales2)#69, (wed_sales1 / wed_sales2)#70, (thu_sales1 / thu_sales2)#71, (fri_sales1 / fri_sales2)#72, (sat_sales1 / sat_sales2)#73] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/simplified.txt index 140a764391..15f2e45693 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/explain.txt index e87f6ce763..542920b01c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (38) +* CometColumnarToRow (38) +- CometTakeOrderedAndProject (37) +- CometFilter (36) +- CometHashAggregate (35) @@ -215,14 +215,14 @@ Condition : (cnt#23 >= 10) Input [2]: [state#22, cnt#23] Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#23 ASC NULLS FIRST], output=[state#22,cnt#23]), [state#22, cnt#23], 100, [cnt#23 ASC NULLS FIRST], [state#22, cnt#23] -(38) ColumnarToRow [codegen id : 1] +(38) CometColumnarToRow [codegen id : 1] Input [2]: [state#22, cnt#23] ===== Subqueries ===== Subquery:1 Hosting operator id = 8 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (43) -+- * ColumnarToRow (42) ++- * CometColumnarToRow (42) +- CometProject (41) +- CometFilter (40) +- CometScan parquet spark_catalog.default.date_dim (39) @@ -243,7 +243,7 @@ Condition : ((isnotnull(d_month_seq#10) AND (d_month_seq#10 = Subquery scalar-su Input [2]: [d_date_sk#9, d_month_seq#10] Arguments: [d_date_sk#9], [d_date_sk#9] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] (43) BroadcastExchange @@ -251,7 +251,7 @@ Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] Subquery:2 Hosting operator id = 40 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* ColumnarToRow (50) +* CometColumnarToRow (50) +- CometHashAggregate (49) +- CometExchange (48) +- CometHashAggregate (47) @@ -289,7 +289,7 @@ Input [1]: [d_month_seq#24] Keys [1]: [d_month_seq#24] Functions: [] -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [1]: [d_month_seq#24] Subquery:3 Hosting operator id = 14 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/simplified.txt index 05566cbca5..1ba3d55d23 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [state,cnt] CometFilter [state,cnt] @@ -25,13 +25,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [d_month_seq] CometExchange [d_month_seq] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/explain.txt index 0d64004d9a..5596623c61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (61) +* CometColumnarToRow (61) +- CometTakeOrderedAndProject (60) +- CometHashAggregate (59) +- CometExchange (58) @@ -339,14 +339,14 @@ Functions [1]: [sum(total_sales#36)] Input [2]: [i_item_id#12, total_sales#41] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#12 ASC NULLS FIRST,total_sales#41 ASC NULLS FIRST], output=[i_item_id#12,total_sales#41]), [i_item_id#12, total_sales#41], 100, [i_item_id#12 ASC NULLS FIRST, total_sales#41 ASC NULLS FIRST], [i_item_id#12, total_sales#41] -(61) ColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [codegen id : 1] Input [2]: [i_item_id#12, total_sales#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (66) -+- * ColumnarToRow (65) ++- * CometColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometScan parquet spark_catalog.default.date_dim (62) @@ -367,7 +367,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (66) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt index 4dd8b3b5bd..ea973a4810 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,total_sales] CometHashAggregate [i_item_id,total_sales,sum,isEmpty,sum(total_sales)] @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/explain.txt index 4a157e64e8..9656e93142 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * Project (65) +- * BroadcastNestedLoopJoin Inner BuildRight (64) - :- * ColumnarToRow (41) + :- * CometColumnarToRow (41) : +- CometHashAggregate (40) : +- CometExchange (39) : +- CometHashAggregate (38) @@ -43,7 +43,7 @@ : +- CometFilter (33) : +- CometScan parquet spark_catalog.default.item (32) +- BroadcastExchange (63) - +- * ColumnarToRow (62) + +- * CometColumnarToRow (62) +- CometHashAggregate (61) +- CometExchange (60) +- CometHashAggregate (59) @@ -256,7 +256,7 @@ Input [1]: [sum#23] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -(41) ColumnarToRow [codegen id : 2] +(41) CometColumnarToRow [codegen id : 2] Input [1]: [promotions#24] (42) CometScan parquet spark_catalog.default.store_sales @@ -345,7 +345,7 @@ Input [1]: [sum#37] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] -(62) ColumnarToRow [codegen id : 1] +(62) CometColumnarToRow [codegen id : 1] Input [1]: [total#38] (63) BroadcastExchange @@ -364,7 +364,7 @@ Input [2]: [promotions#24, total#38] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (70) -+- * ColumnarToRow (69) ++- * CometColumnarToRow (69) +- CometProject (68) +- CometFilter (67) +- CometScan parquet spark_catalog.default.date_dim (66) @@ -385,7 +385,7 @@ Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1 Input [3]: [d_date_sk#14, d_year#15, d_moy#16] Arguments: [d_date_sk#14], [d_date_sk#14] -(69) ColumnarToRow [codegen id : 1] +(69) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] (70) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/simplified.txt index 548614416e..30c16b696d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (2) Project [promotions,total] BroadcastNestedLoopJoin - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [promotions,sum,sum(UnscaledValue(ss_ext_sales_price))] CometExchange #1 @@ -23,7 +23,7 @@ WholeStageCodegen (2) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -54,7 +54,7 @@ WholeStageCodegen (2) InputAdapter BroadcastExchange #9 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [total,sum,sum(UnscaledValue(ss_ext_sales_price))] CometExchange #10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/explain.txt index c174784d38..b80eab371d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (28) +* CometColumnarToRow (28) +- CometTakeOrderedAndProject (27) +- CometHashAggregate (26) +- CometExchange (25) @@ -158,6 +158,6 @@ Functions [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) TH Input [8]: [substr(w_warehouse_name, 1, 20)#20, sm_type#9, web_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25] Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#20 ASC NULLS FIRST,sm_type#9 ASC NULLS FIRST,web_name#11 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#20,sm_type#9,web_name#11,30 days #21,31 - 60 days #22,61 - 90 days #23,91 - 120 days #24,>120 days #25]), [substr(w_warehouse_name, 1, 20)#20, sm_type#9, web_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25], 100, [substr(w_warehouse_name, 1, 20)#20 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, web_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#20, sm_type#9, web_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [8]: [substr(w_warehouse_name, 1, 20)#20, sm_type#9, web_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/simplified.txt index 6a9b1ced35..002eedf08e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] CometHashAggregate [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,sum,sum,sum,sum,sum,sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 30) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 60) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 90) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) > 120) THEN 1 ELSE 0 END)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/explain.txt index 2a5ae3d1f4..08e1a99a58 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (29) +- * Project (28) +- * Filter (27) +- Window (26) - +- * ColumnarToRow (25) + +- * CometColumnarToRow (25) +- CometSort (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -144,7 +144,7 @@ Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometNative Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] Arguments: [i_manager_id#5, sum_sales#20, _w0#21], [i_manager_id#5 ASC NULLS FIRST] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] (26) Window @@ -167,7 +167,7 @@ Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (34) -+- * ColumnarToRow (33) ++- * CometColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] -(33) ColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#15, d_moy#17] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/simplified.txt index 8ea16ea740..8e1ce8c997 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_manager_id] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_manager_id,sum_sales,_w0] CometExchange [i_manager_id] #1 @@ -27,7 +27,7 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_month_seq,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/explain.txt index 2dcffc7a77..8fd7206781 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (173) +* CometColumnarToRow (173) +- CometSort (172) +- CometColumnarExchange (171) +- CometProject (170) @@ -938,14 +938,14 @@ Arguments: rangepartitioning(product_name#76 ASC NULLS FIRST, store_name#78 ASC Input [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] Arguments: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162], [product_name#76 ASC NULLS FIRST, store_name#78 ASC NULLS FIRST, cnt#162 ASC NULLS FIRST] -(173) ColumnarToRow [codegen id : 1] +(173) CometColumnarToRow [codegen id : 1] Input [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (177) -+- * ColumnarToRow (176) ++- * CometColumnarToRow (176) +- CometFilter (175) +- CometScan parquet spark_catalog.default.date_dim (174) @@ -961,7 +961,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(176) ColumnarToRow [codegen id : 1] +(176) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] (177) BroadcastExchange @@ -970,7 +970,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#104 IN dynamicpruning#105 BroadcastExchange (181) -+- * ColumnarToRow (180) ++- * CometColumnarToRow (180) +- CometFilter (179) +- CometScan parquet spark_catalog.default.date_dim (178) @@ -986,7 +986,7 @@ ReadSchema: struct Input [2]: [d_date_sk#117, d_year#118] Condition : ((isnotnull(d_year#118) AND (d_year#118 = 2000)) AND isnotnull(d_date_sk#117)) -(180) ColumnarToRow [codegen id : 1] +(180) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#117, d_year#118] (181) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt index 49a08d6f8b..0ce55b5714 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] CometColumnarExchange [product_name,store_name,cnt] #1 @@ -51,7 +51,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -158,7 +158,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #22 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/explain.txt index e0f1b47992..09ddb55e37 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (39) +* CometColumnarToRow (39) +- CometTakeOrderedAndProject (38) +- CometProject (37) +- CometBroadcastHashJoin (36) @@ -219,14 +219,14 @@ Arguments: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wh Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16] Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#2 ASC NULLS FIRST,i_item_desc#13 ASC NULLS FIRST], output=[s_store_name#2,i_item_desc#13,revenue#11,i_current_price#14,i_wholesale_cost#15,i_brand#16]), [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16], 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#13 ASC NULLS FIRST], [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16] -(39) ColumnarToRow [codegen id : 1] +(39) CometColumnarToRow [codegen id : 1] Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (44) -+- * ColumnarToRow (43) ++- * CometColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan parquet spark_catalog.default.date_dim (40) @@ -247,7 +247,7 @@ Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_mont Input [2]: [d_date_sk#8, d_month_seq#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(43) ColumnarToRow [codegen id : 1] +(43) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#8] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/simplified.txt index be08ee23a0..28412503a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] CometProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/explain.txt index 693eb5662d..58f093ae40 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (50) +* CometColumnarToRow (50) +- CometTakeOrderedAndProject (49) +- CometHashAggregate (48) +- CometExchange (47) @@ -277,14 +277,14 @@ Functions [36]: [sum(jan_sales#141), sum(feb_sales#142), sum(mar_sales#143), sum Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#139, year#140, jan_sales#263, feb_sales#264, mar_sales#265, apr_sales#266, may_sales#267, jun_sales#268, jul_sales#269, aug_sales#270, sep_sales#271, oct_sales#272, nov_sales#273, dec_sales#274, jan_sales_per_sq_foot#275, feb_sales_per_sq_foot#276, mar_sales_per_sq_foot#277, apr_sales_per_sq_foot#278, may_sales_per_sq_foot#279, jun_sales_per_sq_foot#280, jul_sales_per_sq_foot#281, aug_sales_per_sq_foot#282, sep_sales_per_sq_foot#283, oct_sales_per_sq_foot#284, nov_sales_per_sq_foot#285, dec_sales_per_sq_foot#286, jan_net#287, feb_net#288, mar_net#289, apr_net#290, may_net#291, jun_net#292, jul_net#293, aug_net#294, sep_net#295, oct_net#296, nov_net#297, dec_net#298] Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_warehouse_name#10 ASC NULLS FIRST], output=[w_warehouse_name#10,w_warehouse_sq_ft#11,w_city#12,w_county#13,w_state#14,w_country#15,ship_carriers#139,year#140,jan_sales#263,feb_sales#264,mar_sales#265,apr_sales#266,may_sales#267,jun_sales#268,jul_sales#269,aug_sales#270,sep_sales#271,oct_sales#272,nov_sales#273,dec_sales#274,jan_sales_per_sq_foot#275,feb_sales_per_sq_foot#276,mar_sales_per_sq_foot#277,apr_sales_per_sq_foot#278,may_sales_per_sq_foot#279,jun_sales_per_sq_foot#280,jul_sales_per_sq_foot#281,aug_sales_per_sq_foot#282,sep_sales_per_sq_foot#283,oct_sales_per_sq_foot#284,nov_sales_per_sq_foot#285,dec_sales_per_sq_foot#286,jan_net#287,feb_net#288,mar_net#289,apr_net#290,may_net#291,jun_net#292,jul_net#293,aug_net#294,sep_net#295,oct_net#296,nov_net#297,dec_net#298]), [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#139, year#140, jan_sales#263, feb_sales#264, mar_sales#265, apr_sales#266, may_sales#267, jun_sales#268, jul_sales#269, aug_sales#270, sep_sales#271, oct_sales#272, nov_sales#273, dec_sales#274, jan_sales_per_sq_foot#275, feb_sales_per_sq_foot#276, mar_sales_per_sq_foot#277, apr_sales_per_sq_foot#278, may_sales_per_sq_foot#279, jun_sales_per_sq_foot#280, jul_sales_per_sq_foot#281, aug_sales_per_sq_foot#282, sep_sales_per_sq_foot#283, oct_sales_per_sq_foot#284, nov_sales_per_sq_foot#285, dec_sales_per_sq_foot#286, jan_net#287, feb_net#288, mar_net#289, apr_net#290, may_net#291, jun_net#292, jul_net#293, aug_net#294, sep_net#295, oct_net#296, nov_net#297, dec_net#298], 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#139, year#140, jan_sales#263, feb_sales#264, mar_sales#265, apr_sales#266, may_sales#267, jun_sales#268, jul_sales#269, aug_sales#270, sep_sales#271, oct_sales#272, nov_sales#273, dec_sales#274, jan_sales_per_sq_foot#275, feb_sales_per_sq_foot#276, mar_sales_per_sq_foot#277, apr_sales_per_sq_foot#278, may_sales_per_sq_foot#279, jun_sales_per_sq_foot#280, jul_sales_per_sq_foot#281, aug_sales_per_sq_foot#282, sep_sales_per_sq_foot#283, oct_sales_per_sq_foot#284, nov_sales_per_sq_foot#285, dec_sales_per_sq_foot#286, jan_net#287, feb_net#288, mar_net#289, apr_net#290, may_net#291, jun_net#292, jul_net#293, aug_net#294, sep_net#295, oct_net#296, nov_net#297, dec_net#298] -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#139, year#140, jan_sales#263, feb_sales#264, mar_sales#265, apr_sales#266, may_sales#267, jun_sales#268, jul_sales#269, aug_sales#270, sep_sales#271, oct_sales#272, nov_sales#273, dec_sales#274, jan_sales_per_sq_foot#275, feb_sales_per_sq_foot#276, mar_sales_per_sq_foot#277, apr_sales_per_sq_foot#278, may_sales_per_sq_foot#279, jun_sales_per_sq_foot#280, jul_sales_per_sq_foot#281, aug_sales_per_sq_foot#282, sep_sales_per_sq_foot#283, oct_sales_per_sq_foot#284, nov_sales_per_sq_foot#285, dec_sales_per_sq_foot#286, jan_net#287, feb_net#288, mar_net#289, apr_net#290, may_net#291, jun_net#292, jul_net#293, aug_net#294, sep_net#295, oct_net#296, nov_net#297, dec_net#298] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (54) -+- * ColumnarToRow (53) ++- * CometColumnarToRow (53) +- CometFilter (52) +- CometScan parquet spark_catalog.default.date_dim (51) @@ -300,7 +300,7 @@ ReadSchema: struct Input [3]: [d_date_sk#16, d_year#17, d_moy#18] Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) -(53) ColumnarToRow [codegen id : 1] +(53) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#16, d_year#17, d_moy#18] (54) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/simplified.txt index a2b5963e13..3529a210a6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum(jan_sales),sum(feb_sales),sum(mar_sales),sum(apr_sales),sum(may_sales),sum(jun_sales),sum(jul_sales),sum(aug_sales),sum(sep_sales),sum(oct_sales),sum(nov_sales),sum(dec_sales),sum((jan_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((feb_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((mar_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((apr_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((may_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jun_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jul_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((aug_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((sep_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((oct_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((nov_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((dec_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum(jan_net),sum(feb_net),sum(mar_net),sum(apr_net),sum(may_net),sum(jun_net),sum(jul_net),sum(aug_net),sum(sep_net),sum(oct_net),sum(nov_net),sum(dec_net)] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/explain.txt index d63914c5e6..231bcf55a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/explain.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject (31) +- * Sort (27) +- Exchange (26) +- WindowGroupLimit (25) - +- * ColumnarToRow (24) + +- * CometColumnarToRow (24) +- CometSort (23) +- CometHashAggregate (22) +- CometExchange (21) @@ -142,7 +142,7 @@ Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(1 Input [9]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, sumsales#30] Arguments: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, sumsales#30], [i_category#19 ASC NULLS FIRST, sumsales#30 DESC NULLS LAST] -(24) ColumnarToRow [codegen id : 1] +(24) CometColumnarToRow [codegen id : 1] Input [9]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, sumsales#30] (25) WindowGroupLimit @@ -177,7 +177,7 @@ Arguments: 100, [i_category#19 ASC NULLS FIRST, i_class#20 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (36) -+- * ColumnarToRow (35) ++- * CometColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan parquet spark_catalog.default.date_dim (32) @@ -198,7 +198,7 @@ Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_mont Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(35) ColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [codegen id : 1] Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/simplified.txt index 1f3b8c203c..68519b6404 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/simplified.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ Exchange [i_category] #1 WindowGroupLimit [i_category,sumsales] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,spark_grouping_id,sum,isEmpty,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] @@ -28,7 +28,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/explain.txt index 121987260e..00c0db86ec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (38) +* CometColumnarToRow (38) +- CometTakeOrderedAndProject (37) +- CometProject (36) +- CometBroadcastHashJoin (35) @@ -213,14 +213,14 @@ Arguments: [c_last_name#27, c_first_name#26, ca_city#33, bought_city#28, ss_tick Input [8]: [c_last_name#27, c_first_name#26, ca_city#33, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#27 ASC NULLS FIRST,ss_ticket_number#5 ASC NULLS FIRST], output=[c_last_name#27,c_first_name#26,ca_city#33,bought_city#28,ss_ticket_number#5,extended_price#29,extended_tax#31,list_price#30]), [c_last_name#27, c_first_name#26, ca_city#33, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30], 100, [c_last_name#27 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#27, c_first_name#26, ca_city#33, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] -(38) ColumnarToRow [codegen id : 1] +(38) CometColumnarToRow [codegen id : 1] Input [8]: [c_last_name#27, c_first_name#26, ca_city#33, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (43) -+- * ColumnarToRow (42) ++- * CometColumnarToRow (42) +- CometProject (41) +- CometFilter (40) +- CometScan parquet spark_catalog.default.date_dim (39) @@ -241,7 +241,7 @@ Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AN Input [3]: [d_date_sk#11, d_year#12, d_dom#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (43) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/simplified.txt index 568fd734b9..986918c8b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] CometProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/explain.txt index 5386a3d166..eaf61eb735 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/explain.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject (44) : :- * Project (27) : : +- * BroadcastHashJoin LeftAnti BuildRight (26) : : :- * BroadcastHashJoin LeftAnti BuildRight (19) - : : : :- * ColumnarToRow (12) + : : : :- * CometColumnarToRow (12) : : : : +- CometBroadcastHashJoin (11) : : : : :- CometFilter (2) : : : : : +- CometScan parquet spark_catalog.default.customer (1) @@ -23,24 +23,24 @@ TakeOrderedAndProject (44) : : : : +- CometFilter (5) : : : : +- CometScan parquet spark_catalog.default.date_dim (4) : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) + : : : +- * CometColumnarToRow (17) : : : +- CometProject (16) : : : +- CometBroadcastHashJoin (15) : : : :- CometScan parquet spark_catalog.default.web_sales (13) : : : +- ReusedExchange (14) : : +- BroadcastExchange (25) - : : +- * ColumnarToRow (24) + : : +- * CometColumnarToRow (24) : : +- CometProject (23) : : +- CometBroadcastHashJoin (22) : : :- CometScan parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (32) - : +- * ColumnarToRow (31) + : +- * CometColumnarToRow (31) : +- CometProject (30) : +- CometFilter (29) : +- CometScan parquet spark_catalog.default.customer_address (28) +- BroadcastExchange (38) - +- * ColumnarToRow (37) + +- * CometColumnarToRow (37) +- CometFilter (36) +- CometScan parquet spark_catalog.default.customer_demographics (35) @@ -100,7 +100,7 @@ Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Right output [1]: [ss_customer_sk#4] Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight -(12) ColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 5] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (13) CometScan parquet spark_catalog.default.web_sales @@ -122,7 +122,7 @@ Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] Arguments: [ws_bill_customer_sk#10], [ws_bill_customer_sk#10] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#10] (18) BroadcastExchange @@ -154,7 +154,7 @@ Arguments: [cs_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] Arguments: [cs_ship_customer_sk#14], [cs_ship_customer_sk#14] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [1]: [cs_ship_customer_sk#14] (25) BroadcastExchange @@ -186,7 +186,7 @@ Condition : (ca_state#19 IN (KY,GA,NM) AND isnotnull(ca_address_sk#18)) Input [2]: [ca_address_sk#18, ca_state#19] Arguments: [ca_address_sk#18], [ca_address_sk#18] -(31) ColumnarToRow [codegen id : 3] +(31) CometColumnarToRow [codegen id : 3] Input [1]: [ca_address_sk#18] (32) BroadcastExchange @@ -214,7 +214,7 @@ ReadSchema: struct Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) ColumnarToRow [codegen id : 3] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) CometScan parquet spark_catalog.default.store_sales @@ -169,7 +169,7 @@ Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] Input [3]: [s_state#15, _w0#18, s_state#15] Arguments: [s_state#15, _w0#18, s_state#15], [s_state#15 ASC NULLS FIRST, _w0#18 DESC NULLS LAST] -(27) ColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [3]: [s_state#15, _w0#18, s_state#15] (28) WindowGroupLimit @@ -258,7 +258,7 @@ Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (50) -+- * ColumnarToRow (49) ++- * CometColumnarToRow (49) +- CometProject (48) +- CometFilter (47) +- CometScan parquet spark_catalog.default.date_dim (46) @@ -279,7 +279,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(49) ColumnarToRow [codegen id : 1] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (50) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/simplified.txt index 7da5aad4f8..371d15bbe8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/simplified.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count Expand [ss_net_profit,s_state,s_county] Project [ss_net_profit,s_state,s_county] BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] @@ -25,7 +25,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -38,7 +38,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #5 WholeStageCodegen (3) BroadcastHashJoin [s_state,s_state] - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] @@ -51,7 +51,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count Window [_w0,s_state] WindowGroupLimit [s_state,_w0] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [s_state,_w0] CometHashAggregate [s_state,_w0,sum,sum(UnscaledValue(ss_net_profit))] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/explain.txt index 79a71af03a..fe8f7c5f76 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (37) +* CometColumnarToRow (37) +- CometSort (36) +- CometColumnarExchange (35) +- CometHashAggregate (34) @@ -209,14 +209,14 @@ Arguments: rangepartitioning(ext_price#41 DESC NULLS LAST, brand_id#39 ASC NULLS Input [5]: [brand_id#39, brand#40, t_hour#35, t_minute#36, ext_price#41] Arguments: [brand_id#39, brand#40, t_hour#35, t_minute#36, ext_price#41], [ext_price#41 DESC NULLS LAST, brand_id#39 ASC NULLS FIRST] -(37) ColumnarToRow [codegen id : 1] +(37) CometColumnarToRow [codegen id : 1] Input [5]: [brand_id#39, brand#40, t_hour#35, t_minute#36, ext_price#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (42) -+- * ColumnarToRow (41) ++- * CometColumnarToRow (41) +- CometProject (40) +- CometFilter (39) +- CometScan parquet spark_catalog.default.date_dim (38) @@ -237,7 +237,7 @@ Condition : ((((isnotnull(d_moy#12) AND isnotnull(d_year#11)) AND (d_moy#12 = 11 Input [3]: [d_date_sk#10, d_year#11, d_moy#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(41) ColumnarToRow [codegen id : 1] +(41) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (42) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt index 88ca8b67df..7214ec67a5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [brand_id,brand,t_hour,t_minute,ext_price] CometColumnarExchange [ext_price,brand_id] #1 @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/explain.txt index 6aae88fff3..4179f3b1d6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (64) +* CometColumnarToRow (64) +- CometTakeOrderedAndProject (63) +- CometHashAggregate (62) +- CometExchange (61) @@ -364,14 +364,14 @@ Functions [1]: [count(1)] Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, [total_cnt#37 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] -(64) ColumnarToRow [codegen id : 1] +(64) CometColumnarToRow [codegen id : 1] Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (69) -+- * ColumnarToRow (68) ++- * CometColumnarToRow (68) +- CometProject (67) +- CometFilter (66) +- CometScan parquet spark_catalog.default.date_dim (65) @@ -392,7 +392,7 @@ Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_dat Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(68) ColumnarToRow [codegen id : 1] +(68) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (69) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/simplified.txt index ccd1b66fa2..fe26a3f5cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count,count(1)] @@ -32,7 +32,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] CometFilter [d_date_sk,d_date,d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/explain.txt index b0ca5520f2..a7c93664a7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (32) +* CometColumnarToRow (32) +- CometSort (31) +- CometColumnarExchange (30) +- CometProject (29) @@ -179,14 +179,14 @@ Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, Co Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] Arguments: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [cnt#17 DESC NULLS LAST] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (37) -+- * ColumnarToRow (36) ++- * CometColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometScan parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt index 4417365325..eaede7232d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometColumnarExchange [cnt] #1 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/explain.txt index c7a559c8ef..6358cf7e31 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (68) +* CometColumnarToRow (68) +- CometTakeOrderedAndProject (67) +- CometProject (66) +- CometBroadcastHashJoin (65) @@ -388,14 +388,14 @@ Arguments: [customer_id#25, customer_first_name#26, customer_last_name#27], [cus Input [3]: [customer_id#25, customer_first_name#26, customer_last_name#27] Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#25 ASC NULLS FIRST,customer_id#25 ASC NULLS FIRST,customer_id#25 ASC NULLS FIRST], output=[customer_id#25,customer_first_name#26,customer_last_name#27]), [customer_id#25, customer_first_name#26, customer_last_name#27], 100, [customer_id#25 ASC NULLS FIRST, customer_id#25 ASC NULLS FIRST, customer_id#25 ASC NULLS FIRST], [customer_id#25, customer_first_name#26, customer_last_name#27] -(68) ColumnarToRow [codegen id : 1] +(68) CometColumnarToRow [codegen id : 1] Input [3]: [customer_id#25, customer_first_name#26, customer_last_name#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (72) -+- * ColumnarToRow (71) ++- * CometColumnarToRow (71) +- CometFilter (70) +- CometScan parquet spark_catalog.default.date_dim (69) @@ -411,7 +411,7 @@ ReadSchema: struct Input [2]: [d_date_sk#9, d_year#10] Condition : (((isnotnull(d_year#10) AND (d_year#10 = 2001)) AND d_year#10 IN (2001,2002)) AND isnotnull(d_date_sk#9)) -(71) ColumnarToRow [codegen id : 1] +(71) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] (72) BroadcastExchange @@ -420,7 +420,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 BroadcastExchange (76) -+- * ColumnarToRow (75) ++- * CometColumnarToRow (75) +- CometFilter (74) +- CometScan parquet spark_catalog.default.date_dim (73) @@ -436,7 +436,7 @@ ReadSchema: struct Input [2]: [d_date_sk#22, d_year#23] Condition : (((isnotnull(d_year#23) AND (d_year#23 = 2002)) AND d_year#23 IN (2001,2002)) AND isnotnull(d_date_sk#22)) -(75) ColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#22, d_year#23] (76) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/simplified.txt index 631a82f1e3..9bde9ac863 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] CometProject [customer_id,customer_first_name,customer_last_name] @@ -23,7 +23,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -46,7 +46,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/explain.txt index 2f33f50a8b..9d68c7a66c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (124) +* CometColumnarToRow (124) +- CometTakeOrderedAndProject (123) +- CometProject (122) +- CometSortMergeJoin (121) @@ -676,14 +676,14 @@ Arguments: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#1 Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#124 ASC NULLS FIRST], output=[prev_year#120,year#121,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#122,curr_yr_cnt#123,sales_cnt_diff#124,sales_amt_diff#125]), [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], 100, [sales_cnt_diff#124 ASC NULLS FIRST], [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] -(124) ColumnarToRow [codegen id : 1] +(124) CometColumnarToRow [codegen id : 1] Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (128) -+- * ColumnarToRow (127) ++- * CometColumnarToRow (127) +- CometFilter (126) +- CometScan parquet spark_catalog.default.date_dim (125) @@ -699,7 +699,7 @@ ReadSchema: struct Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(127) ColumnarToRow [codegen id : 1] +(127) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] (128) BroadcastExchange @@ -712,7 +712,7 @@ Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN d Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 BroadcastExchange (132) -+- * ColumnarToRow (131) ++- * CometColumnarToRow (131) +- CometFilter (130) +- CometScan parquet spark_catalog.default.date_dim (129) @@ -728,7 +728,7 @@ ReadSchema: struct Input [2]: [d_date_sk#77, d_year#78] Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(131) ColumnarToRow [codegen id : 1] +(131) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#77, d_year#78] (132) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/simplified.txt index 9e340130a5..5a8af58abc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] @@ -27,7 +27,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -102,7 +102,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/explain.txt index a2a17ccfce..24ed6809af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (34) +* CometColumnarToRow (34) +- CometTakeOrderedAndProject (33) +- CometHashAggregate (32) +- CometExchange (31) @@ -190,6 +190,6 @@ Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#12))] Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#39, sales_amt#40] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#10 ASC NULLS FIRST,col_name#11 ASC NULLS FIRST,d_year#8 ASC NULLS FIRST,d_qoy#9 ASC NULLS FIRST,i_category#6 ASC NULLS FIRST], output=[channel#10,col_name#11,d_year#8,d_qoy#9,i_category#6,sales_cnt#39,sales_amt#40]), [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#39, sales_amt#40], 100, [channel#10 ASC NULLS FIRST, col_name#11 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#9 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#39, sales_amt#40] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#39, sales_amt#40] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/simplified.txt index eb3e14d5ec..3e89713523 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt,count,sum,count(1),sum(UnscaledValue(ext_sales_price))] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/explain.txt index 661b23c47b..dccd7c8e56 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/explain.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject (84) +- * HashAggregate (81) +- * Expand (80) +- Union (79) - :- * ColumnarToRow (31) + :- * CometColumnarToRow (31) : +- CometProject (30) : +- CometBroadcastHashJoin (29) : :- CometHashAggregate (16) @@ -39,7 +39,7 @@ TakeOrderedAndProject (84) :- * Project (50) : +- * BroadcastNestedLoopJoin Inner BuildLeft (49) : :- BroadcastExchange (40) - : : +- * ColumnarToRow (39) + : : +- * CometColumnarToRow (39) : : +- CometHashAggregate (38) : : +- CometExchange (37) : : +- CometHashAggregate (36) @@ -47,7 +47,7 @@ TakeOrderedAndProject (84) : : +- CometBroadcastHashJoin (34) : : :- CometScan parquet spark_catalog.default.catalog_sales (32) : : +- ReusedExchange (33) - : +- * ColumnarToRow (48) + : +- * CometColumnarToRow (48) : +- CometHashAggregate (47) : +- CometExchange (46) : +- CometHashAggregate (45) @@ -55,7 +55,7 @@ TakeOrderedAndProject (84) : +- CometBroadcastHashJoin (43) : :- CometScan parquet spark_catalog.default.catalog_returns (41) : +- ReusedExchange (42) - +- * ColumnarToRow (78) + +- * CometColumnarToRow (78) +- CometProject (77) +- CometBroadcastHashJoin (76) :- CometHashAggregate (63) @@ -226,7 +226,7 @@ Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] Arguments: [sales#22, returns#24, profit#25, channel#26, id#27], [sales#22, coalesce(returns#20, 0.00) AS returns#24, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#25, store channel AS channel#26, s_store_sk#8 AS id#27] -(31) ColumnarToRow [codegen id : 1] +(31) CometColumnarToRow [codegen id : 1] Input [5]: [sales#22, returns#24, profit#25, channel#26, id#27] (32) CometScan parquet spark_catalog.default.catalog_sales @@ -262,7 +262,7 @@ Input [3]: [cs_call_center_sk#28, sum#34, sum#35] Keys [1]: [cs_call_center_sk#28] Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#29)), sum(UnscaledValue(cs_net_profit#30))] -(39) ColumnarToRow [codegen id : 2] +(39) CometColumnarToRow [codegen id : 2] Input [3]: [cs_call_center_sk#28, sales#36, profit#37] (40) BroadcastExchange @@ -302,7 +302,7 @@ Input [2]: [sum#43, sum#44] Keys: [] Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] -(48) ColumnarToRow +(48) CometColumnarToRow Input [2]: [returns#45, profit_loss#46] (49) BroadcastNestedLoopJoin [codegen id : 3] @@ -438,7 +438,7 @@ Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] Arguments: [sales#70, returns#72, profit#73, channel#74, id#75], [sales#70, coalesce(returns#68, 0.00) AS returns#72, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#73, web channel AS channel#74, wp_web_page_sk#56 AS id#75] -(78) ColumnarToRow [codegen id : 4] +(78) CometColumnarToRow [codegen id : 4] Input [5]: [sales#70, returns#72, profit#73, channel#74, id#75] (79) Union @@ -473,7 +473,7 @@ Arguments: 100, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (89) -+- * ColumnarToRow (88) ++- * CometColumnarToRow (88) +- CometProject (87) +- CometFilter (86) +- CometScan parquet spark_catalog.default.date_dim (85) @@ -494,7 +494,7 @@ Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 < Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(88) ColumnarToRow [codegen id : 1] +(88) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (89) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/simplified.txt index e8226ab1fa..60fcf20e24 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/simplified.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter Union WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [returns,profit,profit_loss,s_store_sk] [sales,returns,profit,channel,id] CometBroadcastHashJoin [s_store_sk,sales,profit,s_store_sk,returns,profit_loss] @@ -25,7 +25,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] @@ -56,7 +56,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter BroadcastExchange #8 WholeStageCodegen (2) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [cs_call_center_sk,sales,profit,sum,sum,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] CometExchange [cs_call_center_sk] #9 @@ -66,7 +66,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [returns,profit_loss,sum,sum,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] CometExchange #10 @@ -77,7 +77,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 WholeStageCodegen (4) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [returns,profit,profit_loss,wp_web_page_sk] [sales,returns,profit,channel,id] CometBroadcastHashJoin [wp_web_page_sk,sales,profit,wp_web_page_sk,returns,profit_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/explain.txt index b8b24b9d0b..41ea1060b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (67) +- * Project (66) - +- * ColumnarToRow (65) + +- * CometColumnarToRow (65) +- CometSortMergeJoin (64) :- CometProject (43) : +- CometSortMergeJoin (42) @@ -360,7 +360,7 @@ Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss Right output [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59], Inner -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] (66) Project [codegen id : 1] @@ -375,7 +375,7 @@ Arguments: 100, [ratio#63 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 D Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (71) -+- * ColumnarToRow (70) ++- * CometColumnarToRow (70) +- CometFilter (69) +- CometScan parquet spark_catalog.default.date_dim (68) @@ -391,7 +391,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(70) ColumnarToRow [codegen id : 1] +(70) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] (71) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/simplified.txt index 3318702eff..6b85e03ae9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] WholeStageCodegen (1) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] - ColumnarToRow + CometColumnarToRow InputAdapter CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] @@ -22,7 +22,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/explain.txt index e697c9038b..c4b54c4ed5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (30) +* CometColumnarToRow (30) +- CometTakeOrderedAndProject (29) +- CometProject (28) +- CometBroadcastHashJoin (27) @@ -169,14 +169,14 @@ Arguments: [c_last_name#23, c_first_name#22, substr(s_city, 1, 30)#26, ss_ticket Input [7]: [c_last_name#23, c_first_name#22, substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#24, profit#25, s_city#15] Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#23 ASC NULLS FIRST,c_first_name#22 ASC NULLS FIRST,substr(s_city#15, 1, 30) ASC NULLS FIRST,profit#25 ASC NULLS FIRST], output=[c_last_name#23,c_first_name#22,substr(s_city, 1, 30)#26,ss_ticket_number#5,amt#24,profit#25]), [c_last_name#23, c_first_name#22, substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#24, profit#25], 100, [c_last_name#23 ASC NULLS FIRST, c_first_name#22 ASC NULLS FIRST, substr(s_city#15, 1, 30) ASC NULLS FIRST, profit#25 ASC NULLS FIRST], [c_last_name#23, c_first_name#22, substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#24, profit#25] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [6]: [c_last_name#23, c_first_name#22, substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#24, profit#25] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (35) -+- * ColumnarToRow (34) ++- * CometColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan parquet spark_catalog.default.date_dim (31) @@ -197,7 +197,7 @@ Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,20 Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/simplified.txt index b69b0cd2e9..d5514ba8a7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit,s_city] CometProject [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit,s_city] @@ -18,7 +18,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dow] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/explain.txt index f02edb1912..c6f5a077fa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (42) +* CometColumnarToRow (42) +- CometTakeOrderedAndProject (41) +- CometHashAggregate (40) +- CometExchange (39) @@ -236,14 +236,14 @@ Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] Input [2]: [s_store_name#9, sum(ss_net_profit)#21] Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#9 ASC NULLS FIRST], output=[s_store_name#9,sum(ss_net_profit)#21]), [s_store_name#9, sum(ss_net_profit)#21], 100, [s_store_name#9 ASC NULLS FIRST], [s_store_name#9, sum(ss_net_profit)#21] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarToRow [codegen id : 1] Input [2]: [s_store_name#9, sum(ss_net_profit)#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (47) -+- * ColumnarToRow (46) ++- * CometColumnarToRow (46) +- CometProject (45) +- CometFilter (44) +- CometScan parquet spark_catalog.default.date_dim (43) @@ -264,7 +264,7 @@ Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) A Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Arguments: [d_date_sk#5], [d_date_sk#5] -(46) ColumnarToRow [codegen id : 1] +(46) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (47) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/simplified.txt index 91d98aca9b..7beec478b3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [s_store_name,sum(ss_net_profit)] CometHashAggregate [s_store_name,sum(ss_net_profit),sum,sum(UnscaledValue(ss_net_profit))] @@ -16,7 +16,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/explain.txt index 65fe1a28b0..acdf464198 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/explain.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject (102) +- * Expand (98) +- Union (97) :- * HashAggregate (38) - : +- * ColumnarToRow (37) + : +- * CometColumnarToRow (37) : +- CometExchange (36) : +- CometHashAggregate (35) : +- CometProject (34) @@ -44,7 +44,7 @@ TakeOrderedAndProject (102) : +- CometFilter (30) : +- CometScan parquet spark_catalog.default.promotion (29) :- * HashAggregate (67) - : +- * ColumnarToRow (66) + : +- * CometColumnarToRow (66) : +- CometExchange (65) : +- CometHashAggregate (64) : +- CometProject (63) @@ -73,7 +73,7 @@ TakeOrderedAndProject (102) : : +- ReusedExchange (58) : +- ReusedExchange (61) +- * HashAggregate (96) - +- * ColumnarToRow (95) + +- * CometColumnarToRow (95) +- CometExchange (94) +- CometHashAggregate (93) +- CometProject (92) @@ -272,7 +272,7 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(co Input [6]: [s_store_id#17, sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] Arguments: hashpartitioning(s_store_id#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(37) ColumnarToRow [codegen id : 1] +(37) CometColumnarToRow [codegen id : 1] Input [6]: [s_store_id#17, sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] (38) HashAggregate [codegen id : 1] @@ -403,7 +403,7 @@ Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#39)), partial_sum(c Input [6]: [cp_catalog_page_id#50, sum#53, sum#54, isEmpty#55, sum#56, isEmpty#57] Arguments: hashpartitioning(cp_catalog_page_id#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(66) ColumnarToRow [codegen id : 2] +(66) CometColumnarToRow [codegen id : 2] Input [6]: [cp_catalog_page_id#50, sum#53, sum#54, isEmpty#55, sum#56, isEmpty#57] (67) HashAggregate [codegen id : 2] @@ -534,7 +534,7 @@ Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(c Input [6]: [web_site_id#81, sum#84, sum#85, isEmpty#86, sum#87, isEmpty#88] Arguments: hashpartitioning(web_site_id#81, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] -(95) ColumnarToRow [codegen id : 3] +(95) CometColumnarToRow [codegen id : 3] Input [6]: [web_site_id#81, sum#84, sum#85, isEmpty#86, sum#87, isEmpty#88] (96) HashAggregate [codegen id : 3] @@ -576,7 +576,7 @@ Arguments: 100, [channel#97 ASC NULLS FIRST, id#98 ASC NULLS FIRST], [channel#97 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (107) -+- * ColumnarToRow (106) ++- * CometColumnarToRow (106) +- CometProject (105) +- CometFilter (104) +- CometScan parquet spark_catalog.default.date_dim (103) @@ -597,7 +597,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(106) ColumnarToRow [codegen id : 1] +(106) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] (107) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt index 012c711bfe..a1ab21e6b5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Union WholeStageCodegen (1) HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [s_store_id] #2 CometHashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] @@ -31,7 +31,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] @@ -58,7 +58,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (2) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [cp_catalog_page_id] #10 CometHashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] @@ -90,7 +90,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [p_promo_sk] #9 WholeStageCodegen (3) HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [web_site_id] #14 CometHashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/explain.txt index 10e1267143..cf88aa2e38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (47) +* CometColumnarToRow (47) +- CometTakeOrderedAndProject (46) +- CometProject (45) +- CometBroadcastHashJoin (44) @@ -263,14 +263,14 @@ Arguments: [c_customer_id#29, c_salutation#31, c_first_name#32, c_last_name#33, Input [16]: [c_customer_id#29, c_salutation#31, c_first_name#32, c_last_name#33, ca_street_number#35, ca_street_name#36, ca_street_type#37, ca_suite_number#38, ca_city#39, ca_county#40, ca_state#41, ca_zip#42, ca_country#43, ca_gmt_offset#44, ca_location_type#45, ctr_total_return#13] Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#29 ASC NULLS FIRST,c_salutation#31 ASC NULLS FIRST,c_first_name#32 ASC NULLS FIRST,c_last_name#33 ASC NULLS FIRST,ca_street_number#35 ASC NULLS FIRST,ca_street_name#36 ASC NULLS FIRST,ca_street_type#37 ASC NULLS FIRST,ca_suite_number#38 ASC NULLS FIRST,ca_city#39 ASC NULLS FIRST,ca_county#40 ASC NULLS FIRST,ca_state#41 ASC NULLS FIRST,ca_zip#42 ASC NULLS FIRST,ca_country#43 ASC NULLS FIRST,ca_gmt_offset#44 ASC NULLS FIRST,ca_location_type#45 ASC NULLS FIRST,ctr_total_return#13 ASC NULLS FIRST], output=[c_customer_id#29,c_salutation#31,c_first_name#32,c_last_name#33,ca_street_number#35,ca_street_name#36,ca_street_type#37,ca_suite_number#38,ca_city#39,ca_county#40,ca_state#41,ca_zip#42,ca_country#43,ca_gmt_offset#44,ca_location_type#45,ctr_total_return#13]), [c_customer_id#29, c_salutation#31, c_first_name#32, c_last_name#33, ca_street_number#35, ca_street_name#36, ca_street_type#37, ca_suite_number#38, ca_city#39, ca_county#40, ca_state#41, ca_zip#42, ca_country#43, ca_gmt_offset#44, ca_location_type#45, ctr_total_return#13], 100, [c_customer_id#29 ASC NULLS FIRST, c_salutation#31 ASC NULLS FIRST, c_first_name#32 ASC NULLS FIRST, c_last_name#33 ASC NULLS FIRST, ca_street_number#35 ASC NULLS FIRST, ca_street_name#36 ASC NULLS FIRST, ca_street_type#37 ASC NULLS FIRST, ca_suite_number#38 ASC NULLS FIRST, ca_city#39 ASC NULLS FIRST, ca_county#40 ASC NULLS FIRST, ca_state#41 ASC NULLS FIRST, ca_zip#42 ASC NULLS FIRST, ca_country#43 ASC NULLS FIRST, ca_gmt_offset#44 ASC NULLS FIRST, ca_location_type#45 ASC NULLS FIRST, ctr_total_return#13 ASC NULLS FIRST], [c_customer_id#29, c_salutation#31, c_first_name#32, c_last_name#33, ca_street_number#35, ca_street_name#36, ca_street_type#37, ca_suite_number#38, ca_city#39, ca_county#40, ca_state#41, ca_zip#42, ca_country#43, ca_gmt_offset#44, ca_location_type#45, ctr_total_return#13] -(47) ColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [16]: [c_customer_id#29, c_salutation#31, c_first_name#32, c_last_name#33, ca_street_number#35, ca_street_name#36, ca_street_type#37, ca_suite_number#38, ca_city#39, ca_county#40, ca_state#41, ca_zip#42, ca_country#43, ca_gmt_offset#44, ca_location_type#45, ctr_total_return#13] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -291,7 +291,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/simplified.txt index b35b762107..c670e94552 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] @@ -21,7 +21,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/explain.txt index 34319a8fc3..6117d4a921 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (26) +* CometColumnarToRow (26) +- CometTakeOrderedAndProject (25) +- CometHashAggregate (24) +- CometExchange (23) @@ -145,14 +145,14 @@ Functions: [] Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#2 ASC NULLS FIRST], output=[i_item_id#2,i_item_desc#3,i_current_price#4]), [i_item_id#2, i_item_desc#3, i_current_price#4], 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#8 IN dynamicpruning#9 BroadcastExchange (31) -+- * ColumnarToRow (30) ++- * CometColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_date#11) AND (d_date#11 >= 2000-05-25)) AND (d_date#1 Input [2]: [d_date_sk#10, d_date#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt index 42c3f40e68..755128d2c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] CometHashAggregate [i_item_id,i_item_desc,i_current_price] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/explain.txt index c0dfa3fdf0..820bc9bd5c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (48) +* CometColumnarToRow (48) +- CometTakeOrderedAndProject (47) +- CometProject (46) +- CometBroadcastHashJoin (45) @@ -265,14 +265,14 @@ Arguments: [item_id#21, sr_item_qty#22, sr_dev#33, cr_item_qty#20, cr_dev#34, wr Input [8]: [item_id#21, sr_item_qty#22, sr_dev#33, cr_item_qty#20, cr_dev#34, wr_item_qty#32, wr_dev#35, average#36] Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_id#21 ASC NULLS FIRST,sr_item_qty#22 ASC NULLS FIRST], output=[item_id#21,sr_item_qty#22,sr_dev#33,cr_item_qty#20,cr_dev#34,wr_item_qty#32,wr_dev#35,average#36]), [item_id#21, sr_item_qty#22, sr_dev#33, cr_item_qty#20, cr_dev#34, wr_item_qty#32, wr_dev#35, average#36], 100, [item_id#21 ASC NULLS FIRST, sr_item_qty#22 ASC NULLS FIRST], [item_id#21, sr_item_qty#22, sr_dev#33, cr_item_qty#20, cr_dev#34, wr_item_qty#32, wr_dev#35, average#36] -(48) ColumnarToRow [codegen id : 1] +(48) CometColumnarToRow [codegen id : 1] Input [8]: [item_id#21, sr_item_qty#22, sr_dev#33, cr_item_qty#20, cr_dev#34, wr_item_qty#32, wr_dev#35, average#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 BroadcastExchange (62) -+- * ColumnarToRow (61) ++- * CometColumnarToRow (61) +- CometProject (60) +- CometBroadcastHashJoin (59) :- CometFilter (50) @@ -344,7 +344,7 @@ Arguments: [d_date#8], [d_date#9], LeftSemi, BuildRight Input [2]: [d_date_sk#7, d_date#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(61) ColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (62) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/simplified.txt index 77345d28a7..0530aad2f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] CometProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] @@ -18,7 +18,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometBroadcastHashJoin [d_date_sk,d_date,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/explain.txt index 572fd7a66f..ca784ef3e7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (32) +- * Project (31) - +- * ColumnarToRow (30) + +- * CometColumnarToRow (30) +- CometBroadcastHashJoin (29) :- CometBroadcastExchange (25) : +- CometProject (24) @@ -172,7 +172,7 @@ Left output [4]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9] Right output [1]: [sr_cdemo_sk#15] Arguments: [cd_demo_sk#9], [sr_cdemo_sk#15], Inner, BuildLeft -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [5]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9, sr_cdemo_sk#15] (31) Project [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt index 646285a082..3101b29dd5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] WholeStageCodegen (1) Project [c_customer_id,c_last_name,c_first_name] - ColumnarToRow + CometColumnarToRow InputAdapter CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/explain.txt index c03da66a07..020414c4d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (45) +* CometColumnarToRow (45) +- CometTakeOrderedAndProject (44) +- CometHashAggregate (43) +- CometExchange (42) @@ -256,14 +256,14 @@ Functions [3]: [avg(ws_quantity#4), avg(UnscaledValue(wr_refunded_cash#16)), avg Input [4]: [substr(r_reason_desc, 1, 20)#38, avg(ws_quantity)#39, avg(wr_refunded_cash)#40, avg(wr_fee)#41] Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(r_reason_desc, 1, 20)#38 ASC NULLS FIRST,avg(ws_quantity)#39 ASC NULLS FIRST,avg(wr_refunded_cash)#40 ASC NULLS FIRST,avg(wr_fee)#41 ASC NULLS FIRST], output=[substr(r_reason_desc, 1, 20)#38,avg(ws_quantity)#39,avg(wr_refunded_cash)#40,avg(wr_fee)#41]), [substr(r_reason_desc, 1, 20)#38, avg(ws_quantity)#39, avg(wr_refunded_cash)#40, avg(wr_fee)#41], 100, [substr(r_reason_desc, 1, 20)#38 ASC NULLS FIRST, avg(ws_quantity)#39 ASC NULLS FIRST, avg(wr_refunded_cash)#40 ASC NULLS FIRST, avg(wr_fee)#41 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#38, avg(ws_quantity)#39, avg(wr_refunded_cash)#40, avg(wr_fee)#41] -(45) ColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [4]: [substr(r_reason_desc, 1, 20)#38, avg(ws_quantity)#39, avg(wr_refunded_cash)#40, avg(wr_fee)#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (50) -+- * ColumnarToRow (49) ++- * CometColumnarToRow (49) +- CometProject (48) +- CometFilter (47) +- CometScan parquet spark_catalog.default.date_dim (46) @@ -284,7 +284,7 @@ Condition : ((isnotnull(d_year#29) AND (d_year#29 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#28, d_year#29] Arguments: [d_date_sk#28], [d_date_sk#28] -(49) ColumnarToRow [codegen id : 1] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#28] (50) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/simplified.txt index 4e4ece1bda..28752f3783 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] CometHashAggregate [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),r_reason_desc,sum,count,sum,count,sum,count,avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee))] @@ -25,7 +25,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/explain.txt index cf40b3e8d5..e7ce31dd2b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (23) +- * Project (22) +- Window (21) - +- * ColumnarToRow (20) + +- * CometColumnarToRow (20) +- CometSort (19) +- CometExchange (18) +- CometHashAggregate (17) @@ -114,7 +114,7 @@ Arguments: hashpartitioning(_w1#17, _w2#18, 5), ENSURE_REQUIREMENTS, CometNative Input [7]: [total_sum#14, i_category#10, i_class#11, lochierarchy#15, _w0#16, _w1#17, _w2#18] Arguments: [total_sum#14, i_category#10, i_class#11, lochierarchy#15, _w0#16, _w1#17, _w2#18], [_w1#17 ASC NULLS FIRST, _w2#18 ASC NULLS FIRST, _w0#16 DESC NULLS LAST] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarToRow [codegen id : 1] Input [7]: [total_sum#14, i_category#10, i_class#11, lochierarchy#15, _w0#16, _w1#17, _w2#18] (21) Window @@ -133,7 +133,7 @@ Arguments: 100, [lochierarchy#15 DESC NULLS LAST, CASE WHEN (lochierarchy#15 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * ColumnarToRow (27) ++- * CometColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(27) ColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/simplified.txt index 86480d58c6..b1bf4c2e38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/simplified.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl InputAdapter Window [_w0,_w1,_w2] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] CometExchange [_w1,_w2] #1 @@ -21,7 +21,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/explain.txt index 28b942e2f7..5081f68284 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/explain.txt @@ -5,7 +5,7 @@ +- * Project (46) +- * BroadcastHashJoin LeftAnti BuildRight (45) :- * BroadcastHashJoin LeftAnti BuildRight (31) - : :- * ColumnarToRow (17) + : :- * CometColumnarToRow (17) : : +- CometHashAggregate (16) : : +- CometExchange (15) : : +- CometHashAggregate (14) @@ -23,7 +23,7 @@ : : +- CometFilter (10) : : +- CometScan parquet spark_catalog.default.customer (9) : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) + : +- * CometColumnarToRow (29) : +- CometHashAggregate (28) : +- CometExchange (27) : +- CometHashAggregate (26) @@ -36,7 +36,7 @@ : : +- ReusedExchange (20) : +- ReusedExchange (23) +- BroadcastExchange (44) - +- * ColumnarToRow (43) + +- * CometColumnarToRow (43) +- CometHashAggregate (42) +- CometExchange (41) +- CometHashAggregate (40) @@ -128,7 +128,7 @@ Input [3]: [c_last_name#9, c_first_name#8, d_date#5] Keys [3]: [c_last_name#9, c_first_name#8, d_date#5] Functions: [] -(17) ColumnarToRow [codegen id : 3] +(17) CometColumnarToRow [codegen id : 3] Input [3]: [c_last_name#9, c_first_name#8, d_date#5] (18) CometScan parquet spark_catalog.default.catalog_sales @@ -181,7 +181,7 @@ Input [3]: [c_last_name#17, c_first_name#16, d_date#14] Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] Functions: [] -(29) ColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [3]: [c_last_name#17, c_first_name#16, d_date#14] (30) BroadcastExchange @@ -244,7 +244,7 @@ Input [3]: [c_last_name#25, c_first_name#24, d_date#22] Keys [3]: [c_last_name#25, c_first_name#24, d_date#22] Functions: [] -(43) ColumnarToRow [codegen id : 2] +(43) CometColumnarToRow [codegen id : 2] Input [3]: [c_last_name#25, c_first_name#24, d_date#22] (44) BroadcastExchange @@ -283,7 +283,7 @@ Results [1]: [count(1)#28 AS count(1)#29] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 BroadcastExchange (54) -+- * ColumnarToRow (53) ++- * CometColumnarToRow (53) +- CometProject (52) +- CometFilter (51) +- CometScan parquet spark_catalog.default.date_dim (50) @@ -304,7 +304,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(53) ColumnarToRow [codegen id : 1] +(53) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] (54) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt index 6e3328ae1d..81cd6f40be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt @@ -7,7 +7,7 @@ WholeStageCodegen (4) Project BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometExchange [c_last_name,c_first_name,d_date] #2 @@ -21,7 +21,7 @@ WholeStageCodegen (4) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] @@ -36,7 +36,7 @@ WholeStageCodegen (4) InputAdapter BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometExchange [c_last_name,c_first_name,d_date] #7 @@ -53,7 +53,7 @@ WholeStageCodegen (4) InputAdapter BroadcastExchange #8 WholeStageCodegen (2) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometExchange [c_last_name,c_first_name,d_date] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/explain.txt index 592e23cd27..6682e151f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/explain.txt @@ -6,7 +6,7 @@ : : : :- * BroadcastNestedLoopJoin Inner BuildRight (88) : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (67) : : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (46) -: : : : : : :- * ColumnarToRow (25) +: : : : : : :- * CometColumnarToRow (25) : : : : : : : +- CometHashAggregate (24) : : : : : : : +- CometExchange (23) : : : : : : : +- CometHashAggregate (22) @@ -32,7 +32,7 @@ : : : : : : : +- CometFilter (17) : : : : : : : +- CometScan parquet spark_catalog.default.store (16) : : : : : : +- BroadcastExchange (45) -: : : : : : +- * ColumnarToRow (44) +: : : : : : +- * CometColumnarToRow (44) : : : : : : +- CometHashAggregate (43) : : : : : : +- CometExchange (42) : : : : : : +- CometHashAggregate (41) @@ -52,7 +52,7 @@ : : : : : : : +- CometScan parquet spark_catalog.default.time_dim (32) : : : : : : +- ReusedExchange (38) : : : : : +- BroadcastExchange (66) -: : : : : +- * ColumnarToRow (65) +: : : : : +- * CometColumnarToRow (65) : : : : : +- CometHashAggregate (64) : : : : : +- CometExchange (63) : : : : : +- CometHashAggregate (62) @@ -72,7 +72,7 @@ : : : : : : +- CometScan parquet spark_catalog.default.time_dim (53) : : : : : +- ReusedExchange (59) : : : : +- BroadcastExchange (87) -: : : : +- * ColumnarToRow (86) +: : : : +- * CometColumnarToRow (86) : : : : +- CometHashAggregate (85) : : : : +- CometExchange (84) : : : : +- CometHashAggregate (83) @@ -92,7 +92,7 @@ : : : : : +- CometScan parquet spark_catalog.default.time_dim (74) : : : : +- ReusedExchange (80) : : : +- BroadcastExchange (108) -: : : +- * ColumnarToRow (107) +: : : +- * CometColumnarToRow (107) : : : +- CometHashAggregate (106) : : : +- CometExchange (105) : : : +- CometHashAggregate (104) @@ -112,7 +112,7 @@ : : : : +- CometScan parquet spark_catalog.default.time_dim (95) : : : +- ReusedExchange (101) : : +- BroadcastExchange (129) -: : +- * ColumnarToRow (128) +: : +- * CometColumnarToRow (128) : : +- CometHashAggregate (127) : : +- CometExchange (126) : : +- CometHashAggregate (125) @@ -132,7 +132,7 @@ : : : +- CometScan parquet spark_catalog.default.time_dim (116) : : +- ReusedExchange (122) : +- BroadcastExchange (150) -: +- * ColumnarToRow (149) +: +- * CometColumnarToRow (149) : +- CometHashAggregate (148) : +- CometExchange (147) : +- CometHashAggregate (146) @@ -152,7 +152,7 @@ : : +- CometScan parquet spark_catalog.default.time_dim (137) : +- ReusedExchange (143) +- BroadcastExchange (171) - +- * ColumnarToRow (170) + +- * CometColumnarToRow (170) +- CometHashAggregate (169) +- CometExchange (168) +- CometHashAggregate (167) @@ -285,7 +285,7 @@ Input [1]: [count#13] Keys: [] Functions [1]: [count(1)] -(25) ColumnarToRow [codegen id : 8] +(25) CometColumnarToRow [codegen id : 8] Input [1]: [h8_30_to_9#14] (26) CometScan parquet spark_catalog.default.store_sales @@ -368,7 +368,7 @@ Input [1]: [count#24] Keys: [] Functions [1]: [count(1)] -(44) ColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [h9_to_9_30#25] (45) BroadcastExchange @@ -459,7 +459,7 @@ Input [1]: [count#35] Keys: [] Functions [1]: [count(1)] -(65) ColumnarToRow [codegen id : 2] +(65) CometColumnarToRow [codegen id : 2] Input [1]: [h9_30_to_10#36] (66) BroadcastExchange @@ -550,7 +550,7 @@ Input [1]: [count#46] Keys: [] Functions [1]: [count(1)] -(86) ColumnarToRow [codegen id : 3] +(86) CometColumnarToRow [codegen id : 3] Input [1]: [h10_to_10_30#47] (87) BroadcastExchange @@ -641,7 +641,7 @@ Input [1]: [count#57] Keys: [] Functions [1]: [count(1)] -(107) ColumnarToRow [codegen id : 4] +(107) CometColumnarToRow [codegen id : 4] Input [1]: [h10_30_to_11#58] (108) BroadcastExchange @@ -732,7 +732,7 @@ Input [1]: [count#68] Keys: [] Functions [1]: [count(1)] -(128) ColumnarToRow [codegen id : 5] +(128) CometColumnarToRow [codegen id : 5] Input [1]: [h11_to_11_30#69] (129) BroadcastExchange @@ -823,7 +823,7 @@ Input [1]: [count#79] Keys: [] Functions [1]: [count(1)] -(149) ColumnarToRow [codegen id : 6] +(149) CometColumnarToRow [codegen id : 6] Input [1]: [h11_30_to_12#80] (150) BroadcastExchange @@ -914,7 +914,7 @@ Input [1]: [count#90] Keys: [] Functions [1]: [count(1)] -(170) ColumnarToRow [codegen id : 7] +(170) CometColumnarToRow [codegen id : 7] Input [1]: [h12_to_12_30#91] (171) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/simplified.txt index 1bb61b6c92..21b72d36a7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/simplified.txt @@ -6,7 +6,7 @@ WholeStageCodegen (8) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h8_30_to_9,count,count(1)] CometExchange #1 @@ -35,7 +35,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h9_to_9_30,count,count(1)] CometExchange #6 @@ -58,7 +58,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #8 WholeStageCodegen (2) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h9_30_to_10,count,count(1)] CometExchange #9 @@ -81,7 +81,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #11 WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h10_to_10_30,count,count(1)] CometExchange #12 @@ -104,7 +104,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #14 WholeStageCodegen (4) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h10_30_to_11,count,count(1)] CometExchange #15 @@ -127,7 +127,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #17 WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h11_to_11_30,count,count(1)] CometExchange #18 @@ -150,7 +150,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #20 WholeStageCodegen (6) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h11_30_to_12,count,count(1)] CometExchange #21 @@ -173,7 +173,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #23 WholeStageCodegen (7) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h12_to_12_30,count,count(1)] CometExchange #24 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/explain.txt index 143ff08683..8346f9b848 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (28) +- * Project (27) +- * Filter (26) +- Window (25) - +- * ColumnarToRow (24) + +- * CometColumnarToRow (24) +- CometSort (23) +- CometExchange (22) +- CometHashAggregate (21) @@ -139,7 +139,7 @@ Arguments: hashpartitioning(i_category#4, i_brand#2, s_store_name#14, s_company_ Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#17, _w0#18] Arguments: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#17, _w0#18], [i_category#4 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST, s_company_name#15 ASC NULLS FIRST] -(24) ColumnarToRow [codegen id : 1] +(24) CometColumnarToRow [codegen id : 1] Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#17, _w0#18] (25) Window @@ -162,7 +162,7 @@ Arguments: 100, [(sum_sales#17 - avg_monthly_sales#19) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (33) -+- * ColumnarToRow (32) ++- * CometColumnarToRow (32) +- CometProject (31) +- CometFilter (30) +- CometScan parquet spark_catalog.default.date_dim (29) @@ -183,7 +183,7 @@ Condition : ((isnotnull(d_year#11) AND (d_year#11 = 1999)) AND isnotnull(d_date_ Input [3]: [d_date_sk#10, d_year#11, d_moy#12] Arguments: [d_date_sk#10, d_moy#12], [d_date_sk#10, d_moy#12] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_moy#12] (33) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/simplified.txt index 97b9563be6..58bb7e979e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 @@ -26,7 +26,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/explain.txt index 7cc8c153c4..d2061bd1d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == * Project (4) -+- * ColumnarToRow (3) ++- * CometColumnarToRow (3) +- CometFilter (2) +- CometScan parquet spark_catalog.default.reason (1) @@ -16,7 +16,7 @@ ReadSchema: struct Input [1]: [r_reason_sk#1] Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) -(3) ColumnarToRow [codegen id : 1] +(3) CometColumnarToRow [codegen id : 1] Input [1]: [r_reason_sk#1] (4) Project [codegen id : 1] @@ -26,7 +26,7 @@ Input [1]: [r_reason_sk#1] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#2, [id=#3] -* ColumnarToRow (12) +* CometColumnarToRow (12) +- CometProject (11) +- CometHashAggregate (10) +- CometExchange (9) @@ -69,7 +69,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(Unscal Input [3]: [count(1)#26, avg(ss_ext_discount_amt)#27, avg(ss_net_paid)#28] Arguments: [mergedValue#29], [named_struct(count(1), count(1)#26, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#27, avg(ss_net_paid), avg(ss_net_paid)#28) AS mergedValue#29] -(12) ColumnarToRow [codegen id : 1] +(12) CometColumnarToRow [codegen id : 1] Input [1]: [mergedValue#29] Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] @@ -77,7 +77,7 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] -* ColumnarToRow (20) +* CometColumnarToRow (20) +- CometProject (19) +- CometHashAggregate (18) +- CometExchange (17) @@ -120,7 +120,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#31)), avg(Unscal Input [3]: [count(1)#39, avg(ss_ext_discount_amt)#40, avg(ss_net_paid)#41] Arguments: [mergedValue#42], [named_struct(count(1), count(1)#39, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#40, avg(ss_net_paid), avg(ss_net_paid)#41) AS mergedValue#42] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarToRow [codegen id : 1] Input [1]: [mergedValue#42] Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] @@ -128,7 +128,7 @@ Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* ColumnarToRow (28) +* CometColumnarToRow (28) +- CometProject (27) +- CometHashAggregate (26) +- CometExchange (25) @@ -171,7 +171,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#44)), avg(Unscal Input [3]: [count(1)#52, avg(ss_ext_discount_amt)#53, avg(ss_net_paid)#54] Arguments: [mergedValue#55], [named_struct(count(1), count(1)#52, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#53, avg(ss_net_paid), avg(ss_net_paid)#54) AS mergedValue#55] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [mergedValue#55] Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] @@ -179,7 +179,7 @@ Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* ColumnarToRow (36) +* CometColumnarToRow (36) +- CometProject (35) +- CometHashAggregate (34) +- CometExchange (33) @@ -222,7 +222,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#57)), avg(Unscal Input [3]: [count(1)#65, avg(ss_ext_discount_amt)#66, avg(ss_net_paid)#67] Arguments: [mergedValue#68], [named_struct(count(1), count(1)#65, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#66, avg(ss_net_paid), avg(ss_net_paid)#67) AS mergedValue#68] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [1]: [mergedValue#68] Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] @@ -230,7 +230,7 @@ Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] -* ColumnarToRow (44) +* CometColumnarToRow (44) +- CometProject (43) +- CometHashAggregate (42) +- CometExchange (41) @@ -273,7 +273,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#70)), avg(Unscal Input [3]: [count(1)#78, avg(ss_ext_discount_amt)#79, avg(ss_net_paid)#80] Arguments: [mergedValue#81], [named_struct(count(1), count(1)#78, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#79, avg(ss_net_paid), avg(ss_net_paid)#80) AS mergedValue#81] -(44) ColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [mergedValue#81] Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/simplified.txt index 3d69c60d08..68961382c0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) Project Subquery #1 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] CometHashAggregate [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count,count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] @@ -15,7 +15,7 @@ WholeStageCodegen (1) ReusedSubquery [mergedValue] #1 Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] CometHashAggregate [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count,count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] @@ -28,7 +28,7 @@ WholeStageCodegen (1) ReusedSubquery [mergedValue] #2 Subquery #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] CometHashAggregate [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count,count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] @@ -41,7 +41,7 @@ WholeStageCodegen (1) ReusedSubquery [mergedValue] #3 Subquery #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] CometHashAggregate [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count,count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] @@ -54,7 +54,7 @@ WholeStageCodegen (1) ReusedSubquery [mergedValue] #4 Subquery #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] CometHashAggregate [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count,count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] @@ -65,7 +65,7 @@ WholeStageCodegen (1) CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [r_reason_sk] CometScan parquet spark_catalog.default.reason [r_reason_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/explain.txt index a6ec6f4b96..38379fb61c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * Project (47) +- * BroadcastNestedLoopJoin Inner BuildRight (46) - :- * ColumnarToRow (25) + :- * CometColumnarToRow (25) : +- CometHashAggregate (24) : +- CometExchange (23) : +- CometHashAggregate (22) @@ -27,7 +27,7 @@ : +- CometFilter (17) : +- CometScan parquet spark_catalog.default.web_page (16) +- BroadcastExchange (45) - +- * ColumnarToRow (44) + +- * CometColumnarToRow (44) +- CometHashAggregate (43) +- CometExchange (42) +- CometHashAggregate (41) @@ -160,7 +160,7 @@ Input [1]: [count#11] Keys: [] Functions [1]: [count(1)] -(25) ColumnarToRow [codegen id : 2] +(25) CometColumnarToRow [codegen id : 2] Input [1]: [amc#12] (26) CometScan parquet spark_catalog.default.web_sales @@ -243,7 +243,7 @@ Input [1]: [count#21] Keys: [] Functions [1]: [count(1)] -(44) ColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [pmc#22] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/simplified.txt index 95fd73d86e..72e45a9ad1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (2) Project [amc,pmc] BroadcastNestedLoopJoin - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [amc,count,count(1)] CometExchange #1 @@ -30,7 +30,7 @@ WholeStageCodegen (2) InputAdapter BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [pmc,count,count(1)] CometExchange #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/explain.txt index ede0953066..26272decdb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (41) +* CometColumnarToRow (41) +- CometSort (40) +- CometColumnarExchange (39) +- CometHashAggregate (38) @@ -232,14 +232,14 @@ Arguments: rangepartitioning(Returns_Loss#28 DESC NULLS LAST, 5), ENSURE_REQUIRE Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] Arguments: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28], [Returns_Loss#28 DESC NULLS LAST] -(41) ColumnarToRow [codegen id : 1] +(41) CometColumnarToRow [codegen id : 1] Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = cr_returned_date_sk#8 IN dynamicpruning#9 BroadcastExchange (46) -+- * ColumnarToRow (45) ++- * CometColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometScan parquet spark_catalog.default.date_dim (42) @@ -260,7 +260,7 @@ Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#12)) AND (d_year#11 = 1 Input [3]: [d_date_sk#10, d_year#11, d_moy#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(45) ColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt index 0dc197b1b4..1847051c7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] CometColumnarExchange [Returns_Loss] #1 @@ -26,7 +26,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/explain.txt index 93bb03fae1..25045775b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (30) +* CometColumnarToRow (30) +- CometHashAggregate (29) +- CometExchange (28) +- CometHashAggregate (27) @@ -168,14 +168,14 @@ Input [1]: [sum#17] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [1]: [Excess Discount Amount #18] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (35) -+- * ColumnarToRow (34) ++- * CometColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan parquet spark_catalog.default.date_dim (31) @@ -196,7 +196,7 @@ Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#1 Input [2]: [d_date_sk#16, d_date#19] Arguments: [d_date_sk#16], [d_date_sk#16] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/simplified.txt index f49dd1ea30..4ed5b0b8ef 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [Excess Discount Amount ,sum,sum(UnscaledValue(ws_ext_discount_amt))] CometExchange #1 @@ -15,7 +15,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/explain.txt index dc64f3c4ce..ae826e9288 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (22) +* CometColumnarToRow (22) +- CometTakeOrderedAndProject (21) +- CometHashAggregate (20) +- CometExchange (19) @@ -119,6 +119,6 @@ Functions [1]: [sum(act_sales#14)] Input [2]: [ss_customer_sk#2, sumsales#17] Arguments: TakeOrderedAndProject(limit=100, orderBy=[sumsales#17 ASC NULLS FIRST,ss_customer_sk#2 ASC NULLS FIRST], output=[ss_customer_sk#2,sumsales#17]), [ss_customer_sk#2, sumsales#17], 100, [sumsales#17 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#17] -(22) ColumnarToRow [codegen id : 1] +(22) CometColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#2, sumsales#17] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/simplified.txt index 6795d7e399..9580e56671 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [ss_customer_sk,sumsales] CometHashAggregate [ss_customer_sk,sumsales,sum,isEmpty,sum(act_sales)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/explain.txt index 4dd9246cd7..338558d002 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/explain.txt @@ -3,7 +3,7 @@ +- Exchange (39) +- * HashAggregate (38) +- * HashAggregate (37) - +- * ColumnarToRow (36) + +- * CometColumnarToRow (36) +- CometHashAggregate (35) +- CometProject (34) +- CometBroadcastHashJoin (33) @@ -203,7 +203,7 @@ Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [3]: [ws_order_number#5, sum#20, sum#21] (37) HashAggregate [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/simplified.txt index 601f577da9..8b7b457074 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/simplified.txt @@ -5,7 +5,7 @@ WholeStageCodegen (2) WholeStageCodegen (1) HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [ws_order_number,sum,sum,ws_ext_ship_cost,ws_net_profit] CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/explain.txt index 986abf83ca..f3f0204103 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/explain.txt @@ -3,7 +3,7 @@ +- Exchange (52) +- * HashAggregate (51) +- * HashAggregate (50) - +- * ColumnarToRow (49) + +- * CometColumnarToRow (49) +- CometHashAggregate (48) +- CometProject (47) +- CometBroadcastHashJoin (46) @@ -270,7 +270,7 @@ Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Keys [1]: [ws_order_number#4] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] -(49) ColumnarToRow [codegen id : 1] +(49) CometColumnarToRow [codegen id : 1] Input [3]: [ws_order_number#4, sum#25, sum#26] (50) HashAggregate [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/simplified.txt index 168f353a7b..021211d58f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/simplified.txt @@ -5,7 +5,7 @@ WholeStageCodegen (2) WholeStageCodegen (1) HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [ws_order_number,sum,sum,ws_ext_ship_cost,ws_net_profit] CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/explain.txt index d2e63bee25..677abdadae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (25) +* CometColumnarToRow (25) +- CometHashAggregate (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -138,6 +138,6 @@ Input [1]: [count#12] Keys: [] Functions [1]: [count(1)] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [1]: [count(1)#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/simplified.txt index d9a87aa3c4..9e86a025ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [count(1),count,count(1)] CometExchange #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/explain.txt index 1c6e9b78c0..f23cf90e66 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (25) +* CometColumnarToRow (25) +- CometHashAggregate (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -139,14 +139,14 @@ Input [3]: [sum#16, sum#17, sum#18] Keys: [] Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [3]: [store_only#19, catalog_only#20, store_and_catalog#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (30) -+- * ColumnarToRow (29) ++- * CometColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan parquet spark_catalog.default.date_dim (26) @@ -167,7 +167,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(29) ColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/simplified.txt index 0036a4bd84..aa5be145e9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [store_only,catalog_only,store_and_catalog,sum,sum,sum,sum(CASE WHEN (isnotnull(customer_sk) AND isnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnotnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END)] CometExchange #1 @@ -16,7 +16,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/explain.txt index 593065ba0b..8c2521b90c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/explain.txt @@ -1,11 +1,11 @@ == Physical Plan == -* ColumnarToRow (25) +* CometColumnarToRow (25) +- CometProject (24) +- CometSort (23) +- CometColumnarExchange (22) +- * Project (21) +- Window (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometSort (18) +- CometExchange (17) +- CometHashAggregate (16) @@ -112,7 +112,7 @@ Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometNativeShuff Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6], [i_class#9 ASC NULLS FIRST] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] (20) Window @@ -135,14 +135,14 @@ Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemreve Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, i_item_id#6] Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] -(25) ColumnarToRow [codegen id : 3] +(25) CometColumnarToRow [codegen id : 3] Input [6]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (30) -+- * ColumnarToRow (29) ++- * CometColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan parquet spark_catalog.default.date_dim (26) @@ -163,7 +163,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(29) ColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/simplified.txt index 01aa117344..a77e84b60b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] @@ -9,7 +9,7 @@ WholeStageCodegen (3) InputAdapter Window [_w0,i_class] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] CometExchange [i_class] #2 @@ -25,7 +25,7 @@ WholeStageCodegen (3) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/explain.txt index 6dfcf8b322..b4c43d5d01 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (28) +* CometColumnarToRow (28) +- CometTakeOrderedAndProject (27) +- CometHashAggregate (26) +- CometExchange (25) @@ -158,6 +158,6 @@ Functions [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) TH Input [8]: [substr(w_warehouse_name, 1, 20)#20, sm_type#9, cc_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25] Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#20 ASC NULLS FIRST,sm_type#9 ASC NULLS FIRST,cc_name#11 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#20,sm_type#9,cc_name#11,30 days #21,31 - 60 days #22,61 - 90 days #23,91 - 120 days #24,>120 days #25]), [substr(w_warehouse_name, 1, 20)#20, sm_type#9, cc_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25], 100, [substr(w_warehouse_name, 1, 20)#20 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#20, sm_type#9, cc_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [8]: [substr(w_warehouse_name, 1, 20)#20, sm_type#9, cc_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/simplified.txt index 51599575db..e112972b12 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] CometHashAggregate [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,sum,sum,sum,sum,sum,sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 30) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 60) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 90) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) > 120) THEN 1 ELSE 0 END)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/explain.txt index d47c4341a4..5cde73d6ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/explain.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject (43) : : : +- * HashAggregate (12) : : : +- Exchange (11) : : : +- * HashAggregate (10) - : : : +- * ColumnarToRow (9) + : : : +- * CometColumnarToRow (9) : : : +- CometProject (8) : : : +- CometBroadcastHashJoin (7) : : : :- CometFilter (2) @@ -27,19 +27,19 @@ TakeOrderedAndProject (43) : : +- * HashAggregate (22) : : +- Exchange (21) : : +- * HashAggregate (20) - : : +- * ColumnarToRow (19) + : : +- * CometColumnarToRow (19) : : +- CometProject (18) : : +- CometBroadcastHashJoin (17) : : :- CometFilter (15) : : : +- CometScan parquet spark_catalog.default.store_returns (14) : : +- ReusedExchange (16) : +- BroadcastExchange (34) - : +- * ColumnarToRow (33) + : +- * CometColumnarToRow (33) : +- CometProject (32) : +- CometFilter (31) : +- CometScan parquet spark_catalog.default.store (30) +- BroadcastExchange (40) - +- * ColumnarToRow (39) + +- * CometColumnarToRow (39) +- CometFilter (38) +- CometScan parquet spark_catalog.default.customer (37) @@ -84,7 +84,7 @@ Arguments: [sr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6] Arguments: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3], [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] -(9) ColumnarToRow [codegen id : 1] +(9) CometColumnarToRow [codegen id : 1] Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] (10) HashAggregate [codegen id : 1] @@ -133,7 +133,7 @@ Arguments: [sr_returned_date_sk#17], [d_date_sk#19], Inner, BuildRight Input [5]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17, d_date_sk#19] Arguments: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16], [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16] -(19) ColumnarToRow [codegen id : 2] +(19) CometColumnarToRow [codegen id : 2] Input [3]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16] (20) HashAggregate [codegen id : 2] @@ -205,7 +205,7 @@ Condition : ((isnotnull(s_state#31) AND (s_state#31 = TN)) AND isnotnull(s_store Input [2]: [s_store_sk#30, s_state#31] Arguments: [s_store_sk#30], [s_store_sk#30] -(33) ColumnarToRow [codegen id : 5] +(33) CometColumnarToRow [codegen id : 5] Input [1]: [s_store_sk#30] (34) BroadcastExchange @@ -233,7 +233,7 @@ ReadSchema: struct Input [2]: [c_customer_sk#32, c_customer_id#33] Condition : isnotnull(c_customer_sk#32) -(39) ColumnarToRow [codegen id : 6] +(39) CometColumnarToRow [codegen id : 6] Input [2]: [c_customer_sk#32, c_customer_id#33] (40) BroadcastExchange @@ -258,7 +258,7 @@ Arguments: 100, [c_customer_id#33 ASC NULLS FIRST], [c_customer_id#33] Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (48) -+- * ColumnarToRow (47) ++- * CometColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometScan parquet spark_catalog.default.date_dim (44) @@ -279,7 +279,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(47) ColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (48) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt index 43bad5d70e..135f48fb79 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt @@ -12,7 +12,7 @@ TakeOrderedAndProject [c_customer_id] Exchange [sr_customer_sk,sr_store_sk] #1 WholeStageCodegen (1) HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] @@ -21,7 +21,7 @@ TakeOrderedAndProject [c_customer_id] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -44,7 +44,7 @@ TakeOrderedAndProject [c_customer_id] Exchange [sr_customer_sk,sr_store_sk] #6 WholeStageCodegen (2) HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] @@ -55,7 +55,7 @@ TakeOrderedAndProject [c_customer_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [s_store_sk] CometFilter [s_store_sk,s_state] @@ -63,7 +63,7 @@ TakeOrderedAndProject [c_customer_id] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/explain.txt index e32e86f270..7544b94314 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/explain.txt @@ -11,7 +11,7 @@ TakeOrderedAndProject (45) : : +- * Filter (27) : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * ColumnarToRow (12) + : : : :- * CometColumnarToRow (12) : : : : +- CometBroadcastHashJoin (11) : : : : :- CometFilter (2) : : : : : +- CometScan parquet spark_catalog.default.customer (1) @@ -24,24 +24,24 @@ TakeOrderedAndProject (45) : : : : +- CometFilter (5) : : : : +- CometScan parquet spark_catalog.default.date_dim (4) : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) + : : : +- * CometColumnarToRow (17) : : : +- CometProject (16) : : : +- CometBroadcastHashJoin (15) : : : :- CometScan parquet spark_catalog.default.web_sales (13) : : : +- ReusedExchange (14) : : +- BroadcastExchange (25) - : : +- * ColumnarToRow (24) + : : +- * CometColumnarToRow (24) : : +- CometProject (23) : : +- CometBroadcastHashJoin (22) : : :- CometScan parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (33) - : +- * ColumnarToRow (32) + : +- * CometColumnarToRow (32) : +- CometProject (31) : +- CometFilter (30) : +- CometScan parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (39) - +- * ColumnarToRow (38) + +- * CometColumnarToRow (38) +- CometFilter (37) +- CometScan parquet spark_catalog.default.customer_demographics (36) @@ -101,7 +101,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) ColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 5] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan parquet spark_catalog.default.web_sales @@ -123,7 +123,7 @@ Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#12] (18) BroadcastExchange @@ -155,7 +155,7 @@ Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [1]: [cs_ship_customer_sk#16] (25) BroadcastExchange @@ -191,7 +191,7 @@ Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana Input [2]: [ca_address_sk#20, ca_county#21] Arguments: [ca_address_sk#20], [ca_address_sk#20] -(32) ColumnarToRow [codegen id : 3] +(32) CometColumnarToRow [codegen id : 3] Input [1]: [ca_address_sk#20] (33) BroadcastExchange @@ -219,7 +219,7 @@ ReadSchema: struct Input [2]: [d_date_sk#14, d_year#15] Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#14)) -(75) ColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] (76) BroadcastExchange @@ -456,7 +456,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#33 BroadcastExchange (80) -+- * ColumnarToRow (79) ++- * CometColumnarToRow (79) +- CometFilter (78) +- CometScan parquet spark_catalog.default.date_dim (77) @@ -472,7 +472,7 @@ ReadSchema: struct Input [2]: [d_date_sk#34, d_year#35] Condition : ((isnotnull(d_year#35) AND (d_year#35 = 2002)) AND isnotnull(d_date_sk#34)) -(79) ColumnarToRow [codegen id : 1] +(79) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#34, d_year#35] (80) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt index 240d88bb7c..c288c3ab55 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt @@ -12,7 +12,7 @@ TakeOrderedAndProject [customer_preferred_cust_flag] Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 WholeStageCodegen (1) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] @@ -26,7 +26,7 @@ TakeOrderedAndProject [customer_preferred_cust_flag] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -41,7 +41,7 @@ TakeOrderedAndProject [customer_preferred_cust_flag] Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 WholeStageCodegen (2) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] @@ -55,7 +55,7 @@ TakeOrderedAndProject [customer_preferred_cust_flag] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -71,7 +71,7 @@ TakeOrderedAndProject [customer_preferred_cust_flag] Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 WholeStageCodegen (4) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] @@ -92,7 +92,7 @@ TakeOrderedAndProject [customer_preferred_cust_flag] Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 WholeStageCodegen (6) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt index 7548befa10..9d67b9354e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt @@ -7,7 +7,7 @@ TakeOrderedAndProject (22) +- * HashAggregate (17) +- Exchange (16) +- * HashAggregate (15) - +- * ColumnarToRow (14) + +- * CometColumnarToRow (14) +- CometProject (13) +- CometBroadcastHashJoin (12) :- CometProject (7) @@ -87,7 +87,7 @@ Arguments: [ws_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] Arguments: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [6]: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] (15) HashAggregate [codegen id : 1] @@ -132,7 +132,7 @@ Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_ite Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (27) -+- * ColumnarToRow (26) ++- * CometColumnarToRow (26) +- CometProject (25) +- CometFilter (24) +- CometScan parquet spark_catalog.default.date_dim (23) @@ -153,7 +153,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (27) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt index 1640fa3b02..81bd3648d3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 WholeStageCodegen (1) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] @@ -24,7 +24,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/explain.txt index b4a7b87249..560452cab7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/explain.txt @@ -2,7 +2,7 @@ * HashAggregate (33) +- Exchange (32) +- * HashAggregate (31) - +- * ColumnarToRow (30) + +- * CometColumnarToRow (30) +- CometProject (29) +- CometBroadcastHashJoin (28) :- CometProject (24) @@ -174,7 +174,7 @@ Arguments: [ss_hdemo_sk#2], [hd_demo_sk#21], Inner, (((((((cd_marital_status#19 Input [9]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#19, cd_education_status#20, hd_demo_sk#21, hd_dep_count#22] Arguments: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8], [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] (31) HashAggregate [codegen id : 1] @@ -199,7 +199,7 @@ Results [4]: [avg(ss_quantity#5)#37 AS avg(ss_quantity)#41, cast((avg(UnscaledVa Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (38) -+- * ColumnarToRow (37) ++- * CometColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometScan parquet spark_catalog.default.date_dim (34) @@ -220,7 +220,7 @@ Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#16, d_year#17] Arguments: [d_date_sk#16], [d_date_sk#16] -(37) ColumnarToRow [codegen id : 1] +(37) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt index cc52bb3230..1064f5cdd6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt @@ -4,7 +4,7 @@ WholeStageCodegen (2) Exchange #1 WholeStageCodegen (1) HashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum,sum,count,sum,count,sum,count,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] CometBroadcastHashJoin [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status,hd_demo_sk,hd_dep_count] @@ -21,7 +21,7 @@ WholeStageCodegen (2) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/explain.txt index 9649f8b07a..c10f3500f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/explain.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject (104) : +- * HashAggregate (64) : +- Exchange (63) : +- * HashAggregate (62) - : +- * ColumnarToRow (61) + : +- * CometColumnarToRow (61) : +- CometProject (60) : +- CometBroadcastHashJoin (59) : :- CometProject (54) @@ -76,7 +76,7 @@ TakeOrderedAndProject (104) : +- * HashAggregate (80) : +- Exchange (79) : +- * HashAggregate (78) - : +- * ColumnarToRow (77) + : +- * CometColumnarToRow (77) : +- CometProject (76) : +- CometBroadcastHashJoin (75) : :- CometProject (73) @@ -92,7 +92,7 @@ TakeOrderedAndProject (104) +- * HashAggregate (96) +- Exchange (95) +- * HashAggregate (94) - +- * ColumnarToRow (93) + +- * CometColumnarToRow (93) +- CometProject (92) +- CometBroadcastHashJoin (91) :- CometProject (89) @@ -390,7 +390,7 @@ Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -(61) ColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [codegen id : 1] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] (62) HashAggregate [codegen id : 1] @@ -463,7 +463,7 @@ Arguments: [cs_sold_date_sk#65], [d_date_sk#72], Inner, BuildRight Input [7]: [cs_quantity#63, cs_list_price#64, cs_sold_date_sk#65, i_brand_id#69, i_class_id#70, i_category_id#71, d_date_sk#72] Arguments: [cs_quantity#63, cs_list_price#64, i_brand_id#69, i_class_id#70, i_category_id#71], [cs_quantity#63, cs_list_price#64, i_brand_id#69, i_class_id#70, i_category_id#71] -(77) ColumnarToRow [codegen id : 3] +(77) CometColumnarToRow [codegen id : 3] Input [5]: [cs_quantity#63, cs_list_price#64, i_brand_id#69, i_class_id#70, i_category_id#71] (78) HashAggregate [codegen id : 3] @@ -536,7 +536,7 @@ Arguments: [ws_sold_date_sk#87], [d_date_sk#94], Inner, BuildRight Input [7]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#87, i_brand_id#91, i_class_id#92, i_category_id#93, d_date_sk#94] Arguments: [ws_quantity#85, ws_list_price#86, i_brand_id#91, i_class_id#92, i_category_id#93], [ws_quantity#85, ws_list_price#86, i_brand_id#91, i_class_id#92, i_category_id#93] -(93) ColumnarToRow [codegen id : 5] +(93) CometColumnarToRow [codegen id : 5] Input [5]: [ws_quantity#85, ws_list_price#86, i_brand_id#91, i_class_id#92, i_category_id#93] (94) HashAggregate [codegen id : 5] @@ -599,7 +599,7 @@ Subquery:1 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquer * HashAggregate (121) +- Exchange (120) +- * HashAggregate (119) - +- * ColumnarToRow (118) + +- * CometColumnarToRow (118) +- CometUnion (117) :- CometProject (108) : +- CometBroadcastHashJoin (107) @@ -677,7 +677,7 @@ Child 0 Input [2]: [quantity#126, list_price#127] Child 1 Input [2]: [quantity#133, list_price#134] Child 2 Input [2]: [quantity#140, list_price#141] -(118) ColumnarToRow [codegen id : 1] +(118) CometColumnarToRow [codegen id : 1] Input [2]: [quantity#126, list_price#127] (119) HashAggregate [codegen id : 1] @@ -706,7 +706,7 @@ Subquery:4 Hosting operator id = 113 Hosting Expression = ws_sold_date_sk#137 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (126) -+- * ColumnarToRow (125) ++- * CometColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometScan parquet spark_catalog.default.date_dim (122) @@ -727,7 +727,7 @@ Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2 Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(125) ColumnarToRow [codegen id : 1] +(125) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] (126) BroadcastExchange @@ -736,7 +736,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (131) -+- * ColumnarToRow (130) ++- * CometColumnarToRow (130) +- CometProject (129) +- CometFilter (128) +- CometScan parquet spark_catalog.default.date_dim (127) @@ -757,7 +757,7 @@ Condition : (((isnotnull(d_year#148) AND (d_year#148 >= 1999)) AND (d_year#148 < Input [2]: [d_date_sk#26, d_year#148] Arguments: [d_date_sk#26], [d_date_sk#26] -(130) ColumnarToRow [codegen id : 1] +(130) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] (131) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt index e0e466c187..a33a5cb807 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Exchange #15 WholeStageCodegen (1) HashAggregate [quantity,list_price] [sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] @@ -41,7 +41,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Exchange [i_brand_id,i_class_id,i_category_id] #2 WholeStageCodegen (1) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] @@ -53,7 +53,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -77,7 +77,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -130,7 +130,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Exchange [i_brand_id,i_class_id,i_category_id] #16 WholeStageCodegen (3) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] @@ -152,7 +152,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Exchange [i_brand_id,i_class_id,i_category_id] #17 WholeStageCodegen (5) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/explain.txt index e4a215cd28..01e1a95648 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/explain.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject (86) : +- * HashAggregate (64) : +- Exchange (63) : +- * HashAggregate (62) - : +- * ColumnarToRow (61) + : +- * CometColumnarToRow (61) : +- CometProject (60) : +- CometBroadcastHashJoin (59) : :- CometProject (54) @@ -71,7 +71,7 @@ TakeOrderedAndProject (86) +- * HashAggregate (82) +- Exchange (81) +- * HashAggregate (80) - +- * ColumnarToRow (79) + +- * CometColumnarToRow (79) +- CometProject (78) +- CometBroadcastHashJoin (77) :- CometProject (72) @@ -372,7 +372,7 @@ Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -(61) ColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [codegen id : 1] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] (62) HashAggregate [codegen id : 1] @@ -457,7 +457,7 @@ Arguments: [ss_sold_date_sk#63], [d_date_sk#70], Inner, BuildRight Input [7]: [ss_quantity#61, ss_list_price#62, ss_sold_date_sk#63, i_brand_id#67, i_class_id#68, i_category_id#69, d_date_sk#70] Arguments: [ss_quantity#61, ss_list_price#62, i_brand_id#67, i_class_id#68, i_category_id#69], [ss_quantity#61, ss_list_price#62, i_brand_id#67, i_class_id#68, i_category_id#69] -(79) ColumnarToRow [codegen id : 2] +(79) CometColumnarToRow [codegen id : 2] Input [5]: [ss_quantity#61, ss_list_price#62, i_brand_id#67, i_class_id#68, i_category_id#69] (80) HashAggregate [codegen id : 2] @@ -502,7 +502,7 @@ Subquery:1 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquer * HashAggregate (103) +- Exchange (102) +- * HashAggregate (101) - +- * ColumnarToRow (100) + +- * CometColumnarToRow (100) +- CometUnion (99) :- CometProject (90) : +- CometBroadcastHashJoin (89) @@ -580,7 +580,7 @@ Child 0 Input [2]: [quantity#90, list_price#91] Child 1 Input [2]: [quantity#97, list_price#98] Child 2 Input [2]: [quantity#104, list_price#105] -(100) ColumnarToRow [codegen id : 1] +(100) CometColumnarToRow [codegen id : 1] Input [2]: [quantity#90, list_price#91] (101) HashAggregate [codegen id : 1] @@ -609,7 +609,7 @@ Subquery:4 Hosting operator id = 95 Hosting Expression = ws_sold_date_sk#101 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (108) -+- * ColumnarToRow (107) ++- * CometColumnarToRow (107) +- CometProject (106) +- CometFilter (105) +- CometScan parquet spark_catalog.default.date_dim (104) @@ -630,7 +630,7 @@ Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subqu Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(107) ColumnarToRow [codegen id : 1] +(107) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] (108) BroadcastExchange @@ -640,7 +640,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] Subquery:7 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* ColumnarToRow (112) +* CometColumnarToRow (112) +- CometProject (111) +- CometFilter (110) +- CometScan parquet spark_catalog.default.date_dim (109) @@ -661,12 +661,12 @@ Condition : (((((isnotnull(d_year#113) AND isnotnull(d_moy#114)) AND isnotnull(d Input [4]: [d_week_seq#112, d_year#113, d_moy#114, d_dom#115] Arguments: [d_week_seq#112], [d_week_seq#112] -(112) ColumnarToRow [codegen id : 1] +(112) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#112] Subquery:8 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (117) -+- * ColumnarToRow (116) ++- * CometColumnarToRow (116) +- CometProject (115) +- CometFilter (114) +- CometScan parquet spark_catalog.default.date_dim (113) @@ -687,7 +687,7 @@ Condition : (((isnotnull(d_year#116) AND (d_year#116 >= 1999)) AND (d_year#116 < Input [2]: [d_date_sk#26, d_year#116] Arguments: [d_date_sk#26], [d_date_sk#26] -(116) ColumnarToRow [codegen id : 1] +(116) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] (117) BroadcastExchange @@ -706,7 +706,7 @@ Subquery:13 Hosting operator id = 83 Hosting Expression = ReusedSubquery Subquer Subquery:14 Hosting operator id = 66 Hosting Expression = ss_sold_date_sk#63 IN dynamicpruning#64 BroadcastExchange (122) -+- * ColumnarToRow (121) ++- * CometColumnarToRow (121) +- CometProject (120) +- CometFilter (119) +- CometScan parquet spark_catalog.default.date_dim (118) @@ -727,7 +727,7 @@ Condition : ((isnotnull(d_week_seq#71) AND (d_week_seq#71 = ReusedSubquery Subqu Input [2]: [d_date_sk#70, d_week_seq#71] Arguments: [d_date_sk#70], [d_date_sk#70] -(121) ColumnarToRow [codegen id : 1] +(121) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#70] (122) BroadcastExchange @@ -737,7 +737,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:15 Hosting operator id = 119 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] Subquery:16 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#72, [id=#73] -* ColumnarToRow (126) +* CometColumnarToRow (126) +- CometProject (125) +- CometFilter (124) +- CometScan parquet spark_catalog.default.date_dim (123) @@ -758,7 +758,7 @@ Condition : (((((isnotnull(d_year#118) AND isnotnull(d_moy#119)) AND isnotnull(d Input [4]: [d_week_seq#117, d_year#118, d_moy#119, d_dom#120] Arguments: [d_week_seq#117], [d_week_seq#117] -(126) ColumnarToRow [codegen id : 1] +(126) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#117] Subquery:17 Hosting operator id = 74 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt index 1fecc0fca4..62cb19cd3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Exchange #14 WholeStageCodegen (1) HashAggregate [quantity,list_price] [sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] @@ -32,7 +32,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Exchange [i_brand_id,i_class_id,i_category_id] #1 WholeStageCodegen (1) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] @@ -44,7 +44,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_week_seq] @@ -52,7 +52,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_week_seq,d_year,d_moy,d_dom] @@ -76,7 +76,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -132,7 +132,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Exchange [i_brand_id,i_class_id,i_category_id] #16 WholeStageCodegen (2) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] @@ -144,7 +144,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_week_seq] @@ -152,7 +152,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] Subquery #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_week_seq,d_year,d_moy,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/explain.txt index fe3c7a681f..8921f0157b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (23) +- * HashAggregate (22) +- Exchange (21) +- * HashAggregate (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometProject (18) +- CometBroadcastHashJoin (17) :- CometProject (12) @@ -112,7 +112,7 @@ Arguments: [cs_sold_date_sk#3], [d_date_sk#10], Inner, BuildRight Input [4]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#9, d_date_sk#10] Arguments: [cs_sales_price#2, ca_zip#9], [cs_sales_price#2, ca_zip#9] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [2]: [cs_sales_price#2, ca_zip#9] (20) HashAggregate [codegen id : 1] @@ -141,7 +141,7 @@ Arguments: 100, [ca_zip#9 ASC NULLS FIRST], [ca_zip#9, sum(cs_sales_price)#16] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * ColumnarToRow (27) ++- * CometColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan parquet spark_catalog.default.date_dim (24) @@ -162,7 +162,7 @@ Condition : ((((isnotnull(d_qoy#12) AND isnotnull(d_year#11)) AND (d_qoy#12 = 2) Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(27) ColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt index a34cd9c1b3..46baec1bf6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] Exchange [ca_zip] #1 WholeStageCodegen (1) HashAggregate [ca_zip,cs_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_sales_price,ca_zip] CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,d_date_sk] @@ -18,7 +18,7 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/explain.txt index 1ff243ff2a..fbace2c0df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/explain.txt @@ -4,7 +4,7 @@ +- * HashAggregate (38) +- * HashAggregate (37) +- * HashAggregate (36) - +- * ColumnarToRow (35) + +- * CometColumnarToRow (35) +- CometProject (34) +- CometBroadcastHashJoin (33) :- CometProject (28) @@ -198,7 +198,7 @@ Arguments: [cs_call_center_sk#3], [cc_call_center_sk#18], Inner, BuildRight Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#18] Arguments: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -(35) ColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [codegen id : 1] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] (36) HashAggregate [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt index 72df9cecef..0c512bf4eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt @@ -6,7 +6,7 @@ WholeStageCodegen (2) HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] HashAggregate [cs_order_number,cs_ext_ship_cost,cs_net_profit] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_order_number,cs_ext_ship_cost,cs_net_profit] CometBroadcastHashJoin [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cc_call_center_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/explain.txt index dab3dfbf36..8a60f99123 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (42) +- * HashAggregate (41) +- Exchange (40) +- * HashAggregate (39) - +- * ColumnarToRow (38) + +- * CometColumnarToRow (38) +- CometProject (37) +- CometBroadcastHashJoin (36) :- CometProject (32) @@ -221,7 +221,7 @@ Arguments: [ss_item_sk#1], [i_item_sk#26], Inner, BuildRight Input [8]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_item_sk#26, i_item_id#27, i_item_desc#28] Arguments: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_item_id#27, i_item_desc#28], [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_item_id#27, i_item_desc#28] -(38) ColumnarToRow [codegen id : 1] +(38) CometColumnarToRow [codegen id : 1] Input [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_item_id#27, i_item_desc#28] (39) HashAggregate [codegen id : 1] @@ -250,7 +250,7 @@ Arguments: 100, [i_item_id#27 ASC NULLS FIRST, i_item_desc#28 ASC NULLS FIRST, s Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (47) -+- * ColumnarToRow (46) ++- * CometColumnarToRow (46) +- CometProject (45) +- CometFilter (44) +- CometScan parquet spark_catalog.default.date_dim (43) @@ -271,7 +271,7 @@ Condition : ((isnotnull(d_quarter_name#20) AND (d_quarter_name#20 = 2001Q1)) AND Input [2]: [d_date_sk#19, d_quarter_name#20] Arguments: [d_date_sk#19], [d_date_sk#19] -(46) ColumnarToRow [codegen id : 1] +(46) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] (47) BroadcastExchange @@ -280,7 +280,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -301,7 +301,7 @@ Condition : (d_quarter_name#22 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk Input [2]: [d_date_sk#21, d_quarter_name#22] Arguments: [d_date_sk#21], [d_date_sk#21] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#21] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt index 4a634b0699..626fe5b919 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s Exchange [i_item_id,i_item_desc,s_state] #1 WholeStageCodegen (1) HashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_sk,i_item_id,i_item_desc] @@ -26,7 +26,7 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_quarter_name] @@ -37,7 +37,7 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_quarter_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/explain.txt index 414a066f7c..92ba462936 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (41) +- * HashAggregate (40) +- Exchange (39) +- * HashAggregate (38) - +- * ColumnarToRow (37) + +- * CometColumnarToRow (37) +- CometExpand (36) +- CometProject (35) +- CometBroadcastHashJoin (34) @@ -214,7 +214,7 @@ Arguments: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, c Input [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, ca_state#23, ca_county#22] Arguments: [[cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, ca_state#23, ca_county#22, 0], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, ca_state#23, null, 1], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, null, null, 3], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, null, null, null, 7], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, null, null, null, null, 15]], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33] -(37) ColumnarToRow [codegen id : 1] +(37) CometColumnarToRow [codegen id : 1] Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33] (38) HashAggregate [codegen id : 1] @@ -243,7 +243,7 @@ Arguments: 100, [ca_country#30 ASC NULLS FIRST, ca_state#31 ASC NULLS FIRST, ca_ Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (46) -+- * ColumnarToRow (45) ++- * CometColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometScan parquet spark_catalog.default.date_dim (42) @@ -264,7 +264,7 @@ Condition : ((isnotnull(d_year#26) AND (d_year#26 = 1998)) AND isnotnull(d_date_ Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(45) ColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt index c49bfc6c88..9929cea43e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag Exchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 WholeStageCodegen (1) HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometExpand [i_item_id,ca_country,ca_state,ca_county] [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] @@ -25,7 +25,7 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/explain.txt index 69d887880b..c7557752a0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (34) +- * HashAggregate (33) +- Exchange (32) +- * HashAggregate (31) - +- * ColumnarToRow (30) + +- * CometColumnarToRow (30) +- CometProject (29) +- CometBroadcastHashJoin (28) :- CometProject (24) @@ -175,7 +175,7 @@ Arguments: [ss_store_sk#6], [s_store_sk#19], Inner, NOT (substr(ca_zip#18, 1, 5) Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, ca_zip#18, s_store_sk#19, s_zip#20] Arguments: [ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13], [ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [5]: [ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] (31) HashAggregate [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt index 60caecf91e..dad450b233 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] Exchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 WholeStageCodegen (1) HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip,s_store_sk,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/explain.txt index 17f54d24ad..ec2be33ab6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (36) +* CometColumnarToRow (36) +- CometSort (35) +- CometColumnarExchange (34) +- * Project (33) @@ -9,7 +9,7 @@ : :- * HashAggregate (14) : : +- Exchange (13) : : +- * HashAggregate (12) - : : +- * ColumnarToRow (11) + : : +- * CometColumnarToRow (11) : : +- CometProject (10) : : +- CometBroadcastHashJoin (9) : : :- CometUnion (5) @@ -21,7 +21,7 @@ : : +- CometFilter (7) : : +- CometScan parquet spark_catalog.default.date_dim (6) : +- BroadcastExchange (19) - : +- * ColumnarToRow (18) + : +- * CometColumnarToRow (18) : +- CometProject (17) : +- CometFilter (16) : +- CometScan parquet spark_catalog.default.date_dim (15) @@ -31,7 +31,7 @@ :- * HashAggregate (23) : +- ReusedExchange (22) +- BroadcastExchange (28) - +- * ColumnarToRow (27) + +- * CometColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan parquet spark_catalog.default.date_dim (24) @@ -87,7 +87,7 @@ Arguments: [sold_date_sk#3], [d_date_sk#9], Inner, BuildRight Input [5]: [sold_date_sk#3, sales_price#4, d_date_sk#9, d_week_seq#10, d_day_name#11] Arguments: [sales_price#4, d_week_seq#10, d_day_name#11], [sales_price#4, d_week_seq#10, d_day_name#11] -(11) ColumnarToRow [codegen id : 1] +(11) CometColumnarToRow [codegen id : 1] Input [3]: [sales_price#4, d_week_seq#10, d_day_name#11] (12) HashAggregate [codegen id : 1] @@ -123,7 +123,7 @@ Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2001)) AND isnotnull(d_week_ Input [2]: [d_week_seq#40, d_year#41] Arguments: [d_week_seq#40], [d_week_seq#40] -(18) ColumnarToRow [codegen id : 2] +(18) CometColumnarToRow [codegen id : 2] Input [1]: [d_week_seq#40] (19) BroadcastExchange @@ -165,7 +165,7 @@ Condition : ((isnotnull(d_year#68) AND (d_year#68 = 2002)) AND isnotnull(d_week_ Input [2]: [d_week_seq#67, d_year#68] Arguments: [d_week_seq#67], [d_week_seq#67] -(27) ColumnarToRow [codegen id : 4] +(27) CometColumnarToRow [codegen id : 4] Input [1]: [d_week_seq#67] (28) BroadcastExchange @@ -204,6 +204,6 @@ Arguments: rangepartitioning(d_week_seq1#42 ASC NULLS FIRST, 5), ENSURE_REQUIREM Input [8]: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#77, round((mon_sales1 / mon_sales2), 2)#78, round((tue_sales1 / tue_sales2), 2)#79, round((wed_sales1 / wed_sales2), 2)#80, round((thu_sales1 / thu_sales2), 2)#81, round((fri_sales1 / fri_sales2), 2)#82, round((sat_sales1 / sat_sales2), 2)#83] Arguments: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#77, round((mon_sales1 / mon_sales2), 2)#78, round((tue_sales1 / tue_sales2), 2)#79, round((wed_sales1 / wed_sales2), 2)#80, round((thu_sales1 / thu_sales2), 2)#81, round((fri_sales1 / fri_sales2), 2)#82, round((sat_sales1 / sat_sales2), 2)#83], [d_week_seq1#42 ASC NULLS FIRST] -(36) ColumnarToRow [codegen id : 7] +(36) CometColumnarToRow [codegen id : 7] Input [8]: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#77, round((mon_sales1 / mon_sales2), 2)#78, round((tue_sales1 / tue_sales2), 2)#79, round((wed_sales1 / wed_sales2), 2)#80, round((thu_sales1 / thu_sales2), 2)#81, round((fri_sales1 / fri_sales2), 2)#82, round((sat_sales1 / sat_sales2), 2)#83] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt index 6991754b7e..51cc8fb59d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (7) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] CometColumnarExchange [d_week_seq1] #1 @@ -13,7 +13,7 @@ WholeStageCodegen (7) Exchange [d_week_seq] #2 WholeStageCodegen (1) HashAggregate [d_week_seq,d_day_name,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [sales_price,d_week_seq,d_day_name] CometBroadcastHashJoin [sold_date_sk,sales_price,d_date_sk,d_week_seq,d_day_name] @@ -28,7 +28,7 @@ WholeStageCodegen (7) InputAdapter BroadcastExchange #4 WholeStageCodegen (2) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_week_seq,d_year] @@ -44,7 +44,7 @@ WholeStageCodegen (7) InputAdapter BroadcastExchange #6 WholeStageCodegen (4) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt index 9eb32de680..c9946d1e46 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt @@ -7,7 +7,7 @@ TakeOrderedAndProject (22) +- * HashAggregate (17) +- Exchange (16) +- * HashAggregate (15) - +- * ColumnarToRow (14) + +- * CometColumnarToRow (14) +- CometProject (13) +- CometBroadcastHashJoin (12) :- CometProject (7) @@ -87,7 +87,7 @@ Arguments: [cs_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] Arguments: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [6]: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] (15) HashAggregate [codegen id : 1] @@ -132,7 +132,7 @@ Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_ite Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (27) -+- * ColumnarToRow (26) ++- * CometColumnarToRow (26) +- CometProject (25) +- CometFilter (24) +- CometScan parquet spark_catalog.default.date_dim (23) @@ -153,7 +153,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (27) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt index 5bfd9a907c..0010579c69 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 WholeStageCodegen (1) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] @@ -24,7 +24,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/explain.txt index 16ce2c766f..50d6002608 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/explain.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject (24) +- * HashAggregate (22) +- Exchange (21) +- * HashAggregate (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometProject (18) +- CometBroadcastHashJoin (17) :- CometProject (13) @@ -113,7 +113,7 @@ Arguments: [inv_date_sk#4], [d_date_sk#11], Inner, BuildRight Input [6]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#9, d_date_sk#11, d_date#12] Arguments: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#9, d_date#12], [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#9, d_date#12] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#9, d_date#12] (20) HashAggregate [codegen id : 1] @@ -146,7 +146,7 @@ Arguments: 100, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (28) -+- * ColumnarToRow (27) ++- * CometColumnarToRow (27) +- CometFilter (26) +- CometScan parquet spark_catalog.default.date_dim (25) @@ -162,7 +162,7 @@ ReadSchema: struct Input [2]: [d_date_sk#11, d_date#12] Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-10)) AND (d_date#12 <= 2000-04-10)) AND isnotnull(d_date_sk#11)) -(27) ColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#11, d_date#12] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt index 2c8892b689..e849072b3a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] Exchange [w_warehouse_name,i_item_id] #1 WholeStageCodegen (1) HashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] [sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id,d_date_sk,d_date] @@ -19,7 +19,7 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/explain.txt index 21c5568c2b..8def906a84 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (24) +- * HashAggregate (23) +- Exchange (22) +- * HashAggregate (21) - +- * ColumnarToRow (20) + +- * CometColumnarToRow (20) +- CometExpand (19) +- CometProject (18) +- CometBroadcastHashJoin (17) @@ -117,7 +117,7 @@ Arguments: [inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, i_ Input [5]: [inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, i_category#11] Arguments: [[inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, i_category#11, 0], [inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, null, 1], [inv_quantity_on_hand#3, i_product_name#12, i_brand#9, null, null, 3], [inv_quantity_on_hand#3, i_product_name#12, null, null, null, 7], [inv_quantity_on_hand#3, null, null, null, null, 15]], [inv_quantity_on_hand#3, i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarToRow [codegen id : 1] Input [6]: [inv_quantity_on_hand#3, i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18] (21) HashAggregate [codegen id : 1] @@ -146,7 +146,7 @@ Arguments: 100, [qoh#24 ASC NULLS FIRST, i_product_name#14 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (29) -+- * ColumnarToRow (28) ++- * CometColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan parquet spark_catalog.default.date_dim (25) @@ -167,7 +167,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_mont Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt index eb25d6d3ee..9238b3a31c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Exchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 WholeStageCodegen (1) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometExpand [i_product_name,i_brand,i_class,i_category] [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category,spark_grouping_id] CometProject [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] @@ -19,7 +19,7 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/explain.txt index 00a790d59a..1b4351a829 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/explain.txt @@ -7,7 +7,7 @@ : +- * BroadcastHashJoin Inner BuildRight (44) : :- * Project (42) : : +- * SortMergeJoin LeftSemi (41) - : : :- * ColumnarToRow (25) + : : :- * CometColumnarToRow (25) : : : +- CometSort (24) : : : +- CometExchange (23) : : : +- CometProject (22) @@ -38,7 +38,7 @@ : : +- * HashAggregate (37) : : +- Exchange (36) : : +- * HashAggregate (35) - : : +- * ColumnarToRow (34) + : : +- * CometColumnarToRow (34) : : +- CometProject (33) : : +- CometBroadcastHashJoin (32) : : :- CometProject (28) @@ -52,7 +52,7 @@ +- * BroadcastHashJoin Inner BuildRight (61) :- * Project (59) : +- * SortMergeJoin LeftSemi (58) - : :- * ColumnarToRow (52) + : :- * CometColumnarToRow (52) : : +- CometSort (51) : : +- CometExchange (50) : : +- CometProject (49) @@ -181,7 +181,7 @@ Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, Come Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] (26) CometScan parquet spark_catalog.default.store_sales @@ -223,7 +223,7 @@ Arguments: [ss_customer_sk#19], [c_customer_sk#23], Inner, BuildRight Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, c_customer_sk#23] Arguments: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23], [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -(34) ColumnarToRow [codegen id : 2] +(34) CometColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] (35) HashAggregate [codegen id : 2] @@ -306,7 +306,7 @@ Arguments: hashpartitioning(ws_bill_customer_sk#35, 5), ENSURE_REQUIREMENTS, Com Input [4]: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] Arguments: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38], [ws_bill_customer_sk#35 ASC NULLS FIRST] -(52) ColumnarToRow [codegen id : 6] +(52) CometColumnarToRow [codegen id : 6] Input [4]: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] (53) ReusedExchange [Reuses operator id: 36] @@ -378,7 +378,7 @@ Results [1]: [sum(sales#33)#53 AS sum(sales)#54] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (71) -+- * ColumnarToRow (70) ++- * CometColumnarToRow (70) +- CometProject (69) +- CometFilter (68) +- CometScan parquet spark_catalog.default.date_dim (67) @@ -399,7 +399,7 @@ Condition : ((((isnotnull(d_year#55) AND isnotnull(d_moy#56)) AND (d_year#55 = 2 Input [3]: [d_date_sk#32, d_year#55, d_moy#56] Arguments: [d_date_sk#32], [d_date_sk#32] -(70) ColumnarToRow [codegen id : 1] +(70) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#32] (71) BroadcastExchange @@ -408,7 +408,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 2 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (76) -+- * ColumnarToRow (75) ++- * CometColumnarToRow (75) +- CometProject (74) +- CometFilter (73) +- CometScan parquet spark_catalog.default.date_dim (72) @@ -429,7 +429,7 @@ Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(75) ColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] (76) BroadcastExchange @@ -443,7 +443,7 @@ Subquery:3 Hosting operator id = 38 Hosting Expression = Subquery scalar-subquer +- * HashAggregate (91) +- Exchange (90) +- * HashAggregate (89) - +- * ColumnarToRow (88) + +- * CometColumnarToRow (88) +- CometProject (87) +- CometBroadcastHashJoin (86) :- CometProject (81) @@ -509,7 +509,7 @@ Arguments: [ss_sold_date_sk#60], [d_date_sk#63], Inner, BuildRight Input [5]: [ss_quantity#58, ss_sales_price#59, ss_sold_date_sk#60, c_customer_sk#62, d_date_sk#63] Arguments: [ss_quantity#58, ss_sales_price#59, c_customer_sk#62], [ss_quantity#58, ss_sales_price#59, c_customer_sk#62] -(88) ColumnarToRow [codegen id : 1] +(88) CometColumnarToRow [codegen id : 1] Input [3]: [ss_quantity#58, ss_sales_price#59, c_customer_sk#62] (89) HashAggregate [codegen id : 1] @@ -550,7 +550,7 @@ Results [1]: [max(csales#70)#73 AS tpcds_cmax#74] Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#60 IN dynamicpruning#61 BroadcastExchange (99) -+- * ColumnarToRow (98) ++- * CometColumnarToRow (98) +- CometProject (97) +- CometFilter (96) +- CometScan parquet spark_catalog.default.date_dim (95) @@ -571,7 +571,7 @@ Condition : (d_year#64 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#63)) Input [2]: [d_date_sk#63, d_year#64] Arguments: [d_date_sk#63], [d_date_sk#63] -(98) ColumnarToRow [codegen id : 1] +(98) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#63] (99) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt index 760cbdd8bc..e1e95ac0f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt @@ -13,7 +13,7 @@ WholeStageCodegen (12) SortMergeJoin [cs_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometExchange [cs_bill_customer_sk] #2 @@ -23,7 +23,7 @@ WholeStageCodegen (12) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -43,7 +43,7 @@ WholeStageCodegen (12) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_year] @@ -72,7 +72,7 @@ WholeStageCodegen (12) Exchange [c_customer_sk] #12 WholeStageCodegen (1) HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_quantity,ss_sales_price,c_customer_sk] CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] @@ -83,7 +83,7 @@ WholeStageCodegen (12) SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -98,7 +98,7 @@ WholeStageCodegen (12) Exchange [c_customer_sk] #9 WholeStageCodegen (2) HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_quantity,ss_sales_price,c_customer_sk] CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] @@ -117,7 +117,7 @@ WholeStageCodegen (12) SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (6) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometExchange [ws_bill_customer_sk] #15 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/explain.txt index 56684d343b..a5ac499555 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/explain.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject (87) : :- * Project (56) : : +- * BroadcastHashJoin Inner BuildRight (55) : : :- * SortMergeJoin LeftSemi (42) - : : : :- * ColumnarToRow (26) + : : : :- * CometColumnarToRow (26) : : : : +- CometSort (25) : : : : +- CometExchange (24) : : : : +- CometProject (23) @@ -41,7 +41,7 @@ TakeOrderedAndProject (87) : : : +- * HashAggregate (38) : : : +- Exchange (37) : : : +- * HashAggregate (36) - : : : +- * ColumnarToRow (35) + : : : +- * CometColumnarToRow (35) : : : +- CometProject (34) : : : +- CometBroadcastHashJoin (33) : : : :- CometProject (29) @@ -52,7 +52,7 @@ TakeOrderedAndProject (87) : : : +- CometScan parquet spark_catalog.default.customer (30) : : +- BroadcastExchange (54) : : +- * SortMergeJoin LeftSemi (53) - : : :- * ColumnarToRow (47) + : : :- * CometColumnarToRow (47) : : : +- CometSort (46) : : : +- CometExchange (45) : : : +- CometFilter (44) @@ -71,7 +71,7 @@ TakeOrderedAndProject (87) :- * Project (79) : +- * BroadcastHashJoin Inner BuildRight (78) : :- * SortMergeJoin LeftSemi (76) - : : :- * ColumnarToRow (70) + : : :- * CometColumnarToRow (70) : : : +- CometSort (69) : : : +- CometExchange (68) : : : +- CometProject (67) @@ -207,7 +207,7 @@ Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, Come Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] (27) CometScan parquet spark_catalog.default.store_sales @@ -249,7 +249,7 @@ Arguments: [ss_customer_sk#19], [c_customer_sk#23], Inner, BuildRight Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, c_customer_sk#23] Arguments: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23], [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -(35) ColumnarToRow [codegen id : 2] +(35) CometColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] (36) HashAggregate [codegen id : 2] @@ -307,7 +307,7 @@ Arguments: hashpartitioning(c_customer_sk#32, 5), ENSURE_REQUIREMENTS, CometNati Input [3]: [c_customer_sk#32, c_first_name#33, c_last_name#34] Arguments: [c_customer_sk#32, c_first_name#33, c_last_name#34], [c_customer_sk#32 ASC NULLS FIRST] -(47) ColumnarToRow [codegen id : 4] +(47) CometColumnarToRow [codegen id : 4] Input [3]: [c_customer_sk#32, c_first_name#33, c_last_name#34] (48) ReusedExchange [Reuses operator id: 37] @@ -415,7 +415,7 @@ Arguments: hashpartitioning(ws_bill_customer_sk#43, 5), ENSURE_REQUIREMENTS, Com Input [4]: [ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45, ws_sold_date_sk#46] Arguments: [ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45, ws_sold_date_sk#46], [ws_bill_customer_sk#43 ASC NULLS FIRST] -(70) ColumnarToRow [codegen id : 11] +(70) CometColumnarToRow [codegen id : 11] Input [4]: [ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45, ws_sold_date_sk#46] (71) ReusedExchange [Reuses operator id: 37] @@ -500,7 +500,7 @@ Arguments: 100, [c_last_name#34 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (92) -+- * ColumnarToRow (91) ++- * CometColumnarToRow (91) +- CometProject (90) +- CometFilter (89) +- CometScan parquet spark_catalog.default.date_dim (88) @@ -521,7 +521,7 @@ Condition : ((((isnotnull(d_year#65) AND isnotnull(d_moy#66)) AND (d_year#65 = 2 Input [3]: [d_date_sk#35, d_year#65, d_moy#66] Arguments: [d_date_sk#35], [d_date_sk#35] -(91) ColumnarToRow [codegen id : 1] +(91) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#35] (92) BroadcastExchange @@ -530,7 +530,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (97) -+- * ColumnarToRow (96) ++- * CometColumnarToRow (96) +- CometProject (95) +- CometFilter (94) +- CometScan parquet spark_catalog.default.date_dim (93) @@ -551,7 +551,7 @@ Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(96) ColumnarToRow [codegen id : 1] +(96) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] (97) BroadcastExchange @@ -565,7 +565,7 @@ Subquery:3 Hosting operator id = 39 Hosting Expression = Subquery scalar-subquer +- * HashAggregate (112) +- Exchange (111) +- * HashAggregate (110) - +- * ColumnarToRow (109) + +- * CometColumnarToRow (109) +- CometProject (108) +- CometBroadcastHashJoin (107) :- CometProject (102) @@ -631,7 +631,7 @@ Arguments: [ss_sold_date_sk#70], [d_date_sk#73], Inner, BuildRight Input [5]: [ss_quantity#68, ss_sales_price#69, ss_sold_date_sk#70, c_customer_sk#72, d_date_sk#73] Arguments: [ss_quantity#68, ss_sales_price#69, c_customer_sk#72], [ss_quantity#68, ss_sales_price#69, c_customer_sk#72] -(109) ColumnarToRow [codegen id : 1] +(109) CometColumnarToRow [codegen id : 1] Input [3]: [ss_quantity#68, ss_sales_price#69, c_customer_sk#72] (110) HashAggregate [codegen id : 1] @@ -672,7 +672,7 @@ Results [1]: [max(csales#80)#83 AS tpcds_cmax#84] Subquery:4 Hosting operator id = 98 Hosting Expression = ss_sold_date_sk#70 IN dynamicpruning#71 BroadcastExchange (120) -+- * ColumnarToRow (119) ++- * CometColumnarToRow (119) +- CometProject (118) +- CometFilter (117) +- CometScan parquet spark_catalog.default.date_dim (116) @@ -693,7 +693,7 @@ Condition : (d_year#74 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#73)) Input [2]: [d_date_sk#73, d_year#74] Arguments: [d_date_sk#73], [d_date_sk#73] -(119) ColumnarToRow [codegen id : 1] +(119) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#73] (120) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt index 1184e50857..8fc45070c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] SortMergeJoin [cs_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometExchange [cs_bill_customer_sk] #2 @@ -24,7 +24,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -44,7 +44,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_year] @@ -73,7 +73,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Exchange [c_customer_sk] #12 WholeStageCodegen (1) HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_quantity,ss_sales_price,c_customer_sk] CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] @@ -84,7 +84,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -99,7 +99,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Exchange [c_customer_sk] #9 WholeStageCodegen (2) HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_quantity,ss_sales_price,c_customer_sk] CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] @@ -115,7 +115,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] SortMergeJoin [c_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (4) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [c_customer_sk,c_first_name,c_last_name] CometExchange [c_customer_sk] #16 @@ -145,7 +145,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (11) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometExchange [ws_bill_customer_sk] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/explain.txt index 9dbf4af839..d7b76b4dde 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/explain.txt @@ -8,7 +8,7 @@ +- * HashAggregate (36) +- * Project (35) +- * BroadcastHashJoin Inner BuildRight (34) - :- * ColumnarToRow (29) + :- * CometColumnarToRow (29) : +- CometProject (28) : +- CometBroadcastHashJoin (27) : :- CometProject (23) @@ -38,7 +38,7 @@ : +- CometFilter (25) : +- CometScan parquet spark_catalog.default.customer (24) +- BroadcastExchange (33) - +- * ColumnarToRow (32) + +- * CometColumnarToRow (32) +- CometFilter (31) +- CometScan parquet spark_catalog.default.customer_address (30) @@ -174,7 +174,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Arguments: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24], [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] -(29) ColumnarToRow [codegen id : 2] +(29) CometColumnarToRow [codegen id : 2] Input [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] (30) CometScan parquet spark_catalog.default.customer_address @@ -188,7 +188,7 @@ ReadSchema: struct Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] (33) BroadcastExchange @@ -256,7 +256,7 @@ Subquery:1 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquer +- * HashAggregate (64) +- * Project (63) +- * BroadcastHashJoin Inner BuildRight (62) - :- * ColumnarToRow (60) + :- * CometColumnarToRow (60) : +- CometProject (59) : +- CometBroadcastHashJoin (58) : :- CometProject (56) @@ -348,7 +348,7 @@ Arguments: [ss_customer_sk#41], [c_customer_sk#57], Inner, BuildRight Input [14]: [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_customer_sk#57, c_first_name#58, c_last_name#59, c_birth_country#60] Arguments: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60], [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60] -(60) ColumnarToRow [codegen id : 2] +(60) CometColumnarToRow [codegen id : 2] Input [12]: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60] (61) ReusedExchange [Reuses operator id: 33] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt index bd14d82504..cd048032d9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt @@ -14,7 +14,7 @@ WholeStageCodegen (4) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] @@ -47,7 +47,7 @@ WholeStageCodegen (4) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] @@ -80,7 +80,7 @@ WholeStageCodegen (4) InputAdapter BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/explain.txt index f27ae40199..9730c4bd08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/explain.txt @@ -8,7 +8,7 @@ +- * HashAggregate (36) +- * Project (35) +- * BroadcastHashJoin Inner BuildRight (34) - :- * ColumnarToRow (29) + :- * CometColumnarToRow (29) : +- CometProject (28) : +- CometBroadcastHashJoin (27) : :- CometProject (23) @@ -38,7 +38,7 @@ : +- CometFilter (25) : +- CometScan parquet spark_catalog.default.customer (24) +- BroadcastExchange (33) - +- * ColumnarToRow (32) + +- * CometColumnarToRow (32) +- CometFilter (31) +- CometScan parquet spark_catalog.default.customer_address (30) @@ -174,7 +174,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Arguments: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24], [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] -(29) ColumnarToRow [codegen id : 2] +(29) CometColumnarToRow [codegen id : 2] Input [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] (30) CometScan parquet spark_catalog.default.customer_address @@ -188,7 +188,7 @@ ReadSchema: struct Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] (33) BroadcastExchange @@ -256,7 +256,7 @@ Subquery:1 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquer +- * HashAggregate (64) +- * Project (63) +- * BroadcastHashJoin Inner BuildRight (62) - :- * ColumnarToRow (60) + :- * CometColumnarToRow (60) : +- CometProject (59) : +- CometBroadcastHashJoin (58) : :- CometProject (56) @@ -348,7 +348,7 @@ Arguments: [ss_customer_sk#41], [c_customer_sk#57], Inner, BuildRight Input [14]: [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_customer_sk#57, c_first_name#58, c_last_name#59, c_birth_country#60] Arguments: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60], [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60] -(60) ColumnarToRow [codegen id : 2] +(60) CometColumnarToRow [codegen id : 2] Input [12]: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60] (61) ReusedExchange [Reuses operator id: 33] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt index bd14d82504..cd048032d9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt @@ -14,7 +14,7 @@ WholeStageCodegen (4) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] @@ -47,7 +47,7 @@ WholeStageCodegen (4) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] @@ -80,7 +80,7 @@ WholeStageCodegen (4) InputAdapter BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt index 35ca891390..77e2d8d9df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (42) +- * HashAggregate (41) +- Exchange (40) +- * HashAggregate (39) - +- * ColumnarToRow (38) + +- * CometColumnarToRow (38) +- CometProject (37) +- CometBroadcastHashJoin (36) :- CometProject (32) @@ -221,7 +221,7 @@ Arguments: [ss_item_sk#1], [i_item_sk#29], Inner, BuildRight Input [9]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28, i_item_sk#29, i_item_id#30, i_item_desc#31] Arguments: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28, i_item_id#30, i_item_desc#31], [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28, i_item_id#30, i_item_desc#31] -(38) ColumnarToRow [codegen id : 1] +(38) CometColumnarToRow [codegen id : 1] Input [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28, i_item_id#30, i_item_desc#31] (39) HashAggregate [codegen id : 1] @@ -250,7 +250,7 @@ Arguments: 100, [i_item_id#30 ASC NULLS FIRST, i_item_desc#31 ASC NULLS FIRST, s Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (47) -+- * ColumnarToRow (46) ++- * CometColumnarToRow (46) +- CometProject (45) +- CometFilter (44) +- CometScan parquet spark_catalog.default.date_dim (43) @@ -271,7 +271,7 @@ Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4) Input [3]: [d_date_sk#19, d_year#20, d_moy#21] Arguments: [d_date_sk#19], [d_date_sk#19] -(46) ColumnarToRow [codegen id : 1] +(46) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] (47) BroadcastExchange @@ -280,7 +280,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -301,7 +301,7 @@ Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= Input [3]: [d_date_sk#22, d_year#23, d_moy#24] Arguments: [d_date_sk#22], [d_date_sk#22] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt index 6335e50ff8..33649c7034 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales Exchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 WholeStageCodegen (1) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] [sum,sum,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] CometBroadcastHashJoin [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] @@ -26,7 +26,7 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -37,7 +37,7 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/explain.txt index 0cbbe17335..a7ee8ae815 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (30) +- * HashAggregate (29) +- Exchange (28) +- * HashAggregate (27) - +- * ColumnarToRow (26) + +- * CometColumnarToRow (26) +- CometProject (25) +- CometBroadcastHashJoin (24) :- CometProject (19) @@ -151,7 +151,7 @@ Arguments: [cs_promo_sk#3], [p_promo_sk#18], Inner, BuildRight Input [7]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17, p_promo_sk#18] Arguments: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] (27) HashAggregate [codegen id : 1] @@ -180,7 +180,7 @@ Arguments: 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#41, agg2#42, Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (35) -+- * ColumnarToRow (34) ++- * CometColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan parquet spark_catalog.default.date_dim (31) @@ -201,7 +201,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt index b98e61704b..5aeefca644 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] Exchange [i_item_id] #1 WholeStageCodegen (1) HashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] CometBroadcastHashJoin [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id,p_promo_sk] @@ -20,7 +20,7 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/explain.txt index f136cc4562..c526eb5cfd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (30) +- * HashAggregate (29) +- Exchange (28) +- * HashAggregate (27) - +- * ColumnarToRow (26) + +- * CometColumnarToRow (26) +- CometExpand (25) +- CometProject (24) +- CometBroadcastHashJoin (23) @@ -151,7 +151,7 @@ Arguments: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i Input [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, s_state#17] Arguments: [[ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, s_state#17, 0], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, null, 1], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, null, null, 3]], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#21, spark_grouping_id#22] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [7]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#21, spark_grouping_id#22] (27) HashAggregate [codegen id : 1] @@ -180,7 +180,7 @@ Arguments: 100, [i_item_id#20 ASC NULLS FIRST, s_state#21 ASC NULLS FIRST], [i_i Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (35) -+- * ColumnarToRow (34) ++- * CometColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan parquet spark_catalog.default.date_dim (31) @@ -201,7 +201,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt index 4d4c824bc9..6b1a0b0592 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] Exchange [i_item_id,s_state,spark_grouping_id] #1 WholeStageCodegen (1) HashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometExpand [i_item_id,s_state] [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state,spark_grouping_id] CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] @@ -21,7 +21,7 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/explain.txt index c59c9e2309..5ff905502c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/explain.txt @@ -10,7 +10,7 @@ : : : : : +- * HashAggregate (7) : : : : : +- Exchange (6) : : : : : +- * HashAggregate (5) -: : : : : +- * ColumnarToRow (4) +: : : : : +- * CometColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) @@ -21,7 +21,7 @@ : : : : +- * HashAggregate (17) : : : : +- Exchange (16) : : : : +- * HashAggregate (15) -: : : : +- * ColumnarToRow (14) +: : : : +- * CometColumnarToRow (14) : : : : +- CometProject (13) : : : : +- CometFilter (12) : : : : +- CometScan parquet spark_catalog.default.store_sales (11) @@ -32,7 +32,7 @@ : : : +- * HashAggregate (29) : : : +- Exchange (28) : : : +- * HashAggregate (27) -: : : +- * ColumnarToRow (26) +: : : +- * CometColumnarToRow (26) : : : +- CometProject (25) : : : +- CometFilter (24) : : : +- CometScan parquet spark_catalog.default.store_sales (23) @@ -43,7 +43,7 @@ : : +- * HashAggregate (41) : : +- Exchange (40) : : +- * HashAggregate (39) -: : +- * ColumnarToRow (38) +: : +- * CometColumnarToRow (38) : : +- CometProject (37) : : +- CometFilter (36) : : +- CometScan parquet spark_catalog.default.store_sales (35) @@ -54,7 +54,7 @@ : +- * HashAggregate (53) : +- Exchange (52) : +- * HashAggregate (51) -: +- * ColumnarToRow (50) +: +- * CometColumnarToRow (50) : +- CometProject (49) : +- CometFilter (48) : +- CometScan parquet spark_catalog.default.store_sales (47) @@ -65,7 +65,7 @@ +- * HashAggregate (65) +- Exchange (64) +- * HashAggregate (63) - +- * ColumnarToRow (62) + +- * CometColumnarToRow (62) +- CometProject (61) +- CometFilter (60) +- CometScan parquet spark_catalog.default.store_sales (59) @@ -86,7 +86,7 @@ Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 0)) AND (ss_quanti Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] Arguments: [ss_list_price#3], [ss_list_price#3] -(4) ColumnarToRow [codegen id : 1] +(4) CometColumnarToRow [codegen id : 1] Input [1]: [ss_list_price#3] (5) HashAggregate [codegen id : 1] @@ -140,7 +140,7 @@ Condition : (((isnotnull(ss_quantity#16) AND (ss_quantity#16 >= 6)) AND (ss_quan Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] Arguments: [ss_list_price#18], [ss_list_price#18] -(14) ColumnarToRow [codegen id : 3] +(14) CometColumnarToRow [codegen id : 3] Input [1]: [ss_list_price#18] (15) HashAggregate [codegen id : 3] @@ -202,7 +202,7 @@ Condition : (((isnotnull(ss_quantity#31) AND (ss_quantity#31 >= 11)) AND (ss_qua Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] Arguments: [ss_list_price#33], [ss_list_price#33] -(26) ColumnarToRow [codegen id : 6] +(26) CometColumnarToRow [codegen id : 6] Input [1]: [ss_list_price#33] (27) HashAggregate [codegen id : 6] @@ -264,7 +264,7 @@ Condition : (((isnotnull(ss_quantity#46) AND (ss_quantity#46 >= 16)) AND (ss_qua Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] Arguments: [ss_list_price#48], [ss_list_price#48] -(38) ColumnarToRow [codegen id : 9] +(38) CometColumnarToRow [codegen id : 9] Input [1]: [ss_list_price#48] (39) HashAggregate [codegen id : 9] @@ -326,7 +326,7 @@ Condition : (((isnotnull(ss_quantity#61) AND (ss_quantity#61 >= 21)) AND (ss_qua Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] Arguments: [ss_list_price#63], [ss_list_price#63] -(50) ColumnarToRow [codegen id : 12] +(50) CometColumnarToRow [codegen id : 12] Input [1]: [ss_list_price#63] (51) HashAggregate [codegen id : 12] @@ -388,7 +388,7 @@ Condition : (((isnotnull(ss_quantity#76) AND (ss_quantity#76 >= 26)) AND (ss_qua Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] Arguments: [ss_list_price#78], [ss_list_price#78] -(62) ColumnarToRow [codegen id : 15] +(62) CometColumnarToRow [codegen id : 15] Input [1]: [ss_list_price#78] (63) HashAggregate [codegen id : 15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt index 2136886fe3..d763f01032 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt @@ -14,7 +14,7 @@ WholeStageCodegen (18) Exchange [ss_list_price] #2 WholeStageCodegen (1) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] @@ -32,7 +32,7 @@ WholeStageCodegen (18) Exchange [ss_list_price] #5 WholeStageCodegen (3) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] @@ -50,7 +50,7 @@ WholeStageCodegen (18) Exchange [ss_list_price] #8 WholeStageCodegen (6) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] @@ -68,7 +68,7 @@ WholeStageCodegen (18) Exchange [ss_list_price] #11 WholeStageCodegen (9) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] @@ -86,7 +86,7 @@ WholeStageCodegen (18) Exchange [ss_list_price] #14 WholeStageCodegen (12) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] @@ -104,7 +104,7 @@ WholeStageCodegen (18) Exchange [ss_list_price] #17 WholeStageCodegen (15) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/explain.txt index 6f27f77d00..d761d81e41 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (45) +- * HashAggregate (44) +- Exchange (43) +- * HashAggregate (42) - +- * ColumnarToRow (41) + +- * CometColumnarToRow (41) +- CometProject (40) +- CometBroadcastHashJoin (39) :- CometProject (35) @@ -240,7 +240,7 @@ Arguments: [ss_item_sk#1], [i_item_sk#30], Inner, BuildRight Input [9]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29, i_item_sk#30, i_item_id#31, i_item_desc#32] Arguments: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29, i_item_id#31, i_item_desc#32], [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29, i_item_id#31, i_item_desc#32] -(41) ColumnarToRow [codegen id : 1] +(41) CometColumnarToRow [codegen id : 1] Input [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29, i_item_id#31, i_item_desc#32] (42) HashAggregate [codegen id : 1] @@ -269,7 +269,7 @@ Arguments: 100, [i_item_id#31 ASC NULLS FIRST, i_item_desc#32 ASC NULLS FIRST, s Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (50) -+- * ColumnarToRow (49) ++- * CometColumnarToRow (49) +- CometProject (48) +- CometFilter (47) +- CometScan parquet spark_catalog.default.date_dim (46) @@ -290,7 +290,7 @@ Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9) Input [3]: [d_date_sk#19, d_year#20, d_moy#21] Arguments: [d_date_sk#19], [d_date_sk#19] -(49) ColumnarToRow [codegen id : 1] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] (50) BroadcastExchange @@ -299,7 +299,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (55) -+- * ColumnarToRow (54) ++- * CometColumnarToRow (54) +- CometProject (53) +- CometFilter (52) +- CometScan parquet spark_catalog.default.date_dim (51) @@ -320,7 +320,7 @@ Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= Input [3]: [d_date_sk#22, d_year#23, d_moy#24] Arguments: [d_date_sk#22], [d_date_sk#22] -(54) ColumnarToRow [codegen id : 1] +(54) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] (55) BroadcastExchange @@ -329,7 +329,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 BroadcastExchange (60) -+- * ColumnarToRow (59) ++- * CometColumnarToRow (59) +- CometProject (58) +- CometFilter (57) +- CometScan parquet spark_catalog.default.date_dim (56) @@ -350,7 +350,7 @@ Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(59) ColumnarToRow [codegen id : 1] +(59) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] (60) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt index ee1f435d50..78fa6a4a34 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales Exchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 WholeStageCodegen (1) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] [sum,sum,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] @@ -26,7 +26,7 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -37,7 +37,7 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -48,7 +48,7 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/explain.txt index 0b7ac89233..2714185a7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (19) +- * HashAggregate (18) +- Exchange (17) +- * HashAggregate (16) - +- * ColumnarToRow (15) + +- * CometColumnarToRow (15) +- CometProject (14) +- CometBroadcastHashJoin (13) :- CometProject (8) @@ -88,7 +88,7 @@ Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#9] Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarToRow [codegen id : 1] Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] (16) HashAggregate [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt index 36bb17a161..4556271c2f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] Exchange [d_year,i_brand,i_brand_id] #1 WholeStageCodegen (1) HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/explain.txt index f13c4c64af..8b7a4b1cbc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/explain.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject (51) : : : +- * HashAggregate (17) : : : +- Exchange (16) : : : +- * HashAggregate (15) - : : : +- * ColumnarToRow (14) + : : : +- * CometColumnarToRow (14) : : : +- CometProject (13) : : : +- CometBroadcastHashJoin (12) : : : :- CometProject (8) @@ -32,7 +32,7 @@ TakeOrderedAndProject (51) : : +- * HashAggregate (30) : : +- Exchange (29) : : +- * HashAggregate (28) - : : +- * ColumnarToRow (27) + : : +- * CometColumnarToRow (27) : : +- CometProject (26) : : +- CometBroadcastHashJoin (25) : : :- CometProject (23) @@ -42,11 +42,11 @@ TakeOrderedAndProject (51) : : : +- ReusedExchange (21) : : +- ReusedExchange (24) : +- BroadcastExchange (41) - : +- * ColumnarToRow (40) + : +- * CometColumnarToRow (40) : +- CometFilter (39) : +- CometScan parquet spark_catalog.default.customer (38) +- BroadcastExchange (48) - +- * ColumnarToRow (47) + +- * CometColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometScan parquet spark_catalog.default.customer_address (44) @@ -116,7 +116,7 @@ Arguments: [wr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#8, ca_state#9] Arguments: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9], [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] (15) HashAggregate [codegen id : 1] @@ -177,7 +177,7 @@ Arguments: [wr_returning_addr_sk#17], [ca_address_sk#22], Inner, BuildRight Input [5]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, ca_address_sk#22, ca_state#23] Arguments: [wr_returning_customer_sk#16, wr_return_amt#18, ca_state#23], [wr_returning_customer_sk#16, wr_return_amt#18, ca_state#23] -(27) ColumnarToRow [codegen id : 2] +(27) CometColumnarToRow [codegen id : 2] Input [3]: [wr_returning_customer_sk#16, wr_return_amt#18, ca_state#23] (28) HashAggregate [codegen id : 2] @@ -245,7 +245,7 @@ ReadSchema: struct Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(99) ColumnarToRow [codegen id : 1] +(99) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] (100) BroadcastExchange @@ -589,7 +589,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 17 Hosting Expression = ss_sold_date_sk#16 IN dynamicpruning#17 BroadcastExchange (104) -+- * ColumnarToRow (103) ++- * CometColumnarToRow (103) +- CometFilter (102) +- CometScan parquet spark_catalog.default.date_dim (101) @@ -605,7 +605,7 @@ ReadSchema: struct Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] Condition : ((((isnotnull(d_qoy#20) AND isnotnull(d_year#19)) AND (d_qoy#20 = 2)) AND (d_year#19 = 2000)) AND isnotnull(d_date_sk#18)) -(103) ColumnarToRow [codegen id : 1] +(103) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] (104) BroadcastExchange @@ -614,7 +614,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:3 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 BroadcastExchange (108) -+- * ColumnarToRow (107) ++- * CometColumnarToRow (107) +- CometFilter (106) +- CometScan parquet spark_catalog.default.date_dim (105) @@ -630,7 +630,7 @@ ReadSchema: struct Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] Condition : ((((isnotnull(d_qoy#32) AND isnotnull(d_year#31)) AND (d_qoy#32 = 3)) AND (d_year#31 = 2000)) AND isnotnull(d_date_sk#30)) -(107) ColumnarToRow [codegen id : 1] +(107) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] (108) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt index 8a759d62f3..d95ac9a9d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (13) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] CometColumnarExchange [ca_county] #1 @@ -17,7 +17,7 @@ WholeStageCodegen (13) Exchange [ca_county,d_qoy,d_year] #2 WholeStageCodegen (1) HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] @@ -28,7 +28,7 @@ WholeStageCodegen (13) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] @@ -46,7 +46,7 @@ WholeStageCodegen (13) Exchange [ca_county,d_qoy,d_year] #7 WholeStageCodegen (2) HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] @@ -57,7 +57,7 @@ WholeStageCodegen (13) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] @@ -73,7 +73,7 @@ WholeStageCodegen (13) Exchange [ca_county,d_qoy,d_year] #11 WholeStageCodegen (4) HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] @@ -84,7 +84,7 @@ WholeStageCodegen (13) SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #12 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] @@ -100,7 +100,7 @@ WholeStageCodegen (13) Exchange [ca_county,d_qoy,d_year] #15 WholeStageCodegen (6) HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] @@ -119,7 +119,7 @@ WholeStageCodegen (13) Exchange [ca_county,d_qoy,d_year] #17 WholeStageCodegen (8) HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] @@ -138,7 +138,7 @@ WholeStageCodegen (13) Exchange [ca_county,d_qoy,d_year] #19 WholeStageCodegen (10) HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/explain.txt index 8fe97f412a..e1f46dd0ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/explain.txt @@ -6,7 +6,7 @@ +- * BroadcastHashJoin Inner BuildRight (27) :- * Project (25) : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * ColumnarToRow (9) + : :- * CometColumnarToRow (9) : : +- CometProject (8) : : +- CometBroadcastHashJoin (7) : : :- CometFilter (2) @@ -20,7 +20,7 @@ : +- * HashAggregate (21) : +- Exchange (20) : +- * HashAggregate (19) - : +- * ColumnarToRow (18) + : +- * CometColumnarToRow (18) : +- CometProject (17) : +- CometBroadcastHashJoin (16) : :- CometFilter (11) @@ -72,7 +72,7 @@ Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] Arguments: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5], [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] -(9) ColumnarToRow [codegen id : 4] +(9) CometColumnarToRow [codegen id : 4] Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] (10) CometScan parquet spark_catalog.default.catalog_sales @@ -115,7 +115,7 @@ Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight Input [4]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9, d_date_sk#11] Arguments: [cs_item_sk#7, cs_ext_discount_amt#8], [cs_item_sk#7, cs_ext_discount_amt#8] -(18) ColumnarToRow [codegen id : 1] +(18) CometColumnarToRow [codegen id : 1] Input [2]: [cs_item_sk#7, cs_ext_discount_amt#8] (19) HashAggregate [codegen id : 1] @@ -189,7 +189,7 @@ Results [1]: [MakeDecimal(sum(UnscaledValue(cs_ext_discount_amt#2))#22,17,2) AS Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (36) -+- * ColumnarToRow (35) ++- * CometColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan parquet spark_catalog.default.date_dim (32) @@ -210,7 +210,7 @@ Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 2000-01-27)) AND (d_date#2 Input [2]: [d_date_sk#19, d_date#24] Arguments: [d_date_sk#19], [d_date_sk#19] -(35) ColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt index b6ba1fbab1..632b051166 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt @@ -8,7 +8,7 @@ WholeStageCodegen (5) BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_ext_discount_amt,cs_sold_date_sk] BroadcastHashJoin [i_item_sk,cs_item_sk,cs_ext_discount_amt,(1.3 * avg(cs_ext_discount_amt))] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] @@ -17,7 +17,7 @@ WholeStageCodegen (5) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] @@ -35,7 +35,7 @@ WholeStageCodegen (5) Exchange [cs_item_sk] #5 WholeStageCodegen (1) HashAggregate [cs_item_sk,cs_ext_discount_amt] [sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_item_sk,cs_ext_discount_amt] CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/explain.txt index 2558133dee..394cba8aa8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/explain.txt @@ -7,7 +7,7 @@ TakeOrderedAndProject (63) :- * HashAggregate (28) : +- Exchange (27) : +- * HashAggregate (26) - : +- * ColumnarToRow (25) + : +- * CometColumnarToRow (25) : +- CometProject (24) : +- CometBroadcastHashJoin (23) : :- CometProject (14) @@ -35,7 +35,7 @@ TakeOrderedAndProject (63) :- * HashAggregate (43) : +- Exchange (42) : +- * HashAggregate (41) - : +- * ColumnarToRow (40) + : +- * CometColumnarToRow (40) : +- CometProject (39) : +- CometBroadcastHashJoin (38) : :- CometProject (36) @@ -50,7 +50,7 @@ TakeOrderedAndProject (63) +- * HashAggregate (58) +- Exchange (57) +- * HashAggregate (56) - +- * ColumnarToRow (55) + +- * CometColumnarToRow (55) +- CometProject (54) +- CometBroadcastHashJoin (53) :- CometProject (51) @@ -180,7 +180,7 @@ Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_manufact_id#12] Arguments: [ss_ext_sales_price#3, i_manufact_id#12], [ss_ext_sales_price#3, i_manufact_id#12] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [2]: [ss_ext_sales_price#3, i_manufact_id#12] (26) HashAggregate [codegen id : 1] @@ -249,7 +249,7 @@ Arguments: [cs_item_sk#20], [i_item_sk#26], Inner, BuildRight Input [4]: [cs_item_sk#20, cs_ext_sales_price#21, i_item_sk#26, i_manufact_id#27] Arguments: [cs_ext_sales_price#21, i_manufact_id#27], [cs_ext_sales_price#21, i_manufact_id#27] -(40) ColumnarToRow [codegen id : 3] +(40) CometColumnarToRow [codegen id : 3] Input [2]: [cs_ext_sales_price#21, i_manufact_id#27] (41) HashAggregate [codegen id : 3] @@ -318,7 +318,7 @@ Arguments: [ws_item_sk#32], [i_item_sk#39], Inner, BuildRight Input [4]: [ws_item_sk#32, ws_ext_sales_price#34, i_item_sk#39, i_manufact_id#40] Arguments: [ws_ext_sales_price#34, i_manufact_id#40], [ws_ext_sales_price#34, i_manufact_id#40] -(55) ColumnarToRow [codegen id : 5] +(55) CometColumnarToRow [codegen id : 5] Input [2]: [ws_ext_sales_price#34, i_manufact_id#40] (56) HashAggregate [codegen id : 5] @@ -367,7 +367,7 @@ Arguments: 100, [total_sales#50 ASC NULLS FIRST], [i_manufact_id#12, total_sales Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (68) -+- * ColumnarToRow (67) ++- * CometColumnarToRow (67) +- CometProject (66) +- CometFilter (65) +- CometScan parquet spark_catalog.default.date_dim (64) @@ -388,7 +388,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(67) ColumnarToRow [codegen id : 1] +(67) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (68) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt index bf0ef7ae47..2e05aa2ec5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject [total_sales,i_manufact_id] Exchange [i_manufact_id] #2 WholeStageCodegen (1) HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_manufact_id] CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] @@ -26,7 +26,7 @@ TakeOrderedAndProject [total_sales,i_manufact_id] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -53,7 +53,7 @@ TakeOrderedAndProject [total_sales,i_manufact_id] Exchange [i_manufact_id] #8 WholeStageCodegen (3) HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_ext_sales_price,i_manufact_id] CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_manufact_id] @@ -73,7 +73,7 @@ TakeOrderedAndProject [total_sales,i_manufact_id] Exchange [i_manufact_id] #9 WholeStageCodegen (5) HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_ext_sales_price,i_manufact_id] CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/explain.txt index 50e3800526..452f25394a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (32) +* CometColumnarToRow (32) +- CometSort (31) +- CometColumnarExchange (30) +- CometProject (29) @@ -179,14 +179,14 @@ Arguments: rangepartitioning(c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] Arguments: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (37) -+- * ColumnarToRow (36) ++- * CometColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometScan parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_do Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt index 779d4cbddb..014786e7fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/explain.txt index c1e19555c9..0730e11302 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/explain.txt @@ -11,7 +11,7 @@ TakeOrderedAndProject (44) : : +- * Filter (27) : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * ColumnarToRow (12) + : : : :- * CometColumnarToRow (12) : : : : +- CometBroadcastHashJoin (11) : : : : :- CometFilter (2) : : : : : +- CometScan parquet spark_catalog.default.customer (1) @@ -24,23 +24,23 @@ TakeOrderedAndProject (44) : : : : +- CometFilter (5) : : : : +- CometScan parquet spark_catalog.default.date_dim (4) : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) + : : : +- * CometColumnarToRow (17) : : : +- CometProject (16) : : : +- CometBroadcastHashJoin (15) : : : :- CometScan parquet spark_catalog.default.web_sales (13) : : : +- ReusedExchange (14) : : +- BroadcastExchange (25) - : : +- * ColumnarToRow (24) + : : +- * CometColumnarToRow (24) : : +- CometProject (23) : : +- CometBroadcastHashJoin (22) : : :- CometScan parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (32) - : +- * ColumnarToRow (31) + : +- * CometColumnarToRow (31) : +- CometFilter (30) : +- CometScan parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (38) - +- * ColumnarToRow (37) + +- * CometColumnarToRow (37) +- CometFilter (36) +- CometScan parquet spark_catalog.default.customer_demographics (35) @@ -100,7 +100,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) ColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 5] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan parquet spark_catalog.default.web_sales @@ -122,7 +122,7 @@ Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#12] (18) BroadcastExchange @@ -154,7 +154,7 @@ Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [1]: [cs_ship_customer_sk#16] (25) BroadcastExchange @@ -186,7 +186,7 @@ ReadSchema: struct Input [2]: [ca_address_sk#20, ca_state#21] Condition : isnotnull(ca_address_sk#20) -(31) ColumnarToRow [codegen id : 3] +(31) CometColumnarToRow [codegen id : 3] Input [2]: [ca_address_sk#20, ca_state#21] (32) BroadcastExchange @@ -214,7 +214,7 @@ ReadSchema: struct= 2000-02-01)) AND (d_date#1 Input [2]: [d_date_sk#10, d_date#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt index 85e86ca8ae..13fcba8592 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] CometHashAggregate [i_item_id,i_item_desc,i_current_price] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/explain.txt index c0e7300dff..8139d06c08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (47) +* CometColumnarToRow (47) +- CometHashAggregate (46) +- CometExchange (45) +- CometHashAggregate (44) @@ -261,14 +261,14 @@ Input [1]: [count#26] Keys: [] Functions [1]: [count(1)] -(47) ColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [1]: [count(1)#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 BroadcastExchange (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt index 8272260f5c..5ce896e6e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [count(1),count,count(1)] CometExchange #1 @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/explain.txt index 6e343640d6..67a961f36e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (49) +* CometColumnarToRow (49) +- CometSort (48) +- CometColumnarExchange (47) +- * BroadcastHashJoin Inner BuildRight (46) @@ -8,7 +8,7 @@ : +- * HashAggregate (22) : +- Exchange (21) : +- * HashAggregate (20) - : +- * ColumnarToRow (19) + : +- * CometColumnarToRow (19) : +- CometProject (18) : +- CometBroadcastHashJoin (17) : :- CometProject (12) @@ -33,7 +33,7 @@ +- * HashAggregate (42) +- Exchange (41) +- * HashAggregate (40) - +- * ColumnarToRow (39) + +- * CometColumnarToRow (39) +- CometProject (38) +- CometBroadcastHashJoin (37) :- CometProject (32) @@ -138,7 +138,7 @@ Arguments: [inv_date_sk#4], [d_date_sk#9], Inner, BuildRight Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#11] Arguments: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11], [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] (20) HashAggregate [codegen id : 1] @@ -231,7 +231,7 @@ Arguments: [inv_date_sk#30], [d_date_sk#35], Inner, BuildRight Input [7]: [inv_quantity_on_hand#29, inv_date_sk#30, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_date_sk#35, d_moy#37] Arguments: [inv_quantity_on_hand#29, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37], [inv_quantity_on_hand#29, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] -(39) ColumnarToRow [codegen id : 2] +(39) CometColumnarToRow [codegen id : 2] Input [5]: [inv_quantity_on_hand#29, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] (40) HashAggregate [codegen id : 2] @@ -278,14 +278,14 @@ Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC N Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#25, cov#26, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#49, cov#50] Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#25, cov#26, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#49, cov#50], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#25 ASC NULLS FIRST, cov#26 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, mean#49 ASC NULLS FIRST, cov#50 ASC NULLS FIRST] -(49) ColumnarToRow [codegen id : 5] +(49) CometColumnarToRow [codegen id : 5] Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#25, cov#26, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#49, cov#50] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (54) -+- * ColumnarToRow (53) ++- * CometColumnarToRow (53) +- CometProject (52) +- CometFilter (51) +- CometScan parquet spark_catalog.default.date_dim (50) @@ -306,7 +306,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(53) ColumnarToRow [codegen id : 1] +(53) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_moy#11] (54) BroadcastExchange @@ -315,7 +315,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 25 Hosting Expression = inv_date_sk#30 IN dynamicpruning#31 BroadcastExchange (59) -+- * ColumnarToRow (58) ++- * CometColumnarToRow (58) +- CometProject (57) +- CometFilter (56) +- CometScan parquet spark_catalog.default.date_dim (55) @@ -336,7 +336,7 @@ Condition : ((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2 Input [3]: [d_date_sk#35, d_year#36, d_moy#37] Arguments: [d_date_sk#35, d_moy#37], [d_date_sk#35, d_moy#37] -(58) ColumnarToRow [codegen id : 1] +(58) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#35, d_moy#37] (59) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt index d681e8bf61..8ce3ac8124 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 @@ -12,7 +12,7 @@ WholeStageCodegen (5) Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 WholeStageCodegen (1) HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] @@ -25,7 +25,7 @@ WholeStageCodegen (5) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] @@ -50,7 +50,7 @@ WholeStageCodegen (5) Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 WholeStageCodegen (2) HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] @@ -63,7 +63,7 @@ WholeStageCodegen (5) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/explain.txt index 5572232cdf..4421b9736b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (49) +* CometColumnarToRow (49) +- CometSort (48) +- CometColumnarExchange (47) +- * BroadcastHashJoin Inner BuildRight (46) @@ -8,7 +8,7 @@ : +- * HashAggregate (22) : +- Exchange (21) : +- * HashAggregate (20) - : +- * ColumnarToRow (19) + : +- * CometColumnarToRow (19) : +- CometProject (18) : +- CometBroadcastHashJoin (17) : :- CometProject (12) @@ -33,7 +33,7 @@ +- * HashAggregate (42) +- Exchange (41) +- * HashAggregate (40) - +- * ColumnarToRow (39) + +- * CometColumnarToRow (39) +- CometProject (38) +- CometBroadcastHashJoin (37) :- CometProject (32) @@ -138,7 +138,7 @@ Arguments: [inv_date_sk#4], [d_date_sk#9], Inner, BuildRight Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#11] Arguments: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11], [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] (20) HashAggregate [codegen id : 1] @@ -231,7 +231,7 @@ Arguments: [inv_date_sk#30], [d_date_sk#35], Inner, BuildRight Input [7]: [inv_quantity_on_hand#29, inv_date_sk#30, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_date_sk#35, d_moy#37] Arguments: [inv_quantity_on_hand#29, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37], [inv_quantity_on_hand#29, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] -(39) ColumnarToRow [codegen id : 2] +(39) CometColumnarToRow [codegen id : 2] Input [5]: [inv_quantity_on_hand#29, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] (40) HashAggregate [codegen id : 2] @@ -278,14 +278,14 @@ Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC N Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#25, cov#26, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#49, cov#50] Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#25, cov#26, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#49, cov#50], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#25 ASC NULLS FIRST, cov#26 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, mean#49 ASC NULLS FIRST, cov#50 ASC NULLS FIRST] -(49) ColumnarToRow [codegen id : 5] +(49) CometColumnarToRow [codegen id : 5] Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#25, cov#26, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#49, cov#50] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (54) -+- * ColumnarToRow (53) ++- * CometColumnarToRow (53) +- CometProject (52) +- CometFilter (51) +- CometScan parquet spark_catalog.default.date_dim (50) @@ -306,7 +306,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(53) ColumnarToRow [codegen id : 1] +(53) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_moy#11] (54) BroadcastExchange @@ -315,7 +315,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 25 Hosting Expression = inv_date_sk#30 IN dynamicpruning#31 BroadcastExchange (59) -+- * ColumnarToRow (58) ++- * CometColumnarToRow (58) +- CometProject (57) +- CometFilter (56) +- CometScan parquet spark_catalog.default.date_dim (55) @@ -336,7 +336,7 @@ Condition : ((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2 Input [3]: [d_date_sk#35, d_year#36, d_moy#37] Arguments: [d_date_sk#35, d_moy#37], [d_date_sk#35, d_moy#37] -(58) ColumnarToRow [codegen id : 1] +(58) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#35, d_moy#37] (59) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt index d681e8bf61..8ce3ac8124 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 @@ -12,7 +12,7 @@ WholeStageCodegen (5) Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 WholeStageCodegen (1) HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] @@ -25,7 +25,7 @@ WholeStageCodegen (5) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] @@ -50,7 +50,7 @@ WholeStageCodegen (5) Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 WholeStageCodegen (2) HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] @@ -63,7 +63,7 @@ WholeStageCodegen (5) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/explain.txt index bac54ba443..7ba2c3d48d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/explain.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject (106) : : : : : +- * HashAggregate (16) : : : : : +- Exchange (15) : : : : : +- * HashAggregate (14) - : : : : : +- * ColumnarToRow (13) + : : : : : +- * CometColumnarToRow (13) : : : : : +- CometProject (12) : : : : : +- CometBroadcastHashJoin (11) : : : : : :- CometProject (7) @@ -30,7 +30,7 @@ TakeOrderedAndProject (106) : : : : +- * HashAggregate (33) : : : : +- Exchange (32) : : : : +- * HashAggregate (31) - : : : : +- * ColumnarToRow (30) + : : : : +- * CometColumnarToRow (30) : : : : +- CometProject (29) : : : : +- CometBroadcastHashJoin (28) : : : : :- CometProject (24) @@ -48,7 +48,7 @@ TakeOrderedAndProject (106) : : : +- * HashAggregate (49) : : : +- Exchange (48) : : : +- * HashAggregate (47) - : : : +- * ColumnarToRow (46) + : : : +- * CometColumnarToRow (46) : : : +- CometProject (45) : : : +- CometBroadcastHashJoin (44) : : : :- CometProject (42) @@ -63,7 +63,7 @@ TakeOrderedAndProject (106) : : +- * HashAggregate (67) : : +- Exchange (66) : : +- * HashAggregate (65) - : : +- * ColumnarToRow (64) + : : +- * CometColumnarToRow (64) : : +- CometProject (63) : : +- CometBroadcastHashJoin (62) : : :- CometProject (60) @@ -79,7 +79,7 @@ TakeOrderedAndProject (106) : +- * HashAggregate (84) : +- Exchange (83) : +- * HashAggregate (82) - : +- * ColumnarToRow (81) + : +- * CometColumnarToRow (81) : +- CometProject (80) : +- CometBroadcastHashJoin (79) : :- CometProject (77) @@ -94,7 +94,7 @@ TakeOrderedAndProject (106) +- * HashAggregate (102) +- Exchange (101) +- * HashAggregate (100) - +- * ColumnarToRow (99) + +- * CometColumnarToRow (99) +- CometProject (98) +- CometBroadcastHashJoin (97) :- CometProject (95) @@ -167,7 +167,7 @@ Arguments: [ss_sold_date_sk#14], [d_date_sk#16], Inner, BuildRight Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14, d_date_sk#16, d_year#17] Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#17], [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#17] -(13) ColumnarToRow [codegen id : 1] +(13) CometColumnarToRow [codegen id : 1] Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#17] (14) HashAggregate [codegen id : 1] @@ -252,7 +252,7 @@ Arguments: [ss_sold_date_sk#38], [d_date_sk#40], Inner, BuildRight Input [14]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38, d_date_sk#40, d_year#41] Arguments: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, d_year#41], [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, d_year#41] -(30) ColumnarToRow [codegen id : 2] +(30) CometColumnarToRow [codegen id : 2] Input [12]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, d_year#41] (31) HashAggregate [codegen id : 2] @@ -331,7 +331,7 @@ Arguments: [cs_sold_date_sk#67], [d_date_sk#69], Inner, BuildRight Input [14]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67, d_date_sk#69, d_year#70] Arguments: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, d_year#70], [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, d_year#70] -(46) ColumnarToRow [codegen id : 4] +(46) CometColumnarToRow [codegen id : 4] Input [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, d_year#70] (47) HashAggregate [codegen id : 4] @@ -418,7 +418,7 @@ Arguments: [cs_sold_date_sk#91], [d_date_sk#93], Inner, BuildRight Input [14]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91, d_date_sk#93, d_year#94] Arguments: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, d_year#94], [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, d_year#94] -(64) ColumnarToRow [codegen id : 6] +(64) CometColumnarToRow [codegen id : 6] Input [12]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, d_year#94] (65) HashAggregate [codegen id : 6] @@ -501,7 +501,7 @@ Arguments: [ws_sold_date_sk#114], [d_date_sk#116], Inner, BuildRight Input [14]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114, d_date_sk#116, d_year#117] Arguments: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, d_year#117], [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, d_year#117] -(81) ColumnarToRow [codegen id : 8] +(81) CometColumnarToRow [codegen id : 8] Input [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, d_year#117] (82) HashAggregate [codegen id : 8] @@ -588,7 +588,7 @@ Arguments: [ws_sold_date_sk#138], [d_date_sk#140], Inner, BuildRight Input [14]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138, d_date_sk#140, d_year#141] Arguments: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, d_year#141], [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, d_year#141] -(99) ColumnarToRow [codegen id : 10] +(99) CometColumnarToRow [codegen id : 10] Input [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, d_year#141] (100) HashAggregate [codegen id : 10] @@ -631,7 +631,7 @@ Arguments: 100, [customer_id#46 ASC NULLS FIRST, customer_first_name#47 ASC NULL Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#15 BroadcastExchange (110) -+- * ColumnarToRow (109) ++- * CometColumnarToRow (109) +- CometFilter (108) +- CometScan parquet spark_catalog.default.date_dim (107) @@ -647,7 +647,7 @@ ReadSchema: struct Input [2]: [d_date_sk#16, d_year#17] Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) -(109) ColumnarToRow [codegen id : 1] +(109) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_year#17] (110) BroadcastExchange @@ -656,7 +656,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#39 BroadcastExchange (114) -+- * ColumnarToRow (113) ++- * CometColumnarToRow (113) +- CometFilter (112) +- CometScan parquet spark_catalog.default.date_dim (111) @@ -672,7 +672,7 @@ ReadSchema: struct Input [2]: [d_date_sk#40, d_year#41] Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2002)) AND isnotnull(d_date_sk#40)) -(113) ColumnarToRow [codegen id : 1] +(113) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#40, d_year#41] (114) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt index a412a54d9f..39261a54f0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 WholeStageCodegen (1) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] @@ -29,7 +29,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -44,7 +44,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #6 WholeStageCodegen (2) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] @@ -58,7 +58,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -74,7 +74,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 WholeStageCodegen (4) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year] @@ -95,7 +95,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 WholeStageCodegen (6) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year] @@ -117,7 +117,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #17 WholeStageCodegen (8) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] @@ -138,7 +138,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #20 WholeStageCodegen (10) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/explain.txt index 908b4909e5..255372bf18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (32) +- * HashAggregate (31) +- Exchange (30) +- * HashAggregate (29) - +- * ColumnarToRow (28) + +- * CometColumnarToRow (28) +- CometProject (27) +- CometBroadcastHashJoin (26) :- CometProject (22) @@ -161,7 +161,7 @@ Arguments: [cs_sold_date_sk#5], [d_date_sk#16], Inner, BuildRight Input [7]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date_sk#16, d_date#17] Arguments: [cs_sales_price#4, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date#17], [cs_sales_price#4, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date#17] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [5]: [cs_sales_price#4, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date#17] (29) HashAggregate [codegen id : 1] @@ -190,7 +190,7 @@ Arguments: 100, [w_state#12 ASC NULLS FIRST, i_item_id#14 ASC NULLS FIRST], [w_s Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (36) -+- * ColumnarToRow (35) ++- * CometColumnarToRow (35) +- CometFilter (34) +- CometScan parquet spark_catalog.default.date_dim (33) @@ -206,7 +206,7 @@ ReadSchema: struct Input [2]: [d_date_sk#16, d_date#17] Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2000-02-10)) AND (d_date#17 <= 2000-04-10)) AND isnotnull(d_date_sk#16)) -(35) ColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_date#17] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt index 25a80286d5..32726e2605 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] Exchange [w_state,i_item_id] #1 WholeStageCodegen (1) HashAggregate [w_state,i_item_id,d_date,cs_sales_price,cr_refunded_cash] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_sales_price,cr_refunded_cash,w_state,i_item_id,d_date] CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id,d_date_sk,d_date] @@ -22,7 +22,7 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/explain.txt index 48a1c60d16..bb3c74daeb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -103,6 +103,6 @@ Functions: [] Input [1]: [i_product_name#3] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#3 ASC NULLS FIRST], output=[i_product_name#3]), [i_product_name#3], 100, [i_product_name#3 ASC NULLS FIRST], [i_product_name#3] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [1]: [i_product_name#3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt index 007103499b..de12ce9581 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_product_name] CometHashAggregate [i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/explain.txt index 724b1e8847..11e01518d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (19) +- * HashAggregate (18) +- Exchange (17) +- * HashAggregate (16) - +- * ColumnarToRow (15) + +- * CometColumnarToRow (15) +- CometProject (14) +- CometBroadcastHashJoin (13) :- CometProject (8) @@ -88,7 +88,7 @@ Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_category_id#8, i_category#9] Arguments: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#9], [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#9] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarToRow [codegen id : 1] Input [4]: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#9] (16) HashAggregate [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt index d6e3733ce2..0f67359665 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] Exchange [d_year,i_category_id,i_category] #1 WholeStageCodegen (1) HashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_category_id,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/explain.txt index 016788b64c..f02a7caf55 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (19) +- * HashAggregate (18) +- Exchange (17) +- * HashAggregate (16) - +- * ColumnarToRow (15) + +- * CometColumnarToRow (15) +- CometProject (14) +- CometBroadcastHashJoin (13) :- CometProject (8) @@ -88,7 +88,7 @@ Arguments: [ss_store_sk#4], [s_store_sk#7], Inner, BuildRight Input [6]: [d_day_name#3, ss_store_sk#4, ss_sales_price#5, s_store_sk#7, s_store_id#8, s_store_name#9] Arguments: [d_day_name#3, ss_sales_price#5, s_store_id#8, s_store_name#9], [d_day_name#3, ss_sales_price#5, s_store_id#8, s_store_name#9] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarToRow [codegen id : 1] Input [4]: [d_day_name#3, ss_sales_price#5, s_store_id#8, s_store_name#9] (16) HashAggregate [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt index f21d846c0f..5d11f4e607 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed Exchange [s_store_name,s_store_id] #1 WholeStageCodegen (1) HashAggregate [s_store_name,s_store_id,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] CometBroadcastHashJoin [d_day_name,ss_store_sk,ss_sales_price,s_store_sk,s_store_id,s_store_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/explain.txt index c9e28476b6..211f6f633d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/explain.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject (43) : : : +- * HashAggregate (7) : : : +- Exchange (6) : : : +- * HashAggregate (5) - : : : +- * ColumnarToRow (4) + : : : +- * CometColumnarToRow (4) : : : +- CometProject (3) : : : +- CometFilter (2) : : : +- CometScan parquet spark_catalog.default.store_sales (1) @@ -38,7 +38,7 @@ TakeOrderedAndProject (43) : : +- * HashAggregate (20) : : +- ReusedExchange (19) : +- BroadcastExchange (37) - : +- * ColumnarToRow (36) + : +- * CometColumnarToRow (36) : +- CometFilter (35) : +- CometScan parquet spark_catalog.default.item (34) +- ReusedExchange (40) @@ -59,7 +59,7 @@ Condition : (isnotnull(ss_store_sk#2) AND (ss_store_sk#2 = 4)) Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] Arguments: [ss_item_sk#1, ss_net_profit#3], [ss_item_sk#1, ss_net_profit#3] -(4) ColumnarToRow [codegen id : 1] +(4) CometColumnarToRow [codegen id : 1] Input [2]: [ss_item_sk#1, ss_net_profit#3] (5) HashAggregate [codegen id : 1] @@ -199,7 +199,7 @@ ReadSchema: struct Input [2]: [i_item_sk#23, i_product_name#24] Condition : isnotnull(i_item_sk#23) -(36) ColumnarToRow [codegen id : 11] +(36) CometColumnarToRow [codegen id : 11] Input [2]: [i_item_sk#23, i_product_name#24] (37) BroadcastExchange @@ -239,7 +239,7 @@ Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery * HashAggregate (50) +- Exchange (49) +- * HashAggregate (48) - +- * ColumnarToRow (47) + +- * CometColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometScan parquet spark_catalog.default.store_sales (44) @@ -260,7 +260,7 @@ Condition : ((isnotnull(ss_store_sk#30) AND (ss_store_sk#30 = 4)) AND isnull(ss_ Input [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32] Arguments: [ss_store_sk#30, ss_net_profit#31], [ss_store_sk#30, ss_net_profit#31] -(47) ColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [2]: [ss_store_sk#30, ss_net_profit#31] (48) HashAggregate [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt index 12c8e0af9f..e233a4a5ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] Exchange [ss_store_sk] #4 WholeStageCodegen (1) HashAggregate [ss_store_sk,ss_net_profit] [sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] @@ -42,7 +42,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] Exchange [ss_item_sk] #3 WholeStageCodegen (1) HashAggregate [ss_item_sk,ss_net_profit] [sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_item_sk,ss_net_profit] CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] @@ -73,7 +73,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] InputAdapter BroadcastExchange #7 WholeStageCodegen (11) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [i_item_sk,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/explain.txt index 962a51203b..d9213e07f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/explain.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject (36) +- * Project (32) +- * Filter (31) +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (30) - :- * ColumnarToRow (24) + :- * CometColumnarToRow (24) : +- CometProject (23) : +- CometBroadcastHashJoin (22) : :- CometProject (18) @@ -31,7 +31,7 @@ TakeOrderedAndProject (36) : +- CometFilter (20) : +- CometScan parquet spark_catalog.default.item (19) +- BroadcastExchange (29) - +- * ColumnarToRow (28) + +- * CometColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan parquet spark_catalog.default.item (25) @@ -149,7 +149,7 @@ Arguments: [ws_item_sk#2], [i_item_sk#15], Inner, BuildRight Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#11, i_item_sk#15, i_item_id#16] Arguments: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16], [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [4]: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16] (25) CometScan parquet spark_catalog.default.item @@ -167,7 +167,7 @@ Condition : i_item_sk#17 IN (2,3,5,7,11,13,17,19,23,29) Input [2]: [i_item_sk#17, i_item_id#18] Arguments: [i_item_id#18], [i_item_id#18] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [i_item_id#18] (29) BroadcastExchange @@ -214,7 +214,7 @@ Arguments: 100, [ca_zip#11 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (41) -+- * ColumnarToRow (40) ++- * CometColumnarToRow (40) +- CometProject (39) +- CometFilter (38) +- CometScan parquet spark_catalog.default.date_dim (37) @@ -235,7 +235,7 @@ Condition : ((((isnotnull(d_qoy#14) AND isnotnull(d_year#13)) AND (d_qoy#14 = 2) Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] Arguments: [d_date_sk#12], [d_date_sk#12] -(40) ColumnarToRow [codegen id : 1] +(40) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#12] (41) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt index f60fdb18a1..a325fb95d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] Project [ws_sales_price,ca_city,ca_zip] Filter [ca_zip,exists] BroadcastHashJoin [i_item_id,i_item_id] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ca_city,ca_zip,i_item_sk,i_item_id] @@ -23,7 +23,7 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] @@ -44,7 +44,7 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] InputAdapter BroadcastExchange #7 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [i_item_id] CometFilter [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/explain.txt index a09bd7640d..e258fab3d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/explain.txt @@ -7,7 +7,7 @@ TakeOrderedAndProject (42) : :- * HashAggregate (29) : : +- Exchange (28) : : +- * HashAggregate (27) - : : +- * ColumnarToRow (26) + : : +- * CometColumnarToRow (26) : : +- CometProject (25) : : +- CometBroadcastHashJoin (24) : : :- CometProject (20) @@ -34,11 +34,11 @@ TakeOrderedAndProject (42) : : +- CometFilter (22) : : +- CometScan parquet spark_catalog.default.customer_address (21) : +- BroadcastExchange (33) - : +- * ColumnarToRow (32) + : +- * CometColumnarToRow (32) : +- CometFilter (31) : +- CometScan parquet spark_catalog.default.customer (30) +- BroadcastExchange (39) - +- * ColumnarToRow (38) + +- * CometColumnarToRow (38) +- CometFilter (37) +- CometScan parquet spark_catalog.default.customer_address (36) @@ -163,7 +163,7 @@ Arguments: [ss_addr_sk#3], [ca_address_sk#18], Inner, BuildRight Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_address_sk#18, ca_city#19] Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19] (27) HashAggregate [codegen id : 1] @@ -195,7 +195,7 @@ ReadSchema: struct Input [2]: [ca_address_sk#33, ca_city#34] Condition : (isnotnull(ca_address_sk#33) AND isnotnull(ca_city#34)) -(38) ColumnarToRow [codegen id : 3] +(38) CometColumnarToRow [codegen id : 3] Input [2]: [ca_address_sk#33, ca_city#34] (39) BroadcastExchange @@ -248,7 +248,7 @@ Arguments: 100, [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (47) -+- * ColumnarToRow (46) ++- * CometColumnarToRow (46) +- CometProject (45) +- CometFilter (44) +- CometScan parquet spark_catalog.default.date_dim (43) @@ -269,7 +269,7 @@ Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(46) ColumnarToRow [codegen id : 1] +(46) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (47) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt index 9ce3a98eb5..d219f2b8ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 WholeStageCodegen (1) HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_address_sk,ca_city] @@ -24,7 +24,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dow] @@ -47,14 +47,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu InputAdapter BroadcastExchange #7 WholeStageCodegen (2) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt index 170424a150..9ec5db82b5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject (45) : : +- * HashAggregate (21) : : +- Exchange (20) : : +- * HashAggregate (19) - : : +- * ColumnarToRow (18) + : : +- * CometColumnarToRow (18) : : +- CometProject (17) : : +- CometBroadcastHashJoin (16) : : :- CometProject (12) @@ -130,7 +130,7 @@ Arguments: [ss_store_sk#5], [s_store_sk#12], Inner, BuildRight Input [9]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11, s_store_sk#12, s_store_name#13, s_company_name#14] Arguments: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14], [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14] -(18) ColumnarToRow [codegen id : 1] +(18) CometColumnarToRow [codegen id : 1] Input [7]: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14] (19) HashAggregate [codegen id : 1] @@ -256,7 +256,7 @@ Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (49) -+- * ColumnarToRow (48) ++- * CometColumnarToRow (48) +- CometFilter (47) +- CometScan parquet spark_catalog.default.date_dim (46) @@ -272,7 +272,7 @@ ReadSchema: struct Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(48) ColumnarToRow [codegen id : 1] +(48) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (49) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt index 1db8e7528d..94669b705c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 WholeStageCodegen (1) HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] @@ -38,7 +38,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/explain.txt index 15b16c1e8f..4cf30d8034 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/explain.txt @@ -2,7 +2,7 @@ * HashAggregate (28) +- Exchange (27) +- * HashAggregate (26) - +- * ColumnarToRow (25) + +- * CometColumnarToRow (25) +- CometProject (24) +- CometBroadcastHashJoin (23) :- CometProject (18) @@ -145,7 +145,7 @@ Arguments: [ss_sold_date_sk#7], [d_date_sk#16], Inner, BuildRight Input [3]: [ss_quantity#4, ss_sold_date_sk#7, d_date_sk#16] Arguments: [ss_quantity#4], [ss_quantity#4] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [1]: [ss_quantity#4] (26) HashAggregate [codegen id : 1] @@ -170,7 +170,7 @@ Results [1]: [sum(ss_quantity#4)#20 AS sum(ss_quantity)#21] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (33) -+- * ColumnarToRow (32) ++- * CometColumnarToRow (32) +- CometProject (31) +- CometFilter (30) +- CometScan parquet spark_catalog.default.date_dim (29) @@ -191,7 +191,7 @@ Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#16, d_year#17] Arguments: [d_date_sk#16], [d_date_sk#16] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] (33) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt index 23e0835359..d9ec3841e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt @@ -4,7 +4,7 @@ WholeStageCodegen (2) Exchange #1 WholeStageCodegen (1) HashAggregate [ss_quantity] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_quantity] CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] @@ -19,7 +19,7 @@ WholeStageCodegen (2) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/explain.txt index 205e71071f..0b28238a07 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/explain.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject (77) : +- * HashAggregate (19) : +- Exchange (18) : +- * HashAggregate (17) - : +- * ColumnarToRow (16) + : +- * CometColumnarToRow (16) : +- CometProject (15) : +- CometBroadcastHashJoin (14) : :- CometProject (9) @@ -40,7 +40,7 @@ TakeOrderedAndProject (77) : +- * HashAggregate (42) : +- Exchange (41) : +- * HashAggregate (40) - : +- * ColumnarToRow (39) + : +- * CometColumnarToRow (39) : +- CometProject (38) : +- CometBroadcastHashJoin (37) : :- CometProject (35) @@ -63,7 +63,7 @@ TakeOrderedAndProject (77) +- * HashAggregate (65) +- Exchange (64) +- * HashAggregate (63) - +- * ColumnarToRow (62) + +- * CometColumnarToRow (62) +- CometProject (61) +- CometBroadcastHashJoin (60) :- CometProject (58) @@ -150,7 +150,7 @@ Arguments: [ws_sold_date_sk#6], [d_date_sk#13], Inner, BuildRight Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarToRow [codegen id : 1] Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] (17) HashAggregate [codegen id : 1] @@ -255,7 +255,7 @@ Arguments: [cs_sold_date_sk#43], [d_date_sk#50], Inner, BuildRight Input [7]: [cs_item_sk#38, cs_quantity#40, cs_net_paid#41, cs_sold_date_sk#43, cr_return_quantity#47, cr_return_amount#48, d_date_sk#50] Arguments: [cs_item_sk#38, cs_quantity#40, cs_net_paid#41, cr_return_quantity#47, cr_return_amount#48], [cs_item_sk#38, cs_quantity#40, cs_net_paid#41, cr_return_quantity#47, cr_return_amount#48] -(39) ColumnarToRow [codegen id : 6] +(39) CometColumnarToRow [codegen id : 6] Input [5]: [cs_item_sk#38, cs_quantity#40, cs_net_paid#41, cr_return_quantity#47, cr_return_amount#48] (40) HashAggregate [codegen id : 6] @@ -360,7 +360,7 @@ Arguments: [ss_sold_date_sk#78], [d_date_sk#85], Inner, BuildRight Input [7]: [ss_item_sk#73, ss_quantity#75, ss_net_paid#76, ss_sold_date_sk#78, sr_return_quantity#82, sr_return_amt#83, d_date_sk#85] Arguments: [ss_item_sk#73, ss_quantity#75, ss_net_paid#76, sr_return_quantity#82, sr_return_amt#83], [ss_item_sk#73, ss_quantity#75, ss_net_paid#76, sr_return_quantity#82, sr_return_amt#83] -(62) ColumnarToRow [codegen id : 11] +(62) CometColumnarToRow [codegen id : 11] Input [5]: [ss_item_sk#73, ss_quantity#75, ss_net_paid#76, sr_return_quantity#82, sr_return_amt#83] (63) HashAggregate [codegen id : 11] @@ -437,7 +437,7 @@ Arguments: 100, [channel#37 ASC NULLS FIRST, return_rank#35 ASC NULLS FIRST, cur Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (82) -+- * ColumnarToRow (81) ++- * CometColumnarToRow (81) +- CometProject (80) +- CometFilter (79) +- CometScan parquet spark_catalog.default.date_dim (78) @@ -458,7 +458,7 @@ Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2 Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(81) ColumnarToRow [codegen id : 1] +(81) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] (82) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt index bd3b21cdbd..d47d447ebc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt @@ -26,7 +26,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] Exchange [ws_item_sk] #3 WholeStageCodegen (1) HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] @@ -39,7 +39,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -70,7 +70,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] Exchange [cs_item_sk] #8 WholeStageCodegen (6) HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] @@ -104,7 +104,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] Exchange [ss_item_sk] #11 WholeStageCodegen (11) HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/explain.txt index a29ef4cec3..8c57f3126b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/explain.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject (70) :- * HashAggregate (22) : +- Exchange (21) : +- * HashAggregate (20) - : +- * ColumnarToRow (19) + : +- * CometColumnarToRow (19) : +- CometProject (18) : +- CometBroadcastHashJoin (17) : :- CometProject (13) @@ -30,7 +30,7 @@ TakeOrderedAndProject (70) :- * HashAggregate (41) : +- Exchange (40) : +- * HashAggregate (39) - : +- * ColumnarToRow (38) + : +- * CometColumnarToRow (38) : +- CometProject (37) : +- CometBroadcastHashJoin (36) : :- CometProject (32) @@ -49,7 +49,7 @@ TakeOrderedAndProject (70) +- * HashAggregate (64) +- Exchange (63) +- * HashAggregate (62) - +- * ColumnarToRow (61) + +- * CometColumnarToRow (61) +- CometProject (60) +- CometBroadcastHashJoin (59) :- CometProject (55) @@ -159,7 +159,7 @@ Arguments: [store_sk#6], [s_store_sk#24], Inner, BuildRight Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#25] Arguments: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25], [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] (20) HashAggregate [codegen id : 1] @@ -252,7 +252,7 @@ Arguments: [page_sk#48], [cp_catalog_page_sk#65], Inner, BuildRight Input [7]: [page_sk#48, sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_sk#65, cp_catalog_page_id#66] Arguments: [sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_id#66], [sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_id#66] -(38) ColumnarToRow [codegen id : 3] +(38) CometColumnarToRow [codegen id : 3] Input [5]: [sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_id#66] (39) HashAggregate [codegen id : 3] @@ -364,7 +364,7 @@ Arguments: [wsr_web_site_sk#89], [web_site_sk#111], Inner, BuildRight Input [7]: [wsr_web_site_sk#89, sales_price#91, profit#92, return_amt#93, net_loss#94, web_site_sk#111, web_site_id#112] Arguments: [sales_price#91, profit#92, return_amt#93, net_loss#94, web_site_id#112], [sales_price#91, profit#92, return_amt#93, net_loss#94, web_site_id#112] -(61) ColumnarToRow [codegen id : 5] +(61) CometColumnarToRow [codegen id : 5] Input [5]: [sales_price#91, profit#92, return_amt#93, net_loss#94, web_site_id#112] (62) HashAggregate [codegen id : 5] @@ -417,7 +417,7 @@ Arguments: 100, [channel#130 ASC NULLS FIRST, id#131 ASC NULLS FIRST], [channel# Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (75) -+- * ColumnarToRow (74) ++- * CometColumnarToRow (74) +- CometProject (73) +- CometFilter (72) +- CometScan parquet spark_catalog.default.date_dim (71) @@ -438,7 +438,7 @@ Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#2 Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(74) ColumnarToRow [codegen id : 1] +(74) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] (75) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt index ceb3e92131..23758f9295 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Exchange [s_store_id] #2 WholeStageCodegen (1) HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [sales_price,profit,return_amt,net_loss,s_store_id] CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] @@ -27,7 +27,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] @@ -49,7 +49,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Exchange [cp_catalog_page_id] #6 WholeStageCodegen (3) HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] @@ -74,7 +74,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Exchange [web_site_id] #8 WholeStageCodegen (5) HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [sales_price,profit,return_amt,net_loss,web_site_id] CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/explain.txt index c55f258e53..212d2159cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (28) +- * HashAggregate (27) +- Exchange (26) +- * HashAggregate (25) - +- * ColumnarToRow (24) + +- * CometColumnarToRow (24) +- CometProject (23) +- CometBroadcastHashJoin (22) :- CometProject (17) @@ -142,7 +142,7 @@ Arguments: [sr_returned_date_sk#9], [d_date_sk#23], Inner, BuildRight Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, d_date_sk#23] Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -(24) ColumnarToRow [codegen id : 1] +(24) CometColumnarToRow [codegen id : 1] Input [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] (25) HashAggregate [codegen id : 1] @@ -171,7 +171,7 @@ Arguments: 100, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRS Subquery:1 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 BroadcastExchange (33) -+- * ColumnarToRow (32) ++- * CometColumnarToRow (32) +- CometProject (31) +- CometFilter (30) +- CometScan parquet spark_catalog.default.date_dim (29) @@ -192,7 +192,7 @@ Condition : ((((isnotnull(d_year#24) AND isnotnull(d_moy#25)) AND (d_year#24 = 2 Input [3]: [d_date_sk#23, d_year#24, d_moy#25] Arguments: [d_date_sk#23], [d_date_sk#23] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#23] (33) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt index aa3f7f2cf8..0cbfa6bc3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s Exchange [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #1 WholeStageCodegen (1) HashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sr_returned_date_sk,ss_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk] @@ -23,7 +23,7 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt index 78c30636d6..6901a5d8c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject (40) : +- * HashAggregate (12) : +- Exchange (11) : +- * HashAggregate (10) - : +- * ColumnarToRow (9) + : +- * CometColumnarToRow (9) : +- CometProject (8) : +- CometBroadcastHashJoin (7) : :- CometFilter (2) @@ -33,7 +33,7 @@ TakeOrderedAndProject (40) +- * HashAggregate (27) +- Exchange (26) +- * HashAggregate (25) - +- * ColumnarToRow (24) + +- * CometColumnarToRow (24) +- CometProject (23) +- CometBroadcastHashJoin (22) :- CometFilter (20) @@ -81,7 +81,7 @@ Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] Arguments: [ws_item_sk#1, ws_sales_price#2, d_date#6], [ws_item_sk#1, ws_sales_price#2, d_date#6] -(9) ColumnarToRow [codegen id : 1] +(9) CometColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] (10) HashAggregate [codegen id : 1] @@ -150,7 +150,7 @@ Arguments: [ss_sold_date_sk#16], [d_date_sk#18], Inner, BuildRight Input [5]: [ss_item_sk#14, ss_sales_price#15, ss_sold_date_sk#16, d_date_sk#18, d_date#19] Arguments: [ss_item_sk#14, ss_sales_price#15, d_date#19], [ss_item_sk#14, ss_sales_price#15, d_date#19] -(24) ColumnarToRow [codegen id : 6] +(24) CometColumnarToRow [codegen id : 6] Input [3]: [ss_item_sk#14, ss_sales_price#15, d_date#19] (25) HashAggregate [codegen id : 6] @@ -229,7 +229,7 @@ Arguments: 100, [item_sk#26 ASC NULLS FIRST, d_date#27 ASC NULLS FIRST], [item_s Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (45) -+- * ColumnarToRow (44) ++- * CometColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan parquet spark_catalog.default.date_dim (41) @@ -250,7 +250,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_mont Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(44) ColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt index f19312ca87..733bf28aaf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt @@ -29,7 +29,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store Exchange [ws_item_sk,d_date] #4 WholeStageCodegen (1) HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_item_sk,ws_sales_price,d_date] CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] @@ -38,7 +38,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] @@ -66,7 +66,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store Exchange [ss_item_sk,d_date] #9 WholeStageCodegen (6) HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_item_sk,ss_sales_price,d_date] CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/explain.txt index 9e017b9279..b9a0d54da2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (19) +- * HashAggregate (18) +- Exchange (17) +- * HashAggregate (16) - +- * ColumnarToRow (15) + +- * CometColumnarToRow (15) +- CometProject (14) +- CometBroadcastHashJoin (13) :- CometProject (8) @@ -88,7 +88,7 @@ Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#9] Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarToRow [codegen id : 1] Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] (16) HashAggregate [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt index d7661b7bef..5e79881384 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [d_year,ext_price,brand_id,brand] Exchange [d_year,i_brand,i_brand_id] #1 WholeStageCodegen (1) HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt index 631d21aa56..b40011ba15 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject (29) +- * HashAggregate (23) +- Exchange (22) +- * HashAggregate (21) - +- * ColumnarToRow (20) + +- * CometColumnarToRow (20) +- CometProject (19) +- CometBroadcastHashJoin (18) :- CometProject (14) @@ -122,7 +122,7 @@ Arguments: [ss_store_sk#11], [s_store_sk#18], Inner, BuildRight Input [5]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17, s_store_sk#18] Arguments: [i_manufact_id#5, ss_sales_price#12, d_qoy#17], [i_manufact_id#5, ss_sales_price#12, d_qoy#17] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarToRow [codegen id : 1] Input [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#17] (21) HashAggregate [codegen id : 1] @@ -171,7 +171,7 @@ Arguments: 100, [avg_quarterly_sales#24 ASC NULLS FIRST, sum_sales#22 ASC NULLS Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (34) -+- * ColumnarToRow (33) ++- * CometColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan parquet spark_catalog.default.date_dim (30) @@ -192,7 +192,7 @@ Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] -(33) ColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#15, d_qoy#17] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt index b90cb42d02..7284321cf4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] Exchange [i_manufact_id,d_qoy] #2 WholeStageCodegen (1) HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [i_manufact_id,ss_sales_price,d_qoy] CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] @@ -31,7 +31,7 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_qoy] CometFilter [d_date_sk,d_month_seq,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/explain.txt index a1fce65e88..058d447b30 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/explain.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject (56) +- * HashAggregate (52) +- Exchange (51) +- * HashAggregate (50) - +- * ColumnarToRow (49) + +- * CometColumnarToRow (49) +- CometProject (48) +- CometBroadcastHashJoin (47) :- CometProject (42) @@ -288,7 +288,7 @@ Arguments: [ss_sold_date_sk#24], [d_date_sk#31], Inner, BuildRight Input [4]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, d_date_sk#31] Arguments: [c_customer_sk#20, ss_ext_sales_price#23], [c_customer_sk#20, ss_ext_sales_price#23] -(49) ColumnarToRow [codegen id : 1] +(49) CometColumnarToRow [codegen id : 1] Input [2]: [c_customer_sk#20, ss_ext_sales_price#23] (50) HashAggregate [codegen id : 1] @@ -335,7 +335,7 @@ Arguments: 100, [segment#40 ASC NULLS FIRST, num_customers#44 ASC NULLS FIRST], Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (61) -+- * ColumnarToRow (60) ++- * CometColumnarToRow (60) +- CometProject (59) +- CometFilter (58) +- CometScan parquet spark_catalog.default.date_dim (57) @@ -356,7 +356,7 @@ Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12 Input [3]: [d_date_sk#17, d_year#18, d_moy#19] Arguments: [d_date_sk#17], [d_date_sk#17] -(60) ColumnarToRow [codegen id : 1] +(60) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#17] (61) BroadcastExchange @@ -367,7 +367,7 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dy Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 BroadcastExchange (66) -+- * ColumnarToRow (65) ++- * CometColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometScan parquet spark_catalog.default.date_dim (62) @@ -388,7 +388,7 @@ Condition : (((isnotnull(d_month_seq#32) AND (d_month_seq#32 >= ReusedSubquery S Input [2]: [d_date_sk#31, d_month_seq#32] Arguments: [d_date_sk#31], [d_date_sk#31] -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#31] (66) BroadcastExchange @@ -400,7 +400,7 @@ Subquery:4 Hosting operator id = 63 Hosting Expression = ReusedSubquery Subquery Subquery:5 Hosting operator id = 63 Hosting Expression = ReusedSubquery Subquery scalar-subquery#34, [id=#36] Subquery:6 Hosting operator id = 62 Hosting Expression = Subquery scalar-subquery#33, [id=#35] -* ColumnarToRow (73) +* CometColumnarToRow (73) +- CometHashAggregate (72) +- CometExchange (71) +- CometHashAggregate (70) @@ -438,11 +438,11 @@ Input [1]: [(d_month_seq + 1)#49] Keys [1]: [(d_month_seq + 1)#49] Functions: [] -(73) ColumnarToRow [codegen id : 1] +(73) CometColumnarToRow [codegen id : 1] Input [1]: [(d_month_seq + 1)#49] Subquery:7 Hosting operator id = 62 Hosting Expression = Subquery scalar-subquery#34, [id=#36] -* ColumnarToRow (80) +* CometColumnarToRow (80) +- CometHashAggregate (79) +- CometExchange (78) +- CometHashAggregate (77) @@ -480,7 +480,7 @@ Input [1]: [(d_month_seq + 3)#53] Keys [1]: [(d_month_seq + 3)#53] Functions: [] -(80) ColumnarToRow [codegen id : 1] +(80) CometColumnarToRow [codegen id : 1] Input [1]: [(d_month_seq + 3)#53] Subquery:8 Hosting operator id = 44 Hosting Expression = ReusedSubquery Subquery scalar-subquery#33, [id=#35] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt index 1dddb45efc..d6f71b841a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] Exchange [c_customer_sk] #2 WholeStageCodegen (1) HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [c_customer_sk,ss_ext_sales_price] CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] @@ -36,7 +36,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -62,7 +62,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -71,7 +71,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] Subquery #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [(d_month_seq + 1)] CometExchange [(d_month_seq + 1)] #10 @@ -81,7 +81,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] Subquery #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [(d_month_seq + 3)] CometExchange [(d_month_seq + 3)] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/explain.txt index fc018e3d16..b679124f99 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (19) +- * HashAggregate (18) +- Exchange (17) +- * HashAggregate (16) - +- * ColumnarToRow (15) + +- * CometColumnarToRow (15) +- CometProject (14) +- CometBroadcastHashJoin (13) :- CometProject (8) @@ -88,7 +88,7 @@ Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight Input [5]: [ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#9] Arguments: [ss_ext_sales_price#5, i_brand_id#8, i_brand#9], [ss_ext_sales_price#5, i_brand_id#8, i_brand#9] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarToRow [codegen id : 1] Input [3]: [ss_ext_sales_price#5, i_brand_id#8, i_brand#9] (16) HashAggregate [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt index 2461ab830b..fd8820237e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [ext_price,brand_id,brand] Exchange [i_brand,i_brand_id] #1 WholeStageCodegen (1) HashAggregate [i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_brand_id,i_brand] CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/explain.txt index 47fc0065ef..c85f1335ac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/explain.txt @@ -7,7 +7,7 @@ TakeOrderedAndProject (63) :- * HashAggregate (28) : +- Exchange (27) : +- * HashAggregate (26) - : +- * ColumnarToRow (25) + : +- * CometColumnarToRow (25) : +- CometProject (24) : +- CometBroadcastHashJoin (23) : :- CometProject (14) @@ -35,7 +35,7 @@ TakeOrderedAndProject (63) :- * HashAggregate (43) : +- Exchange (42) : +- * HashAggregate (41) - : +- * ColumnarToRow (40) + : +- * CometColumnarToRow (40) : +- CometProject (39) : +- CometBroadcastHashJoin (38) : :- CometProject (36) @@ -50,7 +50,7 @@ TakeOrderedAndProject (63) +- * HashAggregate (58) +- Exchange (57) +- * HashAggregate (56) - +- * ColumnarToRow (55) + +- * CometColumnarToRow (55) +- CometProject (54) +- CometBroadcastHashJoin (53) :- CometProject (51) @@ -180,7 +180,7 @@ Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_item_id#12] Arguments: [ss_ext_sales_price#3, i_item_id#12], [ss_ext_sales_price#3, i_item_id#12] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [2]: [ss_ext_sales_price#3, i_item_id#12] (26) HashAggregate [codegen id : 1] @@ -249,7 +249,7 @@ Arguments: [cs_item_sk#20], [i_item_sk#26], Inner, BuildRight Input [4]: [cs_item_sk#20, cs_ext_sales_price#21, i_item_sk#26, i_item_id#27] Arguments: [cs_ext_sales_price#21, i_item_id#27], [cs_ext_sales_price#21, i_item_id#27] -(40) ColumnarToRow [codegen id : 3] +(40) CometColumnarToRow [codegen id : 3] Input [2]: [cs_ext_sales_price#21, i_item_id#27] (41) HashAggregate [codegen id : 3] @@ -318,7 +318,7 @@ Arguments: [ws_item_sk#32], [i_item_sk#39], Inner, BuildRight Input [4]: [ws_item_sk#32, ws_ext_sales_price#34, i_item_sk#39, i_item_id#40] Arguments: [ws_ext_sales_price#34, i_item_id#40], [ws_ext_sales_price#34, i_item_id#40] -(55) ColumnarToRow [codegen id : 5] +(55) CometColumnarToRow [codegen id : 5] Input [2]: [ws_ext_sales_price#34, i_item_id#40] (56) HashAggregate [codegen id : 5] @@ -367,7 +367,7 @@ Arguments: 100, [total_sales#50 ASC NULLS FIRST], [i_item_id#12, total_sales#50] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (68) -+- * ColumnarToRow (67) ++- * CometColumnarToRow (67) +- CometProject (66) +- CometFilter (65) +- CometScan parquet spark_catalog.default.date_dim (64) @@ -388,7 +388,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(67) ColumnarToRow [codegen id : 1] +(67) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (68) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt index c4ce35e153..f5afdec6a7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject [total_sales,i_item_id] Exchange [i_item_id] #2 WholeStageCodegen (1) HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_item_id] CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] @@ -26,7 +26,7 @@ TakeOrderedAndProject [total_sales,i_item_id] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -53,7 +53,7 @@ TakeOrderedAndProject [total_sales,i_item_id] Exchange [i_item_id] #8 WholeStageCodegen (3) HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_ext_sales_price,i_item_id] CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] @@ -73,7 +73,7 @@ TakeOrderedAndProject [total_sales,i_item_id] Exchange [i_item_id] #9 WholeStageCodegen (5) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_ext_sales_price,i_item_id] CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt index 8423e825d4..8971318bc5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject (45) : : +- * HashAggregate (21) : : +- Exchange (20) : : +- * HashAggregate (19) - : : +- * ColumnarToRow (18) + : : +- * CometColumnarToRow (18) : : +- CometProject (17) : : +- CometBroadcastHashJoin (16) : : :- CometProject (12) @@ -130,7 +130,7 @@ Arguments: [cs_call_center_sk#4], [cc_call_center_sk#12], Inner, BuildRight Input [8]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11, cc_call_center_sk#12, cc_name#13] Arguments: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13], [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] -(18) ColumnarToRow [codegen id : 1] +(18) CometColumnarToRow [codegen id : 1] Input [6]: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] (19) HashAggregate [codegen id : 1] @@ -256,7 +256,7 @@ Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name# Subquery:1 Hosting operator id = 3 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (49) -+- * ColumnarToRow (48) ++- * CometColumnarToRow (48) +- CometFilter (47) +- CometScan parquet spark_catalog.default.date_dim (46) @@ -272,7 +272,7 @@ ReadSchema: struct Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(48) ColumnarToRow [codegen id : 1] +(48) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (49) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt index b64c2efdc5..059450de83 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ Exchange [i_category,i_brand,cc_name,d_year,d_moy] #2 WholeStageCodegen (1) HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] @@ -38,7 +38,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/explain.txt index 809315c9e7..120367c48d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/explain.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject (53) : : +- * HashAggregate (19) : : +- Exchange (18) : : +- * HashAggregate (17) - : : +- * ColumnarToRow (16) + : : +- * CometColumnarToRow (16) : : +- CometProject (15) : : +- CometBroadcastHashJoin (14) : : :- CometProject (7) @@ -29,7 +29,7 @@ TakeOrderedAndProject (53) : +- * HashAggregate (32) : +- Exchange (31) : +- * HashAggregate (30) - : +- * ColumnarToRow (29) + : +- * CometColumnarToRow (29) : +- CometProject (28) : +- CometBroadcastHashJoin (27) : :- CometProject (25) @@ -43,7 +43,7 @@ TakeOrderedAndProject (53) +- * HashAggregate (48) +- Exchange (47) +- * HashAggregate (46) - +- * ColumnarToRow (45) + +- * CometColumnarToRow (45) +- CometProject (44) +- CometBroadcastHashJoin (43) :- CometProject (41) @@ -126,7 +126,7 @@ Arguments: [ss_sold_date_sk#3], [d_date_sk#7], Inner, BuildRight Input [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, d_date_sk#7] Arguments: [ss_ext_sales_price#2, i_item_id#6], [ss_ext_sales_price#2, i_item_id#6] -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarToRow [codegen id : 1] Input [2]: [ss_ext_sales_price#2, i_item_id#6] (17) HashAggregate [codegen id : 1] @@ -187,7 +187,7 @@ Arguments: [cs_sold_date_sk#17], [d_date_sk#21], Inner, BuildRight Input [4]: [cs_ext_sales_price#16, cs_sold_date_sk#17, i_item_id#20, d_date_sk#21] Arguments: [cs_ext_sales_price#16, i_item_id#20], [cs_ext_sales_price#16, i_item_id#20] -(29) ColumnarToRow [codegen id : 2] +(29) CometColumnarToRow [codegen id : 2] Input [2]: [cs_ext_sales_price#16, i_item_id#20] (30) HashAggregate [codegen id : 2] @@ -262,7 +262,7 @@ Arguments: [ws_sold_date_sk#29], [d_date_sk#33], Inner, BuildRight Input [4]: [ws_ext_sales_price#28, ws_sold_date_sk#29, i_item_id#32, d_date_sk#33] Arguments: [ws_ext_sales_price#28, i_item_id#32], [ws_ext_sales_price#28, i_item_id#32] -(45) ColumnarToRow [codegen id : 4] +(45) CometColumnarToRow [codegen id : 4] Input [2]: [ws_ext_sales_price#28, i_item_id#32] (46) HashAggregate [codegen id : 4] @@ -309,7 +309,7 @@ Arguments: 100, [item_id#13 ASC NULLS FIRST, ss_item_rev#14 ASC NULLS FIRST], [i Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (63) -+- * ColumnarToRow (62) ++- * CometColumnarToRow (62) +- CometProject (61) +- CometBroadcastHashJoin (60) :- CometFilter (55) @@ -359,7 +359,7 @@ Arguments: [d_date#8], [d_date#9], LeftSemi, BuildRight Input [2]: [d_date_sk#7, d_date#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(62) ColumnarToRow [codegen id : 1] +(62) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (63) BroadcastExchange @@ -369,7 +369,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 57 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] Subquery:3 Hosting operator id = 56 Hosting Expression = Subquery scalar-subquery#44, [id=#45] -* ColumnarToRow (67) +* CometColumnarToRow (67) +- CometProject (66) +- CometFilter (65) +- CometScan parquet spark_catalog.default.date_dim (64) @@ -390,7 +390,7 @@ Condition : (isnotnull(d_date#46) AND (d_date#46 = 2000-01-03)) Input [2]: [d_date#46, d_week_seq#47] Arguments: [d_week_seq#47], [d_week_seq#47] -(67) ColumnarToRow [codegen id : 1] +(67) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#47] Subquery:4 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt index 0f5a783b94..b4de84ce61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev Exchange [i_item_id] #1 WholeStageCodegen (1) HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_item_id] CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,d_date_sk] @@ -21,7 +21,7 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometBroadcastHashJoin [d_date_sk,d_date,d_date] @@ -34,7 +34,7 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_date,d_week_seq] @@ -57,7 +57,7 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev Exchange [i_item_id] #7 WholeStageCodegen (2) HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_ext_sales_price,i_item_id] CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,d_date_sk] @@ -77,7 +77,7 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev Exchange [i_item_id] #9 WholeStageCodegen (4) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_ext_sales_price,i_item_id] CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/explain.txt index b471c34359..0a103605e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/explain.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject (43) : : :- * HashAggregate (11) : : : +- Exchange (10) : : : +- * HashAggregate (9) - : : : +- * ColumnarToRow (8) + : : : +- * CometColumnarToRow (8) : : : +- CometProject (7) : : : +- CometBroadcastHashJoin (6) : : : :- CometFilter (2) @@ -18,11 +18,11 @@ TakeOrderedAndProject (43) : : : +- CometFilter (4) : : : +- CometScan parquet spark_catalog.default.date_dim (3) : : +- BroadcastExchange (15) - : : +- * ColumnarToRow (14) + : : +- * CometColumnarToRow (14) : : +- CometFilter (13) : : +- CometScan parquet spark_catalog.default.store (12) : +- BroadcastExchange (22) - : +- * ColumnarToRow (21) + : +- * CometColumnarToRow (21) : +- CometProject (20) : +- CometFilter (19) : +- CometScan parquet spark_catalog.default.date_dim (18) @@ -34,11 +34,11 @@ TakeOrderedAndProject (43) : :- * HashAggregate (26) : : +- ReusedExchange (25) : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) + : +- * CometColumnarToRow (29) : +- CometFilter (28) : +- CometScan parquet spark_catalog.default.store (27) +- BroadcastExchange (37) - +- * ColumnarToRow (36) + +- * CometColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometScan parquet spark_catalog.default.date_dim (33) @@ -80,7 +80,7 @@ Arguments: [ss_sold_date_sk#3], [d_date_sk#4], Inner, BuildRight Input [6]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3, d_date_sk#4, d_week_seq#5, d_day_name#6] Arguments: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#6], [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#6] -(8) ColumnarToRow [codegen id : 1] +(8) CometColumnarToRow [codegen id : 1] Input [4]: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#6] (9) HashAggregate [codegen id : 1] @@ -112,7 +112,7 @@ ReadSchema: struct Input [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] Condition : (isnotnull(s_store_sk#35) AND isnotnull(s_store_id#36)) -(14) ColumnarToRow [codegen id : 2] +(14) CometColumnarToRow [codegen id : 2] Input [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] (15) BroadcastExchange @@ -144,7 +144,7 @@ Condition : (((isnotnull(d_month_seq#38) AND (d_month_seq#38 >= 1212)) AND (d_mo Input [2]: [d_month_seq#38, d_week_seq#39] Arguments: [d_week_seq#39], [d_week_seq#39] -(21) ColumnarToRow [codegen id : 3] +(21) CometColumnarToRow [codegen id : 3] Input [1]: [d_week_seq#39] (22) BroadcastExchange @@ -182,7 +182,7 @@ ReadSchema: struct Input [2]: [s_store_sk#68, s_store_id#69] Condition : (isnotnull(s_store_sk#68) AND isnotnull(s_store_id#69)) -(29) ColumnarToRow [codegen id : 5] +(29) CometColumnarToRow [codegen id : 5] Input [2]: [s_store_sk#68, s_store_id#69] (30) BroadcastExchange @@ -214,7 +214,7 @@ Condition : (((isnotnull(d_month_seq#70) AND (d_month_seq#70 >= 1224)) AND (d_mo Input [2]: [d_month_seq#70, d_week_seq#71] Arguments: [d_week_seq#71], [d_week_seq#71] -(36) ColumnarToRow [codegen id : 6] +(36) CometColumnarToRow [codegen id : 6] Input [1]: [d_week_seq#71] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt index 58bae46d1e..0989c806c2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt @@ -11,7 +11,7 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s Exchange [d_week_seq,ss_store_sk] #1 WholeStageCodegen (1) HashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] CometBroadcastHashJoin [ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_week_seq,d_day_name] @@ -23,14 +23,14 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_month_seq,d_week_seq] @@ -48,14 +48,14 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s InputAdapter BroadcastExchange #6 WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (6) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_month_seq,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/explain.txt index 80ba7d7f4f..f26ff00aa6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/explain.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject (40) +- * HashAggregate (36) +- * Project (35) +- * BroadcastHashJoin Inner BuildRight (34) - :- * ColumnarToRow (19) + :- * CometColumnarToRow (19) : +- CometProject (18) : +- CometBroadcastHashJoin (17) : :- CometProject (12) @@ -28,7 +28,7 @@ TakeOrderedAndProject (40) +- BroadcastExchange (33) +- * Project (32) +- * BroadcastHashJoin Inner BuildRight (31) - :- * ColumnarToRow (22) + :- * CometColumnarToRow (22) : +- CometFilter (21) : +- CometScan parquet spark_catalog.default.item (20) +- BroadcastExchange (30) @@ -36,7 +36,7 @@ TakeOrderedAndProject (40) +- * HashAggregate (28) +- Exchange (27) +- * HashAggregate (26) - +- * ColumnarToRow (25) + +- * CometColumnarToRow (25) +- CometFilter (24) +- CometScan parquet spark_catalog.default.item (23) @@ -129,7 +129,7 @@ Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight Input [4]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7, d_date_sk#9] Arguments: [ca_state#2, ss_item_sk#5], [ca_state#2, ss_item_sk#5] -(19) ColumnarToRow [codegen id : 4] +(19) CometColumnarToRow [codegen id : 4] Input [2]: [ca_state#2, ss_item_sk#5] (20) CometScan parquet spark_catalog.default.item @@ -143,7 +143,7 @@ ReadSchema: struct Input [3]: [i_item_sk#13, i_current_price#14, i_category#15] Condition : ((isnotnull(i_current_price#14) AND isnotnull(i_category#15)) AND isnotnull(i_item_sk#13)) -(22) ColumnarToRow [codegen id : 3] +(22) CometColumnarToRow [codegen id : 3] Input [3]: [i_item_sk#13, i_current_price#14, i_category#15] (23) CometScan parquet spark_catalog.default.item @@ -157,7 +157,7 @@ ReadSchema: struct Input [2]: [i_current_price#16, i_category#17] Condition : isnotnull(i_category#17) -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [2]: [i_current_price#16, i_category#17] (26) HashAggregate [codegen id : 1] @@ -240,7 +240,7 @@ Arguments: 100, [cnt#28 ASC NULLS FIRST], [state#27, cnt#28] Subquery:1 Hosting operator id = 8 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (45) -+- * ColumnarToRow (44) ++- * CometColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan parquet spark_catalog.default.date_dim (41) @@ -261,7 +261,7 @@ Condition : ((isnotnull(d_month_seq#10) AND (d_month_seq#10 = ReusedSubquery Sub Input [2]: [d_date_sk#9, d_month_seq#10] Arguments: [d_date_sk#9], [d_date_sk#9] -(44) ColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] (45) BroadcastExchange @@ -271,7 +271,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 42 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] Subquery:3 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* ColumnarToRow (52) +* CometColumnarToRow (52) +- CometHashAggregate (51) +- CometExchange (50) +- CometHashAggregate (49) @@ -309,7 +309,7 @@ Input [1]: [d_month_seq#29] Keys [1]: [d_month_seq#29] Functions: [] -(52) ColumnarToRow [codegen id : 1] +(52) CometColumnarToRow [codegen id : 1] Input [1]: [d_month_seq#29] Subquery:4 Hosting operator id = 14 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt index c5bbfc7d58..93e0567ac4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject [cnt,state] HashAggregate [ca_state] [count,count] Project [ca_state] BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ca_state,ss_item_sk] CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] @@ -27,7 +27,7 @@ TakeOrderedAndProject [cnt,state] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -35,7 +35,7 @@ TakeOrderedAndProject [cnt,state] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [d_month_seq] CometExchange [d_month_seq] #5 @@ -54,7 +54,7 @@ TakeOrderedAndProject [cnt,state] WholeStageCodegen (3) Project [i_item_sk] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [i_item_sk,i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] @@ -67,7 +67,7 @@ TakeOrderedAndProject [cnt,state] Exchange [i_category] #9 WholeStageCodegen (1) HashAggregate [i_category,i_current_price] [sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/explain.txt index 09e20ef233..c8a1391653 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/explain.txt @@ -7,7 +7,7 @@ TakeOrderedAndProject (63) :- * HashAggregate (28) : +- Exchange (27) : +- * HashAggregate (26) - : +- * ColumnarToRow (25) + : +- * CometColumnarToRow (25) : +- CometProject (24) : +- CometBroadcastHashJoin (23) : :- CometProject (14) @@ -35,7 +35,7 @@ TakeOrderedAndProject (63) :- * HashAggregate (43) : +- Exchange (42) : +- * HashAggregate (41) - : +- * ColumnarToRow (40) + : +- * CometColumnarToRow (40) : +- CometProject (39) : +- CometBroadcastHashJoin (38) : :- CometProject (36) @@ -50,7 +50,7 @@ TakeOrderedAndProject (63) +- * HashAggregate (58) +- Exchange (57) +- * HashAggregate (56) - +- * ColumnarToRow (55) + +- * CometColumnarToRow (55) +- CometProject (54) +- CometBroadcastHashJoin (53) :- CometProject (51) @@ -180,7 +180,7 @@ Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_item_id#12] Arguments: [ss_ext_sales_price#3, i_item_id#12], [ss_ext_sales_price#3, i_item_id#12] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [2]: [ss_ext_sales_price#3, i_item_id#12] (26) HashAggregate [codegen id : 1] @@ -249,7 +249,7 @@ Arguments: [cs_item_sk#20], [i_item_sk#26], Inner, BuildRight Input [4]: [cs_item_sk#20, cs_ext_sales_price#21, i_item_sk#26, i_item_id#27] Arguments: [cs_ext_sales_price#21, i_item_id#27], [cs_ext_sales_price#21, i_item_id#27] -(40) ColumnarToRow [codegen id : 3] +(40) CometColumnarToRow [codegen id : 3] Input [2]: [cs_ext_sales_price#21, i_item_id#27] (41) HashAggregate [codegen id : 3] @@ -318,7 +318,7 @@ Arguments: [ws_item_sk#32], [i_item_sk#39], Inner, BuildRight Input [4]: [ws_item_sk#32, ws_ext_sales_price#34, i_item_sk#39, i_item_id#40] Arguments: [ws_ext_sales_price#34, i_item_id#40], [ws_ext_sales_price#34, i_item_id#40] -(55) ColumnarToRow [codegen id : 5] +(55) CometColumnarToRow [codegen id : 5] Input [2]: [ws_ext_sales_price#34, i_item_id#40] (56) HashAggregate [codegen id : 5] @@ -367,7 +367,7 @@ Arguments: 100, [i_item_id#12 ASC NULLS FIRST, total_sales#50 ASC NULLS FIRST], Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (68) -+- * ColumnarToRow (67) ++- * CometColumnarToRow (67) +- CometProject (66) +- CometFilter (65) +- CometScan parquet spark_catalog.default.date_dim (64) @@ -388,7 +388,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(67) ColumnarToRow [codegen id : 1] +(67) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (68) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt index 51025cf34f..e79d027908 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject [i_item_id,total_sales] Exchange [i_item_id] #2 WholeStageCodegen (1) HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_item_id] CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] @@ -26,7 +26,7 @@ TakeOrderedAndProject [i_item_id,total_sales] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -53,7 +53,7 @@ TakeOrderedAndProject [i_item_id,total_sales] Exchange [i_item_id] #8 WholeStageCodegen (3) HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_ext_sales_price,i_item_id] CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] @@ -73,7 +73,7 @@ TakeOrderedAndProject [i_item_id,total_sales] Exchange [i_item_id] #9 WholeStageCodegen (5) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_ext_sales_price,i_item_id] CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/explain.txt index 098b72016b..d80fce5073 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/explain.txt @@ -4,7 +4,7 @@ :- * HashAggregate (41) : +- Exchange (40) : +- * HashAggregate (39) - : +- * ColumnarToRow (38) + : +- * CometColumnarToRow (38) : +- CometProject (37) : +- CometBroadcastHashJoin (36) : :- CometProject (31) @@ -46,7 +46,7 @@ +- * HashAggregate (62) +- Exchange (61) +- * HashAggregate (60) - +- * ColumnarToRow (59) + +- * CometColumnarToRow (59) +- CometProject (58) +- CometBroadcastHashJoin (57) :- CometProject (55) @@ -242,7 +242,7 @@ Arguments: [ss_item_sk#1], [i_item_sk#21], Inner, BuildRight Input [3]: [ss_item_sk#1, ss_ext_sales_price#5, i_item_sk#21] Arguments: [ss_ext_sales_price#5], [ss_ext_sales_price#5] -(38) ColumnarToRow [codegen id : 1] +(38) CometColumnarToRow [codegen id : 1] Input [1]: [ss_ext_sales_price#5] (39) HashAggregate [codegen id : 1] @@ -335,7 +335,7 @@ Arguments: [ss_item_sk#27], [i_item_sk#38], Inner, BuildRight Input [3]: [ss_item_sk#27, ss_ext_sales_price#30, i_item_sk#38] Arguments: [ss_ext_sales_price#30], [ss_ext_sales_price#30] -(59) ColumnarToRow [codegen id : 2] +(59) CometColumnarToRow [codegen id : 2] Input [1]: [ss_ext_sales_price#30] (60) HashAggregate [codegen id : 2] @@ -372,7 +372,7 @@ Input [2]: [promotions#26, total#42] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (70) -+- * ColumnarToRow (69) ++- * CometColumnarToRow (69) +- CometProject (68) +- CometFilter (67) +- CometScan parquet spark_catalog.default.date_dim (66) @@ -393,7 +393,7 @@ Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1 Input [3]: [d_date_sk#14, d_year#15, d_moy#16] Arguments: [d_date_sk#14], [d_date_sk#14] -(69) ColumnarToRow [codegen id : 1] +(69) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] (70) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt index 8d39d9e4ea..ea5ca5f307 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt @@ -6,7 +6,7 @@ WholeStageCodegen (4) Exchange #1 WholeStageCodegen (1) HashAggregate [ss_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_ext_sales_price] CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] @@ -25,7 +25,7 @@ WholeStageCodegen (4) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -61,7 +61,7 @@ WholeStageCodegen (4) Exchange #10 WholeStageCodegen (2) HashAggregate [ss_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_ext_sales_price] CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/explain.txt index 274be3d0fd..9b1b01d4d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (28) +- * HashAggregate (27) +- Exchange (26) +- * HashAggregate (25) - +- * ColumnarToRow (24) + +- * CometColumnarToRow (24) +- CometProject (23) +- CometBroadcastHashJoin (22) :- CometProject (17) @@ -140,7 +140,7 @@ Arguments: [ws_ship_date_sk#1], [d_date_sk#12], Inner, BuildRight Input [6]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9, web_name#11, d_date_sk#12] Arguments: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#9, web_name#11, _groupingexpression#14], [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#9, web_name#11, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#14] -(24) ColumnarToRow [codegen id : 1] +(24) CometColumnarToRow [codegen id : 1] Input [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#9, web_name#11, _groupingexpression#14] (25) HashAggregate [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt index b9e35e2825..7bc2207d50 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days Exchange [_groupingexpression,sm_type,web_name] #1 WholeStageCodegen (1) HashAggregate [_groupingexpression,sm_type,web_name,ws_ship_date_sk,ws_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression] CometBroadcastHashJoin [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt index 3124ad22e9..08397ae1e7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject (29) +- * HashAggregate (23) +- Exchange (22) +- * HashAggregate (21) - +- * ColumnarToRow (20) + +- * CometColumnarToRow (20) +- CometProject (19) +- CometBroadcastHashJoin (18) :- CometProject (14) @@ -122,7 +122,7 @@ Arguments: [ss_store_sk#11], [s_store_sk#18], Inner, BuildRight Input [5]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17, s_store_sk#18] Arguments: [i_manager_id#5, ss_sales_price#12, d_moy#17], [i_manager_id#5, ss_sales_price#12, d_moy#17] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarToRow [codegen id : 1] Input [3]: [i_manager_id#5, ss_sales_price#12, d_moy#17] (21) HashAggregate [codegen id : 1] @@ -171,7 +171,7 @@ Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#24 ASC NULLS Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (34) -+- * ColumnarToRow (33) ++- * CometColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan parquet spark_catalog.default.date_dim (30) @@ -192,7 +192,7 @@ Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] -(33) ColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#15, d_moy#17] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt index e84d3c2a05..36760bdc35 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] Exchange [i_manager_id,d_moy] #2 WholeStageCodegen (1) HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [i_manager_id,ss_sales_price,d_moy] CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] @@ -31,7 +31,7 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_month_seq,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt index a32680a584..0c943f9bbd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (181) +* CometColumnarToRow (181) +- CometSort (180) +- CometColumnarExchange (179) +- * Project (178) @@ -40,7 +40,7 @@ : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (34) : : : : : : : : : : : : : : : :- * Project (32) : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (31) - : : : : : : : : : : : : : : : : :- * ColumnarToRow (11) + : : : : : : : : : : : : : : : : :- * CometColumnarToRow (11) : : : : : : : : : : : : : : : : : +- CometSort (10) : : : : : : : : : : : : : : : : : +- CometExchange (9) : : : : : : : : : : : : : : : : : +- CometProject (8) @@ -57,7 +57,7 @@ : : : : : : : : : : : : : : : : +- * HashAggregate (27) : : : : : : : : : : : : : : : : +- Exchange (26) : : : : : : : : : : : : : : : : +- * HashAggregate (25) - : : : : : : : : : : : : : : : : +- * ColumnarToRow (24) + : : : : : : : : : : : : : : : : +- * CometColumnarToRow (24) : : : : : : : : : : : : : : : : +- CometProject (23) : : : : : : : : : : : : : : : : +- CometSortMergeJoin (22) : : : : : : : : : : : : : : : : :- CometSort (16) @@ -72,44 +72,44 @@ : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (17) : : : : : : : : : : : : : : : +- ReusedExchange (33) : : : : : : : : : : : : : : +- BroadcastExchange (39) - : : : : : : : : : : : : : : +- * ColumnarToRow (38) + : : : : : : : : : : : : : : +- * CometColumnarToRow (38) : : : : : : : : : : : : : : +- CometFilter (37) : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (36) : : : : : : : : : : : : : +- BroadcastExchange (45) - : : : : : : : : : : : : : +- * ColumnarToRow (44) + : : : : : : : : : : : : : +- * CometColumnarToRow (44) : : : : : : : : : : : : : +- CometFilter (43) : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (42) : : : : : : : : : : : : +- BroadcastExchange (51) - : : : : : : : : : : : : +- * ColumnarToRow (50) + : : : : : : : : : : : : +- * CometColumnarToRow (50) : : : : : : : : : : : : +- CometFilter (49) : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (48) : : : : : : : : : : : +- ReusedExchange (54) : : : : : : : : : : +- BroadcastExchange (60) - : : : : : : : : : : +- * ColumnarToRow (59) + : : : : : : : : : : +- * CometColumnarToRow (59) : : : : : : : : : : +- CometFilter (58) : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (57) : : : : : : : : : +- ReusedExchange (63) : : : : : : : : +- BroadcastExchange (69) - : : : : : : : : +- * ColumnarToRow (68) + : : : : : : : : +- * CometColumnarToRow (68) : : : : : : : : +- CometFilter (67) : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (66) : : : : : : : +- BroadcastExchange (75) - : : : : : : : +- * ColumnarToRow (74) + : : : : : : : +- * CometColumnarToRow (74) : : : : : : : +- CometFilter (73) : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (72) : : : : : : +- ReusedExchange (78) : : : : : +- BroadcastExchange (84) - : : : : : +- * ColumnarToRow (83) + : : : : : +- * CometColumnarToRow (83) : : : : : +- CometFilter (82) : : : : : +- CometScan parquet spark_catalog.default.customer_address (81) : : : : +- ReusedExchange (87) : : : +- BroadcastExchange (93) - : : : +- * ColumnarToRow (92) + : : : +- * CometColumnarToRow (92) : : : +- CometFilter (91) : : : +- CometScan parquet spark_catalog.default.income_band (90) : : +- ReusedExchange (96) : +- BroadcastExchange (103) - : +- * ColumnarToRow (102) + : +- * CometColumnarToRow (102) : +- CometProject (101) : +- CometFilter (100) : +- CometScan parquet spark_catalog.default.item (99) @@ -149,7 +149,7 @@ : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (129) : : : : : : : : : : : : : : :- * Project (127) : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (126) - : : : : : : : : : : : : : : : :- * ColumnarToRow (120) + : : : : : : : : : : : : : : : :- * CometColumnarToRow (120) : : : : : : : : : : : : : : : : +- CometSort (119) : : : : : : : : : : : : : : : : +- CometExchange (118) : : : : : : : : : : : : : : : : +- CometProject (117) @@ -230,7 +230,7 @@ Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeSh Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] -(11) ColumnarToRow [codegen id : 1] +(11) CometColumnarToRow [codegen id : 1] Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] (12) CometScan parquet spark_catalog.default.catalog_sales @@ -288,7 +288,7 @@ Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number# Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] (25) HashAggregate [codegen id : 2] @@ -355,7 +355,7 @@ ReadSchema: struct Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] Condition : ((isnotnull(s_store_sk#39) AND isnotnull(s_store_name#40)) AND isnotnull(s_zip#41)) -(38) ColumnarToRow [codegen id : 5] +(38) CometColumnarToRow [codegen id : 5] Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] (39) BroadcastExchange @@ -383,7 +383,7 @@ ReadSchema: struct Input [2]: [d_date_sk#48, d_year#49] Condition : isnotnull(d_date_sk#48) -(50) ColumnarToRow [codegen id : 7] +(50) CometColumnarToRow [codegen id : 7] Input [2]: [d_date_sk#48, d_year#49] (51) BroadcastExchange @@ -452,7 +452,7 @@ ReadSchema: struct Input [2]: [cd_demo_sk#52, cd_marital_status#53] Condition : (isnotnull(cd_demo_sk#52) AND isnotnull(cd_marital_status#53)) -(59) ColumnarToRow [codegen id : 9] +(59) CometColumnarToRow [codegen id : 9] Input [2]: [cd_demo_sk#52, cd_marital_status#53] (60) BroadcastExchange @@ -493,7 +493,7 @@ ReadSchema: struct Input [1]: [p_promo_sk#56] Condition : isnotnull(p_promo_sk#56) -(68) ColumnarToRow [codegen id : 11] +(68) CometColumnarToRow [codegen id : 11] Input [1]: [p_promo_sk#56] (69) BroadcastExchange @@ -521,7 +521,7 @@ ReadSchema: struct Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] Condition : (isnotnull(hd_demo_sk#57) AND isnotnull(hd_income_band_sk#58)) -(74) ColumnarToRow [codegen id : 12] +(74) CometColumnarToRow [codegen id : 12] Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] (75) BroadcastExchange @@ -562,7 +562,7 @@ ReadSchema: struct Input [1]: [ib_income_band_sk#71] Condition : isnotnull(ib_income_band_sk#71) -(92) ColumnarToRow [codegen id : 16] +(92) CometColumnarToRow [codegen id : 16] Input [1]: [ib_income_band_sk#71] (93) BroadcastExchange @@ -648,7 +648,7 @@ Condition : ((((((isnotnull(i_current_price#74) AND i_color#75 IN (purple Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] Arguments: [i_item_sk#73, i_product_name#76], [i_item_sk#73, i_product_name#76] -(102) ColumnarToRow [codegen id : 18] +(102) CometColumnarToRow [codegen id : 18] Input [2]: [i_item_sk#73, i_product_name#76] (103) BroadcastExchange @@ -735,7 +735,7 @@ Arguments: hashpartitioning(ss_item_sk#106, 5), ENSURE_REQUIREMENTS, CometNative Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] Arguments: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117], [ss_item_sk#106 ASC NULLS FIRST] -(120) ColumnarToRow [codegen id : 21] +(120) CometColumnarToRow [codegen id : 21] Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] (121) ReusedExchange [Reuses operator id: 26] @@ -1005,14 +1005,14 @@ Arguments: rangepartitioning(product_name#89 ASC NULLS FIRST, store_name#91 ASC Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#181, s2#182, s3#183, syear#179, cnt#180] Arguments: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#181, s2#182, s3#183, syear#179, cnt#180], [product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#180 ASC NULLS FIRST] -(181) ColumnarToRow [codegen id : 42] +(181) CometColumnarToRow [codegen id : 42] Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#181, s2#182, s3#183, syear#179, cnt#180] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (185) -+- * ColumnarToRow (184) ++- * CometColumnarToRow (184) +- CometFilter (183) +- CometScan parquet spark_catalog.default.date_dim (182) @@ -1028,7 +1028,7 @@ ReadSchema: struct Input [2]: [d_date_sk#37, d_year#38] Condition : ((isnotnull(d_year#38) AND (d_year#38 = 1999)) AND isnotnull(d_date_sk#37)) -(184) ColumnarToRow [codegen id : 1] +(184) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#37, d_year#38] (185) BroadcastExchange @@ -1037,7 +1037,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 110 Hosting Expression = ss_sold_date_sk#117 IN dynamicpruning#118 BroadcastExchange (189) -+- * ColumnarToRow (188) ++- * CometColumnarToRow (188) +- CometFilter (187) +- CometScan parquet spark_catalog.default.date_dim (186) @@ -1053,7 +1053,7 @@ ReadSchema: struct Input [2]: [d_date_sk#132, d_year#133] Condition : ((isnotnull(d_year#133) AND (d_year#133 = 2000)) AND isnotnull(d_date_sk#132)) -(188) ColumnarToRow [codegen id : 1] +(188) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#132, d_year#133] (189) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt index 1448eff58e..7e4271de48 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (42) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] CometColumnarExchange [product_name,store_name,cnt] #1 @@ -48,7 +48,7 @@ WholeStageCodegen (42) SortMergeJoin [ss_item_sk,cs_item_sk] InputAdapter WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometExchange [ss_item_sk] #3 @@ -60,7 +60,7 @@ WholeStageCodegen (42) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -77,7 +77,7 @@ WholeStageCodegen (42) Exchange [cs_item_sk] #6 WholeStageCodegen (2) HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] @@ -96,21 +96,21 @@ WholeStageCodegen (42) InputAdapter BroadcastExchange #9 WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [s_store_sk,s_store_name,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] InputAdapter BroadcastExchange #10 WholeStageCodegen (6) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter BroadcastExchange #11 WholeStageCodegen (7) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -119,7 +119,7 @@ WholeStageCodegen (42) InputAdapter BroadcastExchange #12 WholeStageCodegen (9) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [cd_demo_sk,cd_marital_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] @@ -128,14 +128,14 @@ WholeStageCodegen (42) InputAdapter BroadcastExchange #13 WholeStageCodegen (11) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [p_promo_sk] CometScan parquet spark_catalog.default.promotion [p_promo_sk] InputAdapter BroadcastExchange #14 WholeStageCodegen (12) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [hd_demo_sk,hd_income_band_sk] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] @@ -144,7 +144,7 @@ WholeStageCodegen (42) InputAdapter BroadcastExchange #15 WholeStageCodegen (14) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] @@ -153,7 +153,7 @@ WholeStageCodegen (42) InputAdapter BroadcastExchange #16 WholeStageCodegen (16) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [ib_income_band_sk] CometScan parquet spark_catalog.default.income_band [ib_income_band_sk] @@ -162,7 +162,7 @@ WholeStageCodegen (42) InputAdapter BroadcastExchange #17 WholeStageCodegen (18) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] CometFilter [i_item_sk,i_current_price,i_color,i_product_name] @@ -209,7 +209,7 @@ WholeStageCodegen (42) SortMergeJoin [ss_item_sk,cs_item_sk] InputAdapter WholeStageCodegen (21) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometExchange [ss_item_sk] #19 @@ -221,7 +221,7 @@ WholeStageCodegen (42) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #21 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/explain.txt index e72d488697..245141857f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/explain.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject (42) : +- * BroadcastHashJoin Inner BuildRight (24) : :- * Project (19) : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * ColumnarToRow (3) + : : :- * CometColumnarToRow (3) : : : +- CometFilter (2) : : : +- CometScan parquet spark_catalog.default.store (1) : : +- BroadcastExchange (17) @@ -14,7 +14,7 @@ TakeOrderedAndProject (42) : : +- * HashAggregate (15) : : +- Exchange (14) : : +- * HashAggregate (13) - : : +- * ColumnarToRow (12) + : : +- * CometColumnarToRow (12) : : +- CometProject (11) : : +- CometBroadcastHashJoin (10) : : :- CometFilter (5) @@ -24,7 +24,7 @@ TakeOrderedAndProject (42) : : +- CometFilter (7) : : +- CometScan parquet spark_catalog.default.date_dim (6) : +- BroadcastExchange (23) - : +- * ColumnarToRow (22) + : +- * CometColumnarToRow (22) : +- CometFilter (21) : +- CometScan parquet spark_catalog.default.item (20) +- BroadcastExchange (39) @@ -35,7 +35,7 @@ TakeOrderedAndProject (42) +- * HashAggregate (34) +- Exchange (33) +- * HashAggregate (32) - +- * ColumnarToRow (31) + +- * CometColumnarToRow (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (27) @@ -54,7 +54,7 @@ ReadSchema: struct Input [2]: [s_store_sk#1, s_store_name#2] Condition : isnotnull(s_store_sk#1) -(3) ColumnarToRow [codegen id : 7] +(3) CometColumnarToRow [codegen id : 7] Input [2]: [s_store_sk#1, s_store_name#2] (4) CometScan parquet spark_catalog.default.store_sales @@ -97,7 +97,7 @@ Arguments: [ss_sold_date_sk#6], [d_date_sk#8], Inner, BuildRight Input [5]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6, d_date_sk#8] Arguments: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5], [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] -(12) ColumnarToRow [codegen id : 1] +(12) CometColumnarToRow [codegen id : 1] Input [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] (13) HashAggregate [codegen id : 1] @@ -147,7 +147,7 @@ ReadSchema: struct= 1176)) AND (d_mont Input [2]: [d_date_sk#8, d_month_seq#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(46) ColumnarToRow [codegen id : 1] +(46) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#8] (47) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt index 84d23af42e..96648cd2fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes BroadcastHashJoin [ss_item_sk,i_item_sk] Project [s_store_name,ss_store_sk,ss_item_sk,revenue] BroadcastHashJoin [s_store_sk,ss_store_sk] - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] @@ -19,7 +19,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes Exchange [ss_store_sk,ss_item_sk] #2 WholeStageCodegen (1) HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_item_sk,ss_store_sk,ss_sales_price] CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] @@ -28,7 +28,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -40,7 +40,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes InputAdapter BroadcastExchange #5 WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] @@ -58,7 +58,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes Exchange [ss_store_sk,ss_item_sk] #8 WholeStageCodegen (4) HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_item_sk,ss_store_sk,ss_sales_price] CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/explain.txt index 05f7b272ba..dae3d4fd9e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/explain.txt @@ -7,7 +7,7 @@ TakeOrderedAndProject (51) :- * HashAggregate (28) : +- Exchange (27) : +- * HashAggregate (26) - : +- * ColumnarToRow (25) + : +- * CometColumnarToRow (25) : +- CometProject (24) : +- CometBroadcastHashJoin (23) : :- CometProject (18) @@ -35,7 +35,7 @@ TakeOrderedAndProject (51) +- * HashAggregate (46) +- Exchange (45) +- * HashAggregate (44) - +- * ColumnarToRow (43) + +- * CometColumnarToRow (43) +- CometProject (42) +- CometBroadcastHashJoin (41) :- CometProject (39) @@ -168,7 +168,7 @@ Arguments: [ws_ship_mode_sk#2], [sm_ship_mode_sk#21], Inner, BuildRight Input [13]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18, sm_ship_mode_sk#21] Arguments: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18], [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] (26) HashAggregate [codegen id : 1] @@ -249,7 +249,7 @@ Arguments: [cs_ship_mode_sk#170], [sm_ship_mode_sk#188], Inner, BuildRight Input [13]: [cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186, sm_ship_mode_sk#188] Arguments: [cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186], [cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] -(43) ColumnarToRow [codegen id : 3] +(43) CometColumnarToRow [codegen id : 3] Input [11]: [cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] (44) HashAggregate [codegen id : 3] @@ -298,7 +298,7 @@ Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_w Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (55) -+- * ColumnarToRow (54) ++- * CometColumnarToRow (54) +- CometFilter (53) +- CometScan parquet spark_catalog.default.date_dim (52) @@ -314,7 +314,7 @@ ReadSchema: struct Input [3]: [d_date_sk#16, d_year#17, d_moy#18] Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) -(54) ColumnarToRow [codegen id : 1] +(54) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#16, d_year#17, d_moy#18] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt index 880c38bbf8..57b1f95771 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 WholeStageCodegen (1) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] CometBroadcastHashJoin [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk] @@ -28,7 +28,7 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] @@ -52,7 +52,7 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #8 WholeStageCodegen (3) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] CometBroadcastHashJoin [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/explain.txt index e07d18f0d2..060f07d258 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/explain.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject (31) +- * HashAggregate (23) +- Exchange (22) +- * HashAggregate (21) - +- * ColumnarToRow (20) + +- * CometColumnarToRow (20) +- CometExpand (19) +- CometProject (18) +- CometBroadcastHashJoin (17) @@ -124,7 +124,7 @@ Arguments: [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand# Input [10]: [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13] Arguments: [[ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, 0], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, null, 1], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, null, null, 3], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, null, null, null, 7], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, null, null, null, null, 15], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, null, null, null, null, null, 31], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, null, null, null, null, null, null, 63], [ss_quantity#3, ss_sales_price#4, i_category#17, null, null, null, null, null, null, null, 127], [ss_quantity#3, ss_sales_price#4, null, null, null, null, null, null, null, null, 255]], [ss_quantity#3, ss_sales_price#4, i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, spark_grouping_id#27] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarToRow [codegen id : 1] Input [11]: [ss_quantity#3, ss_sales_price#4, i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, spark_grouping_id#27] (21) HashAggregate [codegen id : 1] @@ -181,7 +181,7 @@ Arguments: 100, [i_category#19 ASC NULLS FIRST, i_class#20 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (36) -+- * ColumnarToRow (35) ++- * CometColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan parquet spark_catalog.default.date_dim (32) @@ -202,7 +202,7 @@ Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_mont Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(35) ColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [codegen id : 1] Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt index 46d9cf4ef6..ca88fff157 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 WholeStageCodegen (1) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometExpand [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] CometProject [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] @@ -30,7 +30,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/explain.txt index d27ff35d49..416a6892d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/explain.txt @@ -7,7 +7,7 @@ TakeOrderedAndProject (42) : :- * HashAggregate (29) : : +- Exchange (28) : : +- * HashAggregate (27) - : : +- * ColumnarToRow (26) + : : +- * CometColumnarToRow (26) : : +- CometProject (25) : : +- CometBroadcastHashJoin (24) : : :- CometProject (20) @@ -34,11 +34,11 @@ TakeOrderedAndProject (42) : : +- CometFilter (22) : : +- CometScan parquet spark_catalog.default.customer_address (21) : +- BroadcastExchange (33) - : +- * ColumnarToRow (32) + : +- * CometColumnarToRow (32) : +- CometFilter (31) : +- CometScan parquet spark_catalog.default.customer (30) +- BroadcastExchange (39) - +- * ColumnarToRow (38) + +- * CometColumnarToRow (38) +- CometFilter (37) +- CometScan parquet spark_catalog.default.customer_address (36) @@ -163,7 +163,7 @@ Arguments: [ss_addr_sk#3], [ca_address_sk#19], Inner, BuildRight Input [8]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_address_sk#19, ca_city#20] Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20] (27) HashAggregate [codegen id : 1] @@ -195,7 +195,7 @@ ReadSchema: struct Input [2]: [ca_address_sk#38, ca_city#39] Condition : (isnotnull(ca_address_sk#38) AND isnotnull(ca_city#39)) -(38) ColumnarToRow [codegen id : 3] +(38) CometColumnarToRow [codegen id : 3] Input [2]: [ca_address_sk#38, ca_city#39] (39) BroadcastExchange @@ -248,7 +248,7 @@ Arguments: 100, [c_last_name#37 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FI Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (47) -+- * ColumnarToRow (46) ++- * CometColumnarToRow (46) +- CometProject (45) +- CometFilter (44) +- CometScan parquet spark_catalog.default.date_dim (43) @@ -269,7 +269,7 @@ Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AN Input [3]: [d_date_sk#11, d_year#12, d_dom#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(46) ColumnarToRow [codegen id : 1] +(46) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (47) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt index 78e1b1ccea..805368ddf8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 WholeStageCodegen (1) HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [sum,sum,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_address_sk,ca_city] @@ -24,7 +24,7 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] @@ -47,14 +47,14 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ InputAdapter BroadcastExchange #7 WholeStageCodegen (2) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/explain.txt index 5386a3d166..eaf61eb735 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/explain.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject (44) : :- * Project (27) : : +- * BroadcastHashJoin LeftAnti BuildRight (26) : : :- * BroadcastHashJoin LeftAnti BuildRight (19) - : : : :- * ColumnarToRow (12) + : : : :- * CometColumnarToRow (12) : : : : +- CometBroadcastHashJoin (11) : : : : :- CometFilter (2) : : : : : +- CometScan parquet spark_catalog.default.customer (1) @@ -23,24 +23,24 @@ TakeOrderedAndProject (44) : : : : +- CometFilter (5) : : : : +- CometScan parquet spark_catalog.default.date_dim (4) : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) + : : : +- * CometColumnarToRow (17) : : : +- CometProject (16) : : : +- CometBroadcastHashJoin (15) : : : :- CometScan parquet spark_catalog.default.web_sales (13) : : : +- ReusedExchange (14) : : +- BroadcastExchange (25) - : : +- * ColumnarToRow (24) + : : +- * CometColumnarToRow (24) : : +- CometProject (23) : : +- CometBroadcastHashJoin (22) : : :- CometScan parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (32) - : +- * ColumnarToRow (31) + : +- * CometColumnarToRow (31) : +- CometProject (30) : +- CometFilter (29) : +- CometScan parquet spark_catalog.default.customer_address (28) +- BroadcastExchange (38) - +- * ColumnarToRow (37) + +- * CometColumnarToRow (37) +- CometFilter (36) +- CometScan parquet spark_catalog.default.customer_demographics (35) @@ -100,7 +100,7 @@ Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Right output [1]: [ss_customer_sk#4] Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight -(12) ColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 5] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (13) CometScan parquet spark_catalog.default.web_sales @@ -122,7 +122,7 @@ Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] Arguments: [ws_bill_customer_sk#10], [ws_bill_customer_sk#10] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#10] (18) BroadcastExchange @@ -154,7 +154,7 @@ Arguments: [cs_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] Arguments: [cs_ship_customer_sk#14], [cs_ship_customer_sk#14] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [1]: [cs_ship_customer_sk#14] (25) BroadcastExchange @@ -186,7 +186,7 @@ Condition : (ca_state#19 IN (KY,GA,NM) AND isnotnull(ca_address_sk#18)) Input [2]: [ca_address_sk#18, ca_state#19] Arguments: [ca_address_sk#18], [ca_address_sk#18] -(31) ColumnarToRow [codegen id : 3] +(31) CometColumnarToRow [codegen id : 3] Input [1]: [ca_address_sk#18] (32) BroadcastExchange @@ -214,7 +214,7 @@ ReadSchema: struct Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) ColumnarToRow [codegen id : 4] +(12) CometColumnarToRow [codegen id : 4] Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) CometScan parquet spark_catalog.default.store_sales @@ -151,7 +151,7 @@ Arguments: [ss_sold_date_sk#12], [d_date_sk#16], Inner, BuildRight Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#15, d_date_sk#16] Arguments: [ss_net_profit#11, s_state#15], [ss_net_profit#11, s_state#15] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarToRow [codegen id : 1] Input [2]: [ss_net_profit#11, s_state#15] (24) HashAggregate [codegen id : 1] @@ -262,7 +262,7 @@ Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (lochierarchy#29 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (50) -+- * ColumnarToRow (49) ++- * CometColumnarToRow (49) +- CometProject (48) +- CometFilter (47) +- CometScan parquet spark_catalog.default.date_dim (46) @@ -283,7 +283,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(49) ColumnarToRow [codegen id : 1] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (50) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt index 4a6dffc082..34fb81acc6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count Expand [ss_net_profit,s_state,s_county] Project [ss_net_profit,s_state,s_county] BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] @@ -25,7 +25,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -38,7 +38,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [s_state,s_state] - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] @@ -57,7 +57,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count Exchange [s_state] #7 WholeStageCodegen (1) HashAggregate [s_state,ss_net_profit] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_net_profit,s_state] CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/explain.txt index a36aa81621..422aa544f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/explain.txt @@ -1,11 +1,11 @@ == Physical Plan == -* ColumnarToRow (38) +* CometColumnarToRow (38) +- CometSort (37) +- CometColumnarExchange (36) +- * HashAggregate (35) +- Exchange (34) +- * HashAggregate (33) - +- * ColumnarToRow (32) + +- * CometColumnarToRow (32) +- CometProject (31) +- CometBroadcastHashJoin (30) :- CometProject (25) @@ -188,7 +188,7 @@ Arguments: [time_sk#15], [t_time_sk#34], Inner, BuildRight Input [7]: [i_brand_id#2, i_brand#3, ext_price#13, time_sk#15, t_time_sk#34, t_hour#35, t_minute#36] Arguments: [i_brand_id#2, i_brand#3, ext_price#13, t_hour#35, t_minute#36], [i_brand_id#2, i_brand#3, ext_price#13, t_hour#35, t_minute#36] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [5]: [i_brand_id#2, i_brand#3, ext_price#13, t_hour#35, t_minute#36] (33) HashAggregate [codegen id : 1] @@ -217,14 +217,14 @@ Arguments: rangepartitioning(ext_price#43 DESC NULLS LAST, brand_id#41 ASC NULLS Input [5]: [brand_id#41, brand#42, t_hour#35, t_minute#36, ext_price#43] Arguments: [brand_id#41, brand#42, t_hour#35, t_minute#36, ext_price#43], [ext_price#43 DESC NULLS LAST, brand_id#41 ASC NULLS FIRST] -(38) ColumnarToRow [codegen id : 3] +(38) CometColumnarToRow [codegen id : 3] Input [5]: [brand_id#41, brand#42, t_hour#35, t_minute#36, ext_price#43] ===== Subqueries ===== Subquery:1 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (43) -+- * ColumnarToRow (42) ++- * CometColumnarToRow (42) +- CometProject (41) +- CometFilter (40) +- CometScan parquet spark_catalog.default.date_dim (39) @@ -245,7 +245,7 @@ Condition : ((((isnotnull(d_moy#12) AND isnotnull(d_year#11)) AND (d_moy#12 = 11 Input [3]: [d_date_sk#10, d_year#11, d_moy#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (43) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt index 1e03b71083..3e50170e04 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [brand_id,brand,t_hour,t_minute,ext_price] CometColumnarExchange [ext_price,brand_id] #1 @@ -9,7 +9,7 @@ WholeStageCodegen (3) Exchange [i_brand,i_brand_id,t_hour,t_minute] #2 WholeStageCodegen (1) HashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] CometBroadcastHashJoin [i_brand_id,i_brand,ext_price,time_sk,t_time_sk,t_hour,t_minute] @@ -27,7 +27,7 @@ WholeStageCodegen (3) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/explain.txt index 6aae88fff3..4179f3b1d6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (64) +* CometColumnarToRow (64) +- CometTakeOrderedAndProject (63) +- CometHashAggregate (62) +- CometExchange (61) @@ -364,14 +364,14 @@ Functions [1]: [count(1)] Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, [total_cnt#37 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] -(64) ColumnarToRow [codegen id : 1] +(64) CometColumnarToRow [codegen id : 1] Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (69) -+- * ColumnarToRow (68) ++- * CometColumnarToRow (68) +- CometProject (67) +- CometFilter (66) +- CometScan parquet spark_catalog.default.date_dim (65) @@ -392,7 +392,7 @@ Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_dat Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(68) ColumnarToRow [codegen id : 1] +(68) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (69) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt index ccd1b66fa2..fe26a3f5cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count,count(1)] @@ -32,7 +32,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] CometFilter [d_date_sk,d_date,d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/explain.txt index b0ca5520f2..a7c93664a7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (32) +* CometColumnarToRow (32) +- CometSort (31) +- CometColumnarExchange (30) +- CometProject (29) @@ -179,14 +179,14 @@ Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, Co Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] Arguments: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [cnt#17 DESC NULLS LAST] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (37) -+- * ColumnarToRow (36) ++- * CometColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometScan parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt index 4417365325..eaede7232d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometColumnarExchange [cnt] #1 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/explain.txt index 70e62b6791..5df317600b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/explain.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject (71) : : : +- * HashAggregate (16) : : : +- Exchange (15) : : : +- * HashAggregate (14) - : : : +- * ColumnarToRow (13) + : : : +- * CometColumnarToRow (13) : : : +- CometProject (12) : : : +- CometBroadcastHashJoin (11) : : : :- CometProject (7) @@ -26,7 +26,7 @@ TakeOrderedAndProject (71) : : +- * HashAggregate (33) : : +- Exchange (32) : : +- * HashAggregate (31) - : : +- * ColumnarToRow (30) + : : +- * CometColumnarToRow (30) : : +- CometProject (29) : : +- CometBroadcastHashJoin (28) : : :- CometProject (24) @@ -44,7 +44,7 @@ TakeOrderedAndProject (71) : +- * HashAggregate (49) : +- Exchange (48) : +- * HashAggregate (47) - : +- * ColumnarToRow (46) + : +- * CometColumnarToRow (46) : +- CometProject (45) : +- CometBroadcastHashJoin (44) : :- CometProject (42) @@ -59,7 +59,7 @@ TakeOrderedAndProject (71) +- * HashAggregate (67) +- Exchange (66) +- * HashAggregate (65) - +- * ColumnarToRow (64) + +- * CometColumnarToRow (64) +- CometProject (63) +- CometBroadcastHashJoin (62) :- CometProject (60) @@ -132,7 +132,7 @@ Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7, d_date_sk#9, d_year#10] Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10], [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] -(13) ColumnarToRow [codegen id : 1] +(13) CometColumnarToRow [codegen id : 1] Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] (14) HashAggregate [codegen id : 1] @@ -217,7 +217,7 @@ Arguments: [ss_sold_date_sk#22], [d_date_sk#24], Inner, BuildRight Input [7]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22, d_date_sk#24, d_year#25] Arguments: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25], [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] -(30) ColumnarToRow [codegen id : 2] +(30) CometColumnarToRow [codegen id : 2] Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] (31) HashAggregate [codegen id : 2] @@ -296,7 +296,7 @@ Arguments: [ws_sold_date_sk#38], [d_date_sk#40], Inner, BuildRight Input [7]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38, d_date_sk#40, d_year#41] Arguments: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41], [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] -(46) ColumnarToRow [codegen id : 4] +(46) CometColumnarToRow [codegen id : 4] Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] (47) HashAggregate [codegen id : 4] @@ -383,7 +383,7 @@ Arguments: [ws_sold_date_sk#53], [d_date_sk#55], Inner, BuildRight Input [7]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53, d_date_sk#55, d_year#56] Arguments: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56], [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] -(64) ColumnarToRow [codegen id : 6] +(64) CometColumnarToRow [codegen id : 6] Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] (65) HashAggregate [codegen id : 6] @@ -426,7 +426,7 @@ Arguments: 100, [customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (75) -+- * ColumnarToRow (74) ++- * CometColumnarToRow (74) +- CometFilter (73) +- CometScan parquet spark_catalog.default.date_dim (72) @@ -442,7 +442,7 @@ ReadSchema: struct Input [2]: [d_date_sk#9, d_year#10] Condition : (((isnotnull(d_year#10) AND (d_year#10 = 2001)) AND d_year#10 IN (2001,2002)) AND isnotnull(d_date_sk#9)) -(74) ColumnarToRow [codegen id : 1] +(74) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] (75) BroadcastExchange @@ -451,7 +451,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 BroadcastExchange (79) -+- * ColumnarToRow (78) ++- * CometColumnarToRow (78) +- CometFilter (77) +- CometScan parquet spark_catalog.default.date_dim (76) @@ -467,7 +467,7 @@ ReadSchema: struct Input [2]: [d_date_sk#24, d_year#25] Condition : (((isnotnull(d_year#25) AND (d_year#25 = 2002)) AND d_year#25 IN (2001,2002)) AND isnotnull(d_date_sk#24)) -(78) ColumnarToRow [codegen id : 1] +(78) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#24, d_year#25] (79) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt index a7aa864ce9..0aa440adb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt @@ -11,7 +11,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] Exchange [c_customer_id,c_first_name,c_last_name,d_year] #1 WholeStageCodegen (1) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] @@ -25,7 +25,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -40,7 +40,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] Exchange [c_customer_id,c_first_name,c_last_name,d_year] #6 WholeStageCodegen (2) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] @@ -54,7 +54,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -70,7 +70,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] Exchange [c_customer_id,c_first_name,c_last_name,d_year] #11 WholeStageCodegen (4) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] @@ -91,7 +91,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] Exchange [c_customer_id,c_first_name,c_last_name,d_year] #14 WholeStageCodegen (6) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/explain.txt index 66c55173a5..083fd37b03 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/explain.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject (125) : +- * HashAggregate (64) : +- Exchange (63) : +- * HashAggregate (62) - : +- * ColumnarToRow (61) + : +- * CometColumnarToRow (61) : +- CometHashAggregate (60) : +- CometExchange (59) : +- CometHashAggregate (58) @@ -75,7 +75,7 @@ TakeOrderedAndProject (125) +- * HashAggregate (119) +- Exchange (118) +- * HashAggregate (117) - +- * ColumnarToRow (116) + +- * CometColumnarToRow (116) +- CometHashAggregate (115) +- CometExchange (114) +- CometHashAggregate (113) @@ -401,7 +401,7 @@ Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_ Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] -(61) ColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [codegen id : 1] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] (62) HashAggregate [codegen id : 1] @@ -645,7 +645,7 @@ Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufac Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] Functions: [] -(116) ColumnarToRow [codegen id : 4] +(116) CometColumnarToRow [codegen id : 4] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] (117) HashAggregate [codegen id : 4] @@ -696,7 +696,7 @@ Arguments: 100, [sales_cnt_diff#137 ASC NULLS FIRST], [prev_year#133, year#134, Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (129) -+- * ColumnarToRow (128) ++- * CometColumnarToRow (128) +- CometFilter (127) +- CometScan parquet spark_catalog.default.date_dim (126) @@ -712,7 +712,7 @@ ReadSchema: struct Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(128) ColumnarToRow [codegen id : 1] +(128) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] (129) BroadcastExchange @@ -725,7 +725,7 @@ Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN d Subquery:4 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 BroadcastExchange (133) -+- * ColumnarToRow (132) ++- * CometColumnarToRow (132) +- CometFilter (131) +- CometScan parquet spark_catalog.default.date_dim (130) @@ -741,7 +741,7 @@ ReadSchema: struct Input [2]: [d_date_sk#81, d_year#82] Condition : ((isnotnull(d_year#82) AND (d_year#82 = 2001)) AND isnotnull(d_date_sk#81)) -(132) ColumnarToRow [codegen id : 1] +(132) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#81, d_year#82] (133) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt index e52b2f7d8a..cec78f1ccc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 WholeStageCodegen (1) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 @@ -33,7 +33,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -97,7 +97,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 WholeStageCodegen (4) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 @@ -116,7 +116,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/explain.txt index 8bfcb729b9..7d4d572b1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (34) +- * HashAggregate (33) +- Exchange (32) +- * HashAggregate (31) - +- * ColumnarToRow (30) + +- * CometColumnarToRow (30) +- CometUnion (29) :- CometProject (12) : +- CometBroadcastHashJoin (11) @@ -172,7 +172,7 @@ Child 0 Input [6]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, ex Child 1 Input [6]: [channel#22, col_name#23, d_year#20, d_qoy#21, i_category#18, ext_sales_price#24] Child 2 Input [6]: [channel#34, col_name#35, d_year#32, d_qoy#33, i_category#30, ext_sales_price#36] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [6]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, ext_sales_price#12] (31) HashAggregate [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt index cfcf6a99ab..c80e222bbc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_ Exchange [channel,col_name,d_year,d_qoy,i_category] #1 WholeStageCodegen (1) HashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] [count,sum,count,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometUnion [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] CometProject [ss_store_sk,ss_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/explain.txt index 210be21218..c975002842 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/explain.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject (86) : :- * HashAggregate (17) : : +- Exchange (16) : : +- * HashAggregate (15) - : : +- * ColumnarToRow (14) + : : +- * CometColumnarToRow (14) : : +- CometProject (13) : : +- CometBroadcastHashJoin (12) : : :- CometProject (8) @@ -28,7 +28,7 @@ TakeOrderedAndProject (86) : +- * HashAggregate (29) : +- Exchange (28) : +- * HashAggregate (27) - : +- * ColumnarToRow (26) + : +- * CometColumnarToRow (26) : +- CometProject (25) : +- CometBroadcastHashJoin (24) : :- CometProject (22) @@ -43,7 +43,7 @@ TakeOrderedAndProject (86) : : +- * HashAggregate (40) : : +- Exchange (39) : : +- * HashAggregate (38) - : : +- * ColumnarToRow (37) + : : +- * CometColumnarToRow (37) : : +- CometProject (36) : : +- CometBroadcastHashJoin (35) : : :- CometScan parquet spark_catalog.default.catalog_sales (33) @@ -51,7 +51,7 @@ TakeOrderedAndProject (86) : +- * HashAggregate (49) : +- Exchange (48) : +- * HashAggregate (47) - : +- * ColumnarToRow (46) + : +- * CometColumnarToRow (46) : +- CometProject (45) : +- CometBroadcastHashJoin (44) : :- CometScan parquet spark_catalog.default.catalog_returns (42) @@ -61,7 +61,7 @@ TakeOrderedAndProject (86) :- * HashAggregate (65) : +- Exchange (64) : +- * HashAggregate (63) - : +- * ColumnarToRow (62) + : +- * CometColumnarToRow (62) : +- CometProject (61) : +- CometBroadcastHashJoin (60) : :- CometProject (56) @@ -76,7 +76,7 @@ TakeOrderedAndProject (86) +- * HashAggregate (77) +- Exchange (76) +- * HashAggregate (75) - +- * ColumnarToRow (74) + +- * CometColumnarToRow (74) +- CometProject (73) +- CometBroadcastHashJoin (72) :- CometProject (70) @@ -151,7 +151,7 @@ Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] Arguments: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8], [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] (15) HashAggregate [codegen id : 1] @@ -208,7 +208,7 @@ Arguments: [sr_store_sk#17], [s_store_sk#23], Inner, BuildRight Input [4]: [sr_store_sk#17, sr_return_amt#18, sr_net_loss#19, s_store_sk#23] Arguments: [sr_return_amt#18, sr_net_loss#19, s_store_sk#23], [sr_return_amt#18, sr_net_loss#19, s_store_sk#23] -(26) ColumnarToRow [codegen id : 2] +(26) CometColumnarToRow [codegen id : 2] Input [3]: [sr_return_amt#18, sr_net_loss#19, s_store_sk#23] (27) HashAggregate [codegen id : 2] @@ -262,7 +262,7 @@ Arguments: [cs_sold_date_sk#39], [d_date_sk#41], Inner, BuildRight Input [5]: [cs_call_center_sk#36, cs_ext_sales_price#37, cs_net_profit#38, cs_sold_date_sk#39, d_date_sk#41] Arguments: [cs_call_center_sk#36, cs_ext_sales_price#37, cs_net_profit#38], [cs_call_center_sk#36, cs_ext_sales_price#37, cs_net_profit#38] -(37) ColumnarToRow [codegen id : 5] +(37) CometColumnarToRow [codegen id : 5] Input [3]: [cs_call_center_sk#36, cs_ext_sales_price#37, cs_net_profit#38] (38) HashAggregate [codegen id : 5] @@ -306,7 +306,7 @@ Arguments: [cr_returned_date_sk#52], [d_date_sk#54], Inner, BuildRight Input [4]: [cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52, d_date_sk#54] Arguments: [cr_return_amount#50, cr_net_loss#51], [cr_return_amount#50, cr_net_loss#51] -(46) ColumnarToRow [codegen id : 7] +(46) CometColumnarToRow [codegen id : 7] Input [2]: [cr_return_amount#50, cr_net_loss#51] (47) HashAggregate [codegen id : 7] @@ -383,7 +383,7 @@ Arguments: [ws_web_page_sk#66], [wp_web_page_sk#72], Inner, BuildRight Input [4]: [ws_web_page_sk#66, ws_ext_sales_price#67, ws_net_profit#68, wp_web_page_sk#72] Arguments: [ws_ext_sales_price#67, ws_net_profit#68, wp_web_page_sk#72], [ws_ext_sales_price#67, ws_net_profit#68, wp_web_page_sk#72] -(62) ColumnarToRow [codegen id : 9] +(62) CometColumnarToRow [codegen id : 9] Input [3]: [ws_ext_sales_price#67, ws_net_profit#68, wp_web_page_sk#72] (63) HashAggregate [codegen id : 9] @@ -440,7 +440,7 @@ Arguments: [wr_web_page_sk#81], [wp_web_page_sk#87], Inner, BuildRight Input [4]: [wr_web_page_sk#81, wr_return_amt#82, wr_net_loss#83, wp_web_page_sk#87] Arguments: [wr_return_amt#82, wr_net_loss#83, wp_web_page_sk#87], [wr_return_amt#82, wr_net_loss#83, wp_web_page_sk#87] -(74) ColumnarToRow [codegen id : 10] +(74) CometColumnarToRow [codegen id : 10] Input [3]: [wr_return_amt#82, wr_net_loss#83, wp_web_page_sk#87] (75) HashAggregate [codegen id : 10] @@ -507,7 +507,7 @@ Arguments: 100, [channel#100 ASC NULLS FIRST, id#101 ASC NULLS FIRST], [channel# Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (91) -+- * ColumnarToRow (90) ++- * CometColumnarToRow (90) +- CometProject (89) +- CometFilter (88) +- CometScan parquet spark_catalog.default.date_dim (87) @@ -528,7 +528,7 @@ Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 < Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(90) ColumnarToRow [codegen id : 1] +(90) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (91) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt index 91786374e3..db1a4d4b56 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Exchange [s_store_sk] #2 WholeStageCodegen (1) HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] @@ -27,7 +27,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] @@ -47,7 +47,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Exchange [s_store_sk] #7 WholeStageCodegen (2) HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [sr_return_amt,sr_net_loss,s_store_sk] CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] @@ -69,7 +69,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Exchange [cs_call_center_sk] #9 WholeStageCodegen (5) HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] @@ -81,7 +81,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Exchange #10 WholeStageCodegen (7) HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cr_return_amount,cr_net_loss] CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] @@ -96,7 +96,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Exchange [wp_web_page_sk] #11 WholeStageCodegen (9) HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] @@ -117,7 +117,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Exchange [wp_web_page_sk] #14 WholeStageCodegen (10) HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/explain.txt index b798add795..f9587646a7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/explain.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject (69) : : +- * HashAggregate (21) : : +- Exchange (20) : : +- * HashAggregate (19) - : : +- * ColumnarToRow (18) + : : +- * CometColumnarToRow (18) : : +- CometProject (17) : : +- CometBroadcastHashJoin (16) : : :- CometProject (12) @@ -31,7 +31,7 @@ TakeOrderedAndProject (69) : +- * HashAggregate (41) : +- Exchange (40) : +- * HashAggregate (39) - : +- * ColumnarToRow (38) + : +- * CometColumnarToRow (38) : +- CometProject (37) : +- CometBroadcastHashJoin (36) : :- CometProject (34) @@ -52,7 +52,7 @@ TakeOrderedAndProject (69) +- * HashAggregate (64) +- Exchange (63) +- * HashAggregate (62) - +- * ColumnarToRow (61) + +- * CometColumnarToRow (61) +- CometProject (60) +- CometBroadcastHashJoin (59) :- CometProject (57) @@ -150,7 +150,7 @@ Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -(18) ColumnarToRow [codegen id : 1] +(18) CometColumnarToRow [codegen id : 1] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] (19) HashAggregate [codegen id : 1] @@ -243,7 +243,7 @@ Arguments: [ws_sold_date_sk#33], [d_date_sk#38], Inner, BuildRight Input [8]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, d_date_sk#38, d_year#39] Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39], [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] -(38) ColumnarToRow [codegen id : 3] +(38) CometColumnarToRow [codegen id : 3] Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] (39) HashAggregate [codegen id : 3] @@ -350,7 +350,7 @@ Arguments: [cs_sold_date_sk#60], [d_date_sk#65], Inner, BuildRight Input [8]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, d_date_sk#65, d_year#66] Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66], [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] -(61) ColumnarToRow [codegen id : 6] +(61) CometColumnarToRow [codegen id : 6] Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] (62) HashAggregate [codegen id : 6] @@ -397,7 +397,7 @@ Arguments: 100, [ratio#81 ASC NULLS FIRST, ss_qty#24 DESC NULLS LAST, ss_wc#25 D Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (73) -+- * ColumnarToRow (72) ++- * CometColumnarToRow (72) +- CometFilter (71) +- CometScan parquet spark_catalog.default.date_dim (70) @@ -413,7 +413,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(72) ColumnarToRow [codegen id : 1] +(72) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] (73) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt index 48eeb00d60..16a4ab0145 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes Exchange [d_year,ss_item_sk,ss_customer_sk] #1 WholeStageCodegen (1) HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] @@ -28,7 +28,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -49,7 +49,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes Exchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 WholeStageCodegen (3) HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] @@ -76,7 +76,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes Exchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 WholeStageCodegen (6) HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/explain.txt index 9fa6bbb19c..a3587c0e89 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/explain.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject (31) :- * HashAggregate (24) : +- Exchange (23) : +- * HashAggregate (22) - : +- * ColumnarToRow (21) + : +- * CometColumnarToRow (21) : +- CometProject (20) : +- CometBroadcastHashJoin (19) : :- CometProject (14) @@ -27,7 +27,7 @@ TakeOrderedAndProject (31) : +- CometFilter (16) : +- CometScan parquet spark_catalog.default.household_demographics (15) +- BroadcastExchange (28) - +- * ColumnarToRow (27) + +- * CometColumnarToRow (27) +- CometFilter (26) +- CometScan parquet spark_catalog.default.customer (25) @@ -128,7 +128,7 @@ Arguments: [ss_hdemo_sk#2], [hd_demo_sk#16], Inner, BuildRight Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15, hd_demo_sk#16] Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] -(21) ColumnarToRow [codegen id : 1] +(21) CometColumnarToRow [codegen id : 1] Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] (22) HashAggregate [codegen id : 1] @@ -160,7 +160,7 @@ ReadSchema: struct Input [3]: [c_customer_sk#27, c_first_name#28, c_last_name#29] Condition : isnotnull(c_customer_sk#27) -(27) ColumnarToRow [codegen id : 2] +(27) CometColumnarToRow [codegen id : 2] Input [3]: [c_customer_sk#27, c_first_name#28, c_last_name#29] (28) BroadcastExchange @@ -185,7 +185,7 @@ Arguments: 100, [c_last_name#29 ASC NULLS FIRST, c_first_name#28 ASC NULLS FIRST Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (36) -+- * ColumnarToRow (35) ++- * CometColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan parquet spark_catalog.default.date_dim (32) @@ -206,7 +206,7 @@ Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,20 Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(35) ColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt index f2883e4503..de120a1a1f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt @@ -7,7 +7,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 WholeStageCodegen (1) HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city,hd_demo_sk] @@ -20,7 +20,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dow] @@ -40,7 +40,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, InputAdapter BroadcastExchange #6 WholeStageCodegen (2) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/explain.txt index 99e0e2f16a..e4cc87a95b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (42) +- * HashAggregate (41) +- Exchange (40) +- * HashAggregate (39) - +- * ColumnarToRow (38) + +- * CometColumnarToRow (38) +- CometProject (37) +- CometBroadcastHashJoin (36) :- CometProject (13) @@ -218,7 +218,7 @@ Arguments: [substr(s_zip#10, 1, 2)], [substr(ca_zip#12, 1, 2)], Inner, BuildRigh Input [4]: [ss_net_profit#2, s_store_name#9, s_zip#10, ca_zip#12] Arguments: [ss_net_profit#2, s_store_name#9], [ss_net_profit#2, s_store_name#9] -(38) ColumnarToRow [codegen id : 1] +(38) CometColumnarToRow [codegen id : 1] Input [2]: [ss_net_profit#2, s_store_name#9] (39) HashAggregate [codegen id : 1] @@ -247,7 +247,7 @@ Arguments: 100, [s_store_name#9 ASC NULLS FIRST], [s_store_name#9, sum(ss_net_pr Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (47) -+- * ColumnarToRow (46) ++- * CometColumnarToRow (46) +- CometProject (45) +- CometFilter (44) +- CometScan parquet spark_catalog.default.date_dim (43) @@ -268,7 +268,7 @@ Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) A Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Arguments: [d_date_sk#5], [d_date_sk#5] -(46) ColumnarToRow [codegen id : 1] +(46) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (47) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt index 150dc4c172..3c6da3f15d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] Exchange [s_store_name] #1 WholeStageCodegen (1) HashAggregate [s_store_name,ss_net_profit] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_net_profit,s_store_name] CometBroadcastHashJoin [ss_net_profit,s_store_name,s_zip,ca_zip] @@ -18,7 +18,7 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/explain.txt index 63109f7d7a..00e763e1a5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/explain.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject (102) :- * HashAggregate (38) : +- Exchange (37) : +- * HashAggregate (36) - : +- * ColumnarToRow (35) + : +- * CometColumnarToRow (35) : +- CometProject (34) : +- CometBroadcastHashJoin (33) : :- CometProject (28) @@ -46,7 +46,7 @@ TakeOrderedAndProject (102) :- * HashAggregate (67) : +- Exchange (66) : +- * HashAggregate (65) - : +- * ColumnarToRow (64) + : +- * CometColumnarToRow (64) : +- CometProject (63) : +- CometBroadcastHashJoin (62) : :- CometProject (60) @@ -75,7 +75,7 @@ TakeOrderedAndProject (102) +- * HashAggregate (96) +- Exchange (95) +- * HashAggregate (94) - +- * ColumnarToRow (93) + +- * CometColumnarToRow (93) +- CometProject (92) +- CometBroadcastHashJoin (91) :- CometProject (89) @@ -263,7 +263,7 @@ Arguments: [ss_promo_sk#3], [p_promo_sk#20], Inner, BuildRight Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17, p_promo_sk#20] Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] -(35) ColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [codegen id : 1] Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] (36) HashAggregate [codegen id : 1] @@ -396,7 +396,7 @@ Arguments: [cs_promo_sk#42], [p_promo_sk#57], Inner, BuildRight Input [7]: [cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55, p_promo_sk#57] Arguments: [cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55], [cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55] -(64) ColumnarToRow [codegen id : 3] +(64) CometColumnarToRow [codegen id : 3] Input [5]: [cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55] (65) HashAggregate [codegen id : 3] @@ -529,7 +529,7 @@ Arguments: [ws_promo_sk#78], [p_promo_sk#93], Inner, BuildRight Input [7]: [ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91, p_promo_sk#93] Arguments: [ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91], [ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91] -(93) ColumnarToRow [codegen id : 5] +(93) CometColumnarToRow [codegen id : 5] Input [5]: [ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91] (94) HashAggregate [codegen id : 5] @@ -582,7 +582,7 @@ Arguments: 100, [channel#112 ASC NULLS FIRST, id#113 ASC NULLS FIRST], [channel# Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (107) -+- * ColumnarToRow (106) ++- * CometColumnarToRow (106) +- CometProject (105) +- CometFilter (104) +- CometScan parquet spark_catalog.default.date_dim (103) @@ -603,7 +603,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(106) ColumnarToRow [codegen id : 1] +(106) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] (107) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt index 717c94e212..202a901ae6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Exchange [s_store_id] #2 WholeStageCodegen (1) HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] @@ -33,7 +33,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] @@ -64,7 +64,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Exchange [cp_catalog_page_id] #10 WholeStageCodegen (3) HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] @@ -98,7 +98,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Exchange [web_site_id] #14 WholeStageCodegen (5) HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/explain.txt index 6b252f661c..5449da2446 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/explain.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject (50) : : : +- * HashAggregate (17) : : : +- Exchange (16) : : : +- * HashAggregate (15) - : : : +- * ColumnarToRow (14) + : : : +- * CometColumnarToRow (14) : : : +- CometProject (13) : : : +- CometBroadcastHashJoin (12) : : : :- CometProject (8) @@ -32,7 +32,7 @@ TakeOrderedAndProject (50) : : +- * HashAggregate (30) : : +- Exchange (29) : : +- * HashAggregate (28) - : : +- * ColumnarToRow (27) + : : +- * CometColumnarToRow (27) : : +- CometProject (26) : : +- CometBroadcastHashJoin (25) : : :- CometProject (23) @@ -42,11 +42,11 @@ TakeOrderedAndProject (50) : : : +- ReusedExchange (21) : : +- ReusedExchange (24) : +- BroadcastExchange (41) - : +- * ColumnarToRow (40) + : +- * CometColumnarToRow (40) : +- CometFilter (39) : +- CometScan parquet spark_catalog.default.customer (38) +- BroadcastExchange (47) - +- * ColumnarToRow (46) + +- * CometColumnarToRow (46) +- CometFilter (45) +- CometScan parquet spark_catalog.default.customer_address (44) @@ -115,7 +115,7 @@ Arguments: [cr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#8, ca_state#9] Arguments: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9], [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] (15) HashAggregate [codegen id : 1] @@ -176,7 +176,7 @@ Arguments: [cr_returning_addr_sk#17], [ca_address_sk#22], Inner, BuildRight Input [5]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, ca_address_sk#22, ca_state#23] Arguments: [cr_returning_customer_sk#16, cr_return_amt_inc_tax#18, ca_state#23], [cr_returning_customer_sk#16, cr_return_amt_inc_tax#18, ca_state#23] -(27) ColumnarToRow [codegen id : 2] +(27) CometColumnarToRow [codegen id : 2] Input [3]: [cr_returning_customer_sk#16, cr_return_amt_inc_tax#18, ca_state#23] (28) HashAggregate [codegen id : 2] @@ -244,7 +244,7 @@ ReadSchema: struct= 2000-05-25)) AND (d_date#1 Input [2]: [d_date_sk#10, d_date#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt index 42c3f40e68..755128d2c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] CometHashAggregate [i_item_id,i_item_desc,i_current_price] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/explain.txt index b7136e1636..52cc3cb6aa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/explain.txt @@ -7,7 +7,7 @@ TakeOrderedAndProject (50) : :- * HashAggregate (19) : : +- Exchange (18) : : +- * HashAggregate (17) - : : +- * ColumnarToRow (16) + : : +- * CometColumnarToRow (16) : : +- CometProject (15) : : +- CometBroadcastHashJoin (14) : : :- CometProject (7) @@ -27,7 +27,7 @@ TakeOrderedAndProject (50) : +- * HashAggregate (31) : +- Exchange (30) : +- * HashAggregate (29) - : +- * ColumnarToRow (28) + : +- * CometColumnarToRow (28) : +- CometProject (27) : +- CometBroadcastHashJoin (26) : :- CometProject (24) @@ -40,7 +40,7 @@ TakeOrderedAndProject (50) +- * HashAggregate (46) +- Exchange (45) +- * HashAggregate (44) - +- * ColumnarToRow (43) + +- * CometColumnarToRow (43) +- CometProject (42) +- CometBroadcastHashJoin (41) :- CometProject (39) @@ -123,7 +123,7 @@ Arguments: [sr_returned_date_sk#3], [d_date_sk#7], Inner, BuildRight Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#6, d_date_sk#7] Arguments: [sr_return_quantity#2, i_item_id#6], [sr_return_quantity#2, i_item_id#6] -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarToRow [codegen id : 1] Input [2]: [sr_return_quantity#2, i_item_id#6] (17) HashAggregate [codegen id : 1] @@ -180,7 +180,7 @@ Arguments: [cr_returned_date_sk#17], [d_date_sk#21], Inner, BuildRight Input [4]: [cr_return_quantity#16, cr_returned_date_sk#17, i_item_id#20, d_date_sk#21] Arguments: [cr_return_quantity#16, i_item_id#20], [cr_return_quantity#16, i_item_id#20] -(28) ColumnarToRow [codegen id : 2] +(28) CometColumnarToRow [codegen id : 2] Input [2]: [cr_return_quantity#16, i_item_id#20] (29) HashAggregate [codegen id : 2] @@ -251,7 +251,7 @@ Arguments: [wr_returned_date_sk#29], [d_date_sk#33], Inner, BuildRight Input [4]: [wr_return_quantity#28, wr_returned_date_sk#29, i_item_id#32, d_date_sk#33] Arguments: [wr_return_quantity#28, i_item_id#32], [wr_return_quantity#28, i_item_id#32] -(43) ColumnarToRow [codegen id : 4] +(43) CometColumnarToRow [codegen id : 4] Input [2]: [wr_return_quantity#28, i_item_id#32] (44) HashAggregate [codegen id : 4] @@ -294,7 +294,7 @@ Arguments: 100, [item_id#13 ASC NULLS FIRST, sr_item_qty#14 ASC NULLS FIRST], [i Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 BroadcastExchange (64) -+- * ColumnarToRow (63) ++- * CometColumnarToRow (63) +- CometProject (62) +- CometBroadcastHashJoin (61) :- CometFilter (52) @@ -367,7 +367,7 @@ Arguments: [d_date#8], [d_date#9], LeftSemi, BuildRight Input [2]: [d_date_sk#7, d_date#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(63) ColumnarToRow [codegen id : 1] +(63) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (64) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt index 8cd3f3f45e..1696b482b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty Exchange [i_item_id] #1 WholeStageCodegen (1) HashAggregate [i_item_id,sr_return_quantity] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [sr_return_quantity,i_item_id] CometBroadcastHashJoin [sr_return_quantity,sr_returned_date_sk,i_item_id,d_date_sk] @@ -20,7 +20,7 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometBroadcastHashJoin [d_date_sk,d_date,d_date] @@ -51,7 +51,7 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty Exchange [i_item_id] #8 WholeStageCodegen (2) HashAggregate [i_item_id,cr_return_quantity] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cr_return_quantity,i_item_id] CometBroadcastHashJoin [cr_return_quantity,cr_returned_date_sk,i_item_id,d_date_sk] @@ -70,7 +70,7 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty Exchange [i_item_id] #10 WholeStageCodegen (4) HashAggregate [i_item_id,wr_return_quantity] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [wr_return_quantity,i_item_id] CometBroadcastHashJoin [wr_return_quantity,wr_returned_date_sk,i_item_id,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/explain.txt index 572fd7a66f..ca784ef3e7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (32) +- * Project (31) - +- * ColumnarToRow (30) + +- * CometColumnarToRow (30) +- CometBroadcastHashJoin (29) :- CometBroadcastExchange (25) : +- CometProject (24) @@ -172,7 +172,7 @@ Left output [4]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9] Right output [1]: [sr_cdemo_sk#15] Arguments: [cd_demo_sk#9], [sr_cdemo_sk#15], Inner, BuildLeft -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [5]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9, sr_cdemo_sk#15] (31) Project [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt index 646285a082..3101b29dd5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] WholeStageCodegen (1) Project [c_customer_id,c_last_name,c_first_name] - ColumnarToRow + CometColumnarToRow InputAdapter CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/explain.txt index 535fb70a91..44b59ee78e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (45) +- * HashAggregate (44) +- Exchange (43) +- * HashAggregate (42) - +- * ColumnarToRow (41) + +- * CometColumnarToRow (41) +- CometProject (40) +- CometBroadcastHashJoin (39) :- CometProject (35) @@ -238,7 +238,7 @@ Arguments: [wr_reason_sk#13], [r_reason_sk#30], Inner, BuildRight Input [6]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, r_reason_sk#30, r_reason_desc#31] Arguments: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#31], [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#31] -(41) ColumnarToRow [codegen id : 1] +(41) CometColumnarToRow [codegen id : 1] Input [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#31] (42) HashAggregate [codegen id : 1] @@ -267,7 +267,7 @@ Arguments: 100, [substr(r_reason_desc, 1, 20)#47 ASC NULLS FIRST, avg(ws_quantit Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (50) -+- * ColumnarToRow (49) ++- * CometColumnarToRow (49) +- CometProject (48) +- CometFilter (47) +- CometScan parquet spark_catalog.default.date_dim (46) @@ -288,7 +288,7 @@ Condition : ((isnotnull(d_year#29) AND (d_year#29 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#28, d_year#29] Arguments: [d_date_sk#28], [d_date_sk#28] -(49) ColumnarToRow [codegen id : 1] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#28] (50) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt index 0d350bd591..284c0fba1c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu Exchange [r_reason_desc] #1 WholeStageCodegen (1) HashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] CometBroadcastHashJoin [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash,r_reason_sk,r_reason_desc] @@ -27,7 +27,7 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt index 018954bf35..978257bdfc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt @@ -7,7 +7,7 @@ TakeOrderedAndProject (23) +- * HashAggregate (18) +- Exchange (17) +- * HashAggregate (16) - +- * ColumnarToRow (15) + +- * CometColumnarToRow (15) +- CometExpand (14) +- CometProject (13) +- CometBroadcastHashJoin (12) @@ -92,7 +92,7 @@ Arguments: [ws_net_paid#2, i_category#9, i_class#8], [ws_net_paid#2, i_category# Input [3]: [ws_net_paid#2, i_category#9, i_class#8] Arguments: [[ws_net_paid#2, i_category#9, i_class#8, 0], [ws_net_paid#2, i_category#9, null, 1], [ws_net_paid#2, null, null, 3]], [ws_net_paid#2, i_category#10, i_class#11, spark_grouping_id#12] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarToRow [codegen id : 1] Input [4]: [ws_net_paid#2, i_category#10, i_class#11, spark_grouping_id#12] (16) HashAggregate [codegen id : 1] @@ -137,7 +137,7 @@ Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * ColumnarToRow (27) ++- * CometColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan parquet spark_catalog.default.date_dim (24) @@ -158,7 +158,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(27) ColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt index e4d028f2fb..edc45ac03b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl Exchange [i_category,i_class,spark_grouping_id] #2 WholeStageCodegen (1) HashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] CometProject [ws_net_paid,i_category,i_class] @@ -25,7 +25,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/explain.txt index 28b942e2f7..5081f68284 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/explain.txt @@ -5,7 +5,7 @@ +- * Project (46) +- * BroadcastHashJoin LeftAnti BuildRight (45) :- * BroadcastHashJoin LeftAnti BuildRight (31) - : :- * ColumnarToRow (17) + : :- * CometColumnarToRow (17) : : +- CometHashAggregate (16) : : +- CometExchange (15) : : +- CometHashAggregate (14) @@ -23,7 +23,7 @@ : : +- CometFilter (10) : : +- CometScan parquet spark_catalog.default.customer (9) : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) + : +- * CometColumnarToRow (29) : +- CometHashAggregate (28) : +- CometExchange (27) : +- CometHashAggregate (26) @@ -36,7 +36,7 @@ : : +- ReusedExchange (20) : +- ReusedExchange (23) +- BroadcastExchange (44) - +- * ColumnarToRow (43) + +- * CometColumnarToRow (43) +- CometHashAggregate (42) +- CometExchange (41) +- CometHashAggregate (40) @@ -128,7 +128,7 @@ Input [3]: [c_last_name#9, c_first_name#8, d_date#5] Keys [3]: [c_last_name#9, c_first_name#8, d_date#5] Functions: [] -(17) ColumnarToRow [codegen id : 3] +(17) CometColumnarToRow [codegen id : 3] Input [3]: [c_last_name#9, c_first_name#8, d_date#5] (18) CometScan parquet spark_catalog.default.catalog_sales @@ -181,7 +181,7 @@ Input [3]: [c_last_name#17, c_first_name#16, d_date#14] Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] Functions: [] -(29) ColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [3]: [c_last_name#17, c_first_name#16, d_date#14] (30) BroadcastExchange @@ -244,7 +244,7 @@ Input [3]: [c_last_name#25, c_first_name#24, d_date#22] Keys [3]: [c_last_name#25, c_first_name#24, d_date#22] Functions: [] -(43) ColumnarToRow [codegen id : 2] +(43) CometColumnarToRow [codegen id : 2] Input [3]: [c_last_name#25, c_first_name#24, d_date#22] (44) BroadcastExchange @@ -283,7 +283,7 @@ Results [1]: [count(1)#28 AS count(1)#29] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 BroadcastExchange (54) -+- * ColumnarToRow (53) ++- * CometColumnarToRow (53) +- CometProject (52) +- CometFilter (51) +- CometScan parquet spark_catalog.default.date_dim (50) @@ -304,7 +304,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(53) ColumnarToRow [codegen id : 1] +(53) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] (54) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt index 6e3328ae1d..81cd6f40be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt @@ -7,7 +7,7 @@ WholeStageCodegen (4) Project BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometExchange [c_last_name,c_first_name,d_date] #2 @@ -21,7 +21,7 @@ WholeStageCodegen (4) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] @@ -36,7 +36,7 @@ WholeStageCodegen (4) InputAdapter BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometExchange [c_last_name,c_first_name,d_date] #7 @@ -53,7 +53,7 @@ WholeStageCodegen (4) InputAdapter BroadcastExchange #8 WholeStageCodegen (2) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometExchange [c_last_name,c_first_name,d_date] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/explain.txt index 592e23cd27..6682e151f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/explain.txt @@ -6,7 +6,7 @@ : : : :- * BroadcastNestedLoopJoin Inner BuildRight (88) : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (67) : : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (46) -: : : : : : :- * ColumnarToRow (25) +: : : : : : :- * CometColumnarToRow (25) : : : : : : : +- CometHashAggregate (24) : : : : : : : +- CometExchange (23) : : : : : : : +- CometHashAggregate (22) @@ -32,7 +32,7 @@ : : : : : : : +- CometFilter (17) : : : : : : : +- CometScan parquet spark_catalog.default.store (16) : : : : : : +- BroadcastExchange (45) -: : : : : : +- * ColumnarToRow (44) +: : : : : : +- * CometColumnarToRow (44) : : : : : : +- CometHashAggregate (43) : : : : : : +- CometExchange (42) : : : : : : +- CometHashAggregate (41) @@ -52,7 +52,7 @@ : : : : : : : +- CometScan parquet spark_catalog.default.time_dim (32) : : : : : : +- ReusedExchange (38) : : : : : +- BroadcastExchange (66) -: : : : : +- * ColumnarToRow (65) +: : : : : +- * CometColumnarToRow (65) : : : : : +- CometHashAggregate (64) : : : : : +- CometExchange (63) : : : : : +- CometHashAggregate (62) @@ -72,7 +72,7 @@ : : : : : : +- CometScan parquet spark_catalog.default.time_dim (53) : : : : : +- ReusedExchange (59) : : : : +- BroadcastExchange (87) -: : : : +- * ColumnarToRow (86) +: : : : +- * CometColumnarToRow (86) : : : : +- CometHashAggregate (85) : : : : +- CometExchange (84) : : : : +- CometHashAggregate (83) @@ -92,7 +92,7 @@ : : : : : +- CometScan parquet spark_catalog.default.time_dim (74) : : : : +- ReusedExchange (80) : : : +- BroadcastExchange (108) -: : : +- * ColumnarToRow (107) +: : : +- * CometColumnarToRow (107) : : : +- CometHashAggregate (106) : : : +- CometExchange (105) : : : +- CometHashAggregate (104) @@ -112,7 +112,7 @@ : : : : +- CometScan parquet spark_catalog.default.time_dim (95) : : : +- ReusedExchange (101) : : +- BroadcastExchange (129) -: : +- * ColumnarToRow (128) +: : +- * CometColumnarToRow (128) : : +- CometHashAggregate (127) : : +- CometExchange (126) : : +- CometHashAggregate (125) @@ -132,7 +132,7 @@ : : : +- CometScan parquet spark_catalog.default.time_dim (116) : : +- ReusedExchange (122) : +- BroadcastExchange (150) -: +- * ColumnarToRow (149) +: +- * CometColumnarToRow (149) : +- CometHashAggregate (148) : +- CometExchange (147) : +- CometHashAggregate (146) @@ -152,7 +152,7 @@ : : +- CometScan parquet spark_catalog.default.time_dim (137) : +- ReusedExchange (143) +- BroadcastExchange (171) - +- * ColumnarToRow (170) + +- * CometColumnarToRow (170) +- CometHashAggregate (169) +- CometExchange (168) +- CometHashAggregate (167) @@ -285,7 +285,7 @@ Input [1]: [count#13] Keys: [] Functions [1]: [count(1)] -(25) ColumnarToRow [codegen id : 8] +(25) CometColumnarToRow [codegen id : 8] Input [1]: [h8_30_to_9#14] (26) CometScan parquet spark_catalog.default.store_sales @@ -368,7 +368,7 @@ Input [1]: [count#24] Keys: [] Functions [1]: [count(1)] -(44) ColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [h9_to_9_30#25] (45) BroadcastExchange @@ -459,7 +459,7 @@ Input [1]: [count#35] Keys: [] Functions [1]: [count(1)] -(65) ColumnarToRow [codegen id : 2] +(65) CometColumnarToRow [codegen id : 2] Input [1]: [h9_30_to_10#36] (66) BroadcastExchange @@ -550,7 +550,7 @@ Input [1]: [count#46] Keys: [] Functions [1]: [count(1)] -(86) ColumnarToRow [codegen id : 3] +(86) CometColumnarToRow [codegen id : 3] Input [1]: [h10_to_10_30#47] (87) BroadcastExchange @@ -641,7 +641,7 @@ Input [1]: [count#57] Keys: [] Functions [1]: [count(1)] -(107) ColumnarToRow [codegen id : 4] +(107) CometColumnarToRow [codegen id : 4] Input [1]: [h10_30_to_11#58] (108) BroadcastExchange @@ -732,7 +732,7 @@ Input [1]: [count#68] Keys: [] Functions [1]: [count(1)] -(128) ColumnarToRow [codegen id : 5] +(128) CometColumnarToRow [codegen id : 5] Input [1]: [h11_to_11_30#69] (129) BroadcastExchange @@ -823,7 +823,7 @@ Input [1]: [count#79] Keys: [] Functions [1]: [count(1)] -(149) ColumnarToRow [codegen id : 6] +(149) CometColumnarToRow [codegen id : 6] Input [1]: [h11_30_to_12#80] (150) BroadcastExchange @@ -914,7 +914,7 @@ Input [1]: [count#90] Keys: [] Functions [1]: [count(1)] -(170) ColumnarToRow [codegen id : 7] +(170) CometColumnarToRow [codegen id : 7] Input [1]: [h12_to_12_30#91] (171) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt index 1bb61b6c92..21b72d36a7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt @@ -6,7 +6,7 @@ WholeStageCodegen (8) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h8_30_to_9,count,count(1)] CometExchange #1 @@ -35,7 +35,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h9_to_9_30,count,count(1)] CometExchange #6 @@ -58,7 +58,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #8 WholeStageCodegen (2) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h9_30_to_10,count,count(1)] CometExchange #9 @@ -81,7 +81,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #11 WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h10_to_10_30,count,count(1)] CometExchange #12 @@ -104,7 +104,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #14 WholeStageCodegen (4) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h10_30_to_11,count,count(1)] CometExchange #15 @@ -127,7 +127,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #17 WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h11_to_11_30,count,count(1)] CometExchange #18 @@ -150,7 +150,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #20 WholeStageCodegen (6) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h11_30_to_12,count,count(1)] CometExchange #21 @@ -173,7 +173,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #23 WholeStageCodegen (7) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h12_to_12_30,count,count(1)] CometExchange #24 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt index 32dc4ffa71..9f18ecfb1d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject (28) +- * HashAggregate (22) +- Exchange (21) +- * HashAggregate (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometProject (18) +- CometBroadcastHashJoin (17) :- CometProject (13) @@ -117,7 +117,7 @@ Arguments: [ss_store_sk#6], [s_store_sk#13], Inner, BuildRight Input [9]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, d_moy#12, s_store_sk#13, s_store_name#14, s_company_name#15] Arguments: [i_brand#2, i_class#3, i_category#4, ss_sales_price#7, d_moy#12, s_store_name#14, s_company_name#15], [i_brand#2, i_class#3, i_category#4, ss_sales_price#7, d_moy#12, s_store_name#14, s_company_name#15] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [7]: [i_brand#2, i_class#3, i_category#4, ss_sales_price#7, d_moy#12, s_store_name#14, s_company_name#15] (20) HashAggregate [codegen id : 1] @@ -166,7 +166,7 @@ Arguments: 100, [(sum_sales#19 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (33) -+- * ColumnarToRow (32) ++- * CometColumnarToRow (32) +- CometProject (31) +- CometFilter (30) +- CometScan parquet spark_catalog.default.date_dim (29) @@ -187,7 +187,7 @@ Condition : ((isnotnull(d_year#11) AND (d_year#11 = 1999)) AND isnotnull(d_date_ Input [3]: [d_date_sk#10, d_year#11, d_moy#12] Arguments: [d_date_sk#10, d_moy#12], [d_date_sk#10, d_moy#12] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_moy#12] (33) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt index 2bfff90d34..fdda66a689 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla Exchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 WholeStageCodegen (1) HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] @@ -30,7 +30,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/explain.txt index 51715d4215..4d5872ba08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == * Project (4) -+- * ColumnarToRow (3) ++- * CometColumnarToRow (3) +- CometFilter (2) +- CometScan parquet spark_catalog.default.reason (1) @@ -16,7 +16,7 @@ ReadSchema: struct Input [1]: [r_reason_sk#1] Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) -(3) ColumnarToRow [codegen id : 1] +(3) CometColumnarToRow [codegen id : 1] Input [1]: [r_reason_sk#1] (4) Project [codegen id : 1] @@ -30,7 +30,7 @@ Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery +- * HashAggregate (11) +- Exchange (10) +- * HashAggregate (9) - +- * ColumnarToRow (8) + +- * CometColumnarToRow (8) +- CometProject (7) +- CometFilter (6) +- CometScan parquet spark_catalog.default.store_sales (5) @@ -51,7 +51,7 @@ Condition : ((isnotnull(ss_quantity#17) AND (ss_quantity#17 >= 1)) AND (ss_quant Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] Arguments: [ss_ext_discount_amt#18, ss_net_paid#19], [ss_ext_discount_amt#18, ss_net_paid#19] -(8) ColumnarToRow [codegen id : 1] +(8) CometColumnarToRow [codegen id : 1] Input [2]: [ss_ext_discount_amt#18, ss_net_paid#19] (9) HashAggregate [codegen id : 1] @@ -85,7 +85,7 @@ Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery +- * HashAggregate (19) +- Exchange (18) +- * HashAggregate (17) - +- * ColumnarToRow (16) + +- * CometColumnarToRow (16) +- CometProject (15) +- CometFilter (14) +- CometScan parquet spark_catalog.default.store_sales (13) @@ -106,7 +106,7 @@ Condition : ((isnotnull(ss_quantity#38) AND (ss_quantity#38 >= 21)) AND (ss_quan Input [4]: [ss_quantity#38, ss_ext_discount_amt#39, ss_net_paid#40, ss_sold_date_sk#41] Arguments: [ss_ext_discount_amt#39, ss_net_paid#40], [ss_ext_discount_amt#39, ss_net_paid#40] -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarToRow [codegen id : 1] Input [2]: [ss_ext_discount_amt#39, ss_net_paid#40] (17) HashAggregate [codegen id : 1] @@ -140,7 +140,7 @@ Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery +- * HashAggregate (27) +- Exchange (26) +- * HashAggregate (25) - +- * ColumnarToRow (24) + +- * CometColumnarToRow (24) +- CometProject (23) +- CometFilter (22) +- CometScan parquet spark_catalog.default.store_sales (21) @@ -161,7 +161,7 @@ Condition : ((isnotnull(ss_quantity#59) AND (ss_quantity#59 >= 41)) AND (ss_quan Input [4]: [ss_quantity#59, ss_ext_discount_amt#60, ss_net_paid#61, ss_sold_date_sk#62] Arguments: [ss_ext_discount_amt#60, ss_net_paid#61], [ss_ext_discount_amt#60, ss_net_paid#61] -(24) ColumnarToRow [codegen id : 1] +(24) CometColumnarToRow [codegen id : 1] Input [2]: [ss_ext_discount_amt#60, ss_net_paid#61] (25) HashAggregate [codegen id : 1] @@ -195,7 +195,7 @@ Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer +- * HashAggregate (35) +- Exchange (34) +- * HashAggregate (33) - +- * ColumnarToRow (32) + +- * CometColumnarToRow (32) +- CometProject (31) +- CometFilter (30) +- CometScan parquet spark_catalog.default.store_sales (29) @@ -216,7 +216,7 @@ Condition : ((isnotnull(ss_quantity#80) AND (ss_quantity#80 >= 61)) AND (ss_quan Input [4]: [ss_quantity#80, ss_ext_discount_amt#81, ss_net_paid#82, ss_sold_date_sk#83] Arguments: [ss_ext_discount_amt#81, ss_net_paid#82], [ss_ext_discount_amt#81, ss_net_paid#82] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [2]: [ss_ext_discount_amt#81, ss_net_paid#82] (33) HashAggregate [codegen id : 1] @@ -250,7 +250,7 @@ Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer +- * HashAggregate (43) +- Exchange (42) +- * HashAggregate (41) - +- * ColumnarToRow (40) + +- * CometColumnarToRow (40) +- CometProject (39) +- CometFilter (38) +- CometScan parquet spark_catalog.default.store_sales (37) @@ -271,7 +271,7 @@ Condition : ((isnotnull(ss_quantity#101) AND (ss_quantity#101 >= 81)) AND (ss_qu Input [4]: [ss_quantity#101, ss_ext_discount_amt#102, ss_net_paid#103, ss_sold_date_sk#104] Arguments: [ss_ext_discount_amt#102, ss_net_paid#103], [ss_ext_discount_amt#102, ss_net_paid#103] -(40) ColumnarToRow [codegen id : 1] +(40) CometColumnarToRow [codegen id : 1] Input [2]: [ss_ext_discount_amt#102, ss_net_paid#103] (41) HashAggregate [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt index 581739cc68..a381a46652 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt @@ -8,7 +8,7 @@ WholeStageCodegen (1) Exchange #1 WholeStageCodegen (1) HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_ext_discount_amt,ss_net_paid] CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] @@ -23,7 +23,7 @@ WholeStageCodegen (1) Exchange #2 WholeStageCodegen (1) HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_ext_discount_amt,ss_net_paid] CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] @@ -38,7 +38,7 @@ WholeStageCodegen (1) Exchange #3 WholeStageCodegen (1) HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_ext_discount_amt,ss_net_paid] CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] @@ -53,7 +53,7 @@ WholeStageCodegen (1) Exchange #4 WholeStageCodegen (1) HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_ext_discount_amt,ss_net_paid] CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] @@ -68,14 +68,14 @@ WholeStageCodegen (1) Exchange #5 WholeStageCodegen (1) HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_ext_discount_amt,ss_net_paid] CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [r_reason_sk] CometScan parquet spark_catalog.default.reason [r_reason_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/explain.txt index a6ec6f4b96..38379fb61c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * Project (47) +- * BroadcastNestedLoopJoin Inner BuildRight (46) - :- * ColumnarToRow (25) + :- * CometColumnarToRow (25) : +- CometHashAggregate (24) : +- CometExchange (23) : +- CometHashAggregate (22) @@ -27,7 +27,7 @@ : +- CometFilter (17) : +- CometScan parquet spark_catalog.default.web_page (16) +- BroadcastExchange (45) - +- * ColumnarToRow (44) + +- * CometColumnarToRow (44) +- CometHashAggregate (43) +- CometExchange (42) +- CometHashAggregate (41) @@ -160,7 +160,7 @@ Input [1]: [count#11] Keys: [] Functions [1]: [count(1)] -(25) ColumnarToRow [codegen id : 2] +(25) CometColumnarToRow [codegen id : 2] Input [1]: [amc#12] (26) CometScan parquet spark_catalog.default.web_sales @@ -243,7 +243,7 @@ Input [1]: [count#21] Keys: [] Functions [1]: [count(1)] -(44) ColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [pmc#22] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt index 95fd73d86e..72e45a9ad1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (2) Project [amc,pmc] BroadcastNestedLoopJoin - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [amc,count,count(1)] CometExchange #1 @@ -30,7 +30,7 @@ WholeStageCodegen (2) InputAdapter BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [pmc,count,count(1)] CometExchange #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/explain.txt index a8fa8f8571..591dce31ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/explain.txt @@ -1,11 +1,11 @@ == Physical Plan == -* ColumnarToRow (42) +* CometColumnarToRow (42) +- CometSort (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- Exchange (38) +- * HashAggregate (37) - +- * ColumnarToRow (36) + +- * CometColumnarToRow (36) +- CometProject (35) +- CometBroadcastHashJoin (34) :- CometProject (29) @@ -211,7 +211,7 @@ Arguments: [c_current_hdemo_sk#15], [hd_demo_sk#22], Inner, BuildRight Input [8]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_hdemo_sk#15, cd_marital_status#20, cd_education_status#21, hd_demo_sk#22] Arguments: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, cd_marital_status#20, cd_education_status#21], [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, cd_marital_status#20, cd_education_status#21] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, cd_marital_status#20, cd_education_status#21] (37) HashAggregate [codegen id : 1] @@ -240,14 +240,14 @@ Arguments: rangepartitioning(Returns_Loss#30 DESC NULLS LAST, 5), ENSURE_REQUIRE Input [4]: [Call_Center#27, Call_Center_Name#28, Manager#29, Returns_Loss#30] Arguments: [Call_Center#27, Call_Center_Name#28, Manager#29, Returns_Loss#30], [Returns_Loss#30 DESC NULLS LAST] -(42) ColumnarToRow [codegen id : 3] +(42) CometColumnarToRow [codegen id : 3] Input [4]: [Call_Center#27, Call_Center_Name#28, Manager#29, Returns_Loss#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = cr_returned_date_sk#8 IN dynamicpruning#9 BroadcastExchange (47) -+- * ColumnarToRow (46) ++- * CometColumnarToRow (46) +- CometProject (45) +- CometFilter (44) +- CometScan parquet spark_catalog.default.date_dim (43) @@ -268,7 +268,7 @@ Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#12)) AND (d_year#11 = 1 Input [3]: [d_date_sk#10, d_year#11, d_moy#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(46) ColumnarToRow [codegen id : 1] +(46) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (47) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt index 67669c48b7..70a78d6454 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] CometColumnarExchange [Returns_Loss] #1 @@ -9,7 +9,7 @@ WholeStageCodegen (3) Exchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 WholeStageCodegen (1) HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status,hd_demo_sk] @@ -31,7 +31,7 @@ WholeStageCodegen (3) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/explain.txt index 3981cc6e2f..2adb572ffc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/explain.txt @@ -6,7 +6,7 @@ +- * BroadcastHashJoin Inner BuildRight (27) :- * Project (25) : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * ColumnarToRow (9) + : :- * CometColumnarToRow (9) : : +- CometProject (8) : : +- CometBroadcastHashJoin (7) : : :- CometFilter (2) @@ -20,7 +20,7 @@ : +- * HashAggregate (21) : +- Exchange (20) : +- * HashAggregate (19) - : +- * ColumnarToRow (18) + : +- * CometColumnarToRow (18) : +- CometProject (17) : +- CometBroadcastHashJoin (16) : :- CometFilter (11) @@ -72,7 +72,7 @@ Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] Arguments: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5], [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] -(9) ColumnarToRow [codegen id : 4] +(9) CometColumnarToRow [codegen id : 4] Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] (10) CometScan parquet spark_catalog.default.web_sales @@ -115,7 +115,7 @@ Arguments: [ws_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight Input [4]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9, d_date_sk#11] Arguments: [ws_item_sk#7, ws_ext_discount_amt#8], [ws_item_sk#7, ws_ext_discount_amt#8] -(18) ColumnarToRow [codegen id : 1] +(18) CometColumnarToRow [codegen id : 1] Input [2]: [ws_item_sk#7, ws_ext_discount_amt#8] (19) HashAggregate [codegen id : 1] @@ -189,7 +189,7 @@ Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#22,17,2) AS Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (36) -+- * ColumnarToRow (35) ++- * CometColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan parquet spark_catalog.default.date_dim (32) @@ -210,7 +210,7 @@ Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 2000-01-27)) AND (d_date#2 Input [2]: [d_date_sk#19, d_date#24] Arguments: [d_date_sk#19], [d_date_sk#19] -(35) ColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt index aa24a049b2..ce96088799 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt @@ -8,7 +8,7 @@ WholeStageCodegen (5) BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_ext_discount_amt,ws_sold_date_sk] BroadcastHashJoin [i_item_sk,ws_item_sk,ws_ext_discount_amt,(1.3 * avg(ws_ext_discount_amt))] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] @@ -17,7 +17,7 @@ WholeStageCodegen (5) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] @@ -35,7 +35,7 @@ WholeStageCodegen (5) Exchange [ws_item_sk] #5 WholeStageCodegen (1) HashAggregate [ws_item_sk,ws_ext_discount_amt] [sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_item_sk,ws_ext_discount_amt] CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/explain.txt index d6e9142db4..0b158b70f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (22) +- * HashAggregate (21) +- Exchange (20) +- * HashAggregate (19) - +- * ColumnarToRow (18) + +- * CometColumnarToRow (18) +- CometProject (17) +- CometBroadcastHashJoin (16) :- CometProject (11) @@ -101,7 +101,7 @@ Arguments: [sr_reason_sk#8], [r_reason_sk#12], Inner, BuildRight Input [6]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10, r_reason_sk#12] Arguments: [ss_customer_sk#2, act_sales#14], [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#10) THEN (cast((ss_quantity#4 - sr_return_quantity#10) as decimal(10,0)) * ss_sales_price#5) ELSE (cast(ss_quantity#4 as decimal(10,0)) * ss_sales_price#5) END AS act_sales#14] -(18) ColumnarToRow [codegen id : 1] +(18) CometColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#2, act_sales#14] (19) HashAggregate [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt index fb8e4602dd..2034b220e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] Exchange [ss_customer_sk] #1 WholeStageCodegen (1) HashAggregate [ss_customer_sk,act_sales] [sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [sr_return_quantity,ss_quantity,ss_sales_price] [ss_customer_sk,act_sales] CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity,r_reason_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/explain.txt index 70aaeb9b6e..f7f762101b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/explain.txt @@ -4,7 +4,7 @@ +- * HashAggregate (38) +- * HashAggregate (37) +- * HashAggregate (36) - +- * ColumnarToRow (35) + +- * CometColumnarToRow (35) +- CometProject (34) +- CometBroadcastHashJoin (33) :- CometProject (28) @@ -198,7 +198,7 @@ Arguments: [ws_web_site_sk#3], [web_site_sk#18], Inner, BuildRight Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#18] Arguments: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -(35) ColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [codegen id : 1] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] (36) HashAggregate [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt index 4d0f1f982b..e1133ac9d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt @@ -6,7 +6,7 @@ WholeStageCodegen (2) HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/explain.txt index 4963b31940..3f5626c442 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/explain.txt @@ -4,7 +4,7 @@ +- * HashAggregate (51) +- * HashAggregate (50) +- * HashAggregate (49) - +- * ColumnarToRow (48) + +- * CometColumnarToRow (48) +- CometProject (47) +- CometBroadcastHashJoin (46) :- CometProject (41) @@ -265,7 +265,7 @@ Arguments: [ws_web_site_sk#3], [web_site_sk#23], Inner, BuildRight Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#23] Arguments: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -(48) ColumnarToRow [codegen id : 1] +(48) CometColumnarToRow [codegen id : 1] Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] (49) HashAggregate [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt index b6d5c5c365..94225db6f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt @@ -6,7 +6,7 @@ WholeStageCodegen (2) HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/explain.txt index d2e63bee25..677abdadae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (25) +* CometColumnarToRow (25) +- CometHashAggregate (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -138,6 +138,6 @@ Input [1]: [count#12] Keys: [] Functions [1]: [count(1)] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [1]: [count(1)#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt index d9a87aa3c4..9e86a025ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [count(1),count,count(1)] CometExchange #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/explain.txt index 1be5c9b947..65f6b498d1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/explain.txt @@ -2,7 +2,7 @@ * HashAggregate (25) +- Exchange (24) +- * HashAggregate (23) - +- * ColumnarToRow (22) + +- * CometColumnarToRow (22) +- CometProject (21) +- CometSortMergeJoin (20) :- CometSort (11) @@ -125,7 +125,7 @@ Arguments: [customer_sk#7, item_sk#8], [customer_sk#14, item_sk#15], FullOuter Input [4]: [customer_sk#7, item_sk#8, customer_sk#14, item_sk#15] Arguments: [customer_sk#7, customer_sk#14], [customer_sk#7, customer_sk#14] -(22) ColumnarToRow [codegen id : 1] +(22) CometColumnarToRow [codegen id : 1] Input [2]: [customer_sk#7, customer_sk#14] (23) HashAggregate [codegen id : 1] @@ -150,7 +150,7 @@ Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14) Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (30) -+- * ColumnarToRow (29) ++- * CometColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan parquet spark_catalog.default.date_dim (26) @@ -171,7 +171,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(29) ColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt index a9a3e66eb6..cd943ffbaa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt @@ -4,7 +4,7 @@ WholeStageCodegen (2) Exchange #1 WholeStageCodegen (1) HashAggregate [customer_sk,customer_sk] [sum,sum,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [customer_sk,customer_sk] CometSortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] @@ -18,7 +18,7 @@ WholeStageCodegen (2) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt index b750a8833e..87ac954684 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (25) +* CometColumnarToRow (25) +- CometProject (24) +- CometSort (23) +- CometColumnarExchange (22) @@ -10,7 +10,7 @@ +- * HashAggregate (17) +- Exchange (16) +- * HashAggregate (15) - +- * ColumnarToRow (14) + +- * CometColumnarToRow (14) +- CometProject (13) +- CometBroadcastHashJoin (12) :- CometProject (7) @@ -90,7 +90,7 @@ Arguments: [ss_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] Arguments: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [6]: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] (15) HashAggregate [codegen id : 1] @@ -139,14 +139,14 @@ Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemreve Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19, i_item_id#6] Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19] -(25) ColumnarToRow [codegen id : 5] +(25) CometColumnarToRow [codegen id : 5] Input [6]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (30) -+- * ColumnarToRow (29) ++- * CometColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan parquet spark_catalog.default.date_dim (26) @@ -167,7 +167,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(29) ColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt index b2c6adb5bb..6e42c615ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] @@ -18,7 +18,7 @@ WholeStageCodegen (5) Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 WholeStageCodegen (1) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] @@ -29,7 +29,7 @@ WholeStageCodegen (5) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/explain.txt index f94f48c207..26ff062e85 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (28) +- * HashAggregate (27) +- Exchange (26) +- * HashAggregate (25) - +- * ColumnarToRow (24) + +- * CometColumnarToRow (24) +- CometProject (23) +- CometBroadcastHashJoin (22) :- CometProject (17) @@ -140,7 +140,7 @@ Arguments: [cs_ship_date_sk#1], [d_date_sk#12], Inner, BuildRight Input [6]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9, cc_name#11, d_date_sk#12] Arguments: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#9, cc_name#11, _groupingexpression#14], [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#9, cc_name#11, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#14] -(24) ColumnarToRow [codegen id : 1] +(24) CometColumnarToRow [codegen id : 1] Input [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#9, cc_name#11, _groupingexpression#14] (25) HashAggregate [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt index 25ff3a2fed..ce1e94b726 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days , Exchange [_groupingexpression,sm_type,cc_name] #1 WholeStageCodegen (1) HashAggregate [_groupingexpression,sm_type,cc_name,cs_ship_date_sk,cs_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression] CometBroadcastHashJoin [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt index 27813d7e70..f9a474c65f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (40) +* CometColumnarToRow (40) +- CometTakeOrderedAndProject (39) +- CometProject (38) +- CometBroadcastHashJoin (37) @@ -224,14 +224,14 @@ Arguments: [c_customer_id#21], [c_customer_id#21] Input [1]: [c_customer_id#21] Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#21 ASC NULLS FIRST], output=[c_customer_id#21]), [c_customer_id#21], 100, [c_customer_id#21 ASC NULLS FIRST], [c_customer_id#21] -(40) ColumnarToRow [codegen id : 1] +(40) CometColumnarToRow [codegen id : 1] Input [1]: [c_customer_id#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (45) -+- * ColumnarToRow (44) ++- * CometColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(44) ColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt index 10d410761e..707c2ff0aa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [c_customer_id] CometProject [c_customer_id] @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt index e32e86f270..7544b94314 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt @@ -11,7 +11,7 @@ TakeOrderedAndProject (45) : : +- * Filter (27) : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * ColumnarToRow (12) + : : : :- * CometColumnarToRow (12) : : : : +- CometBroadcastHashJoin (11) : : : : :- CometFilter (2) : : : : : +- CometScan parquet spark_catalog.default.customer (1) @@ -24,24 +24,24 @@ TakeOrderedAndProject (45) : : : : +- CometFilter (5) : : : : +- CometScan parquet spark_catalog.default.date_dim (4) : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) + : : : +- * CometColumnarToRow (17) : : : +- CometProject (16) : : : +- CometBroadcastHashJoin (15) : : : :- CometScan parquet spark_catalog.default.web_sales (13) : : : +- ReusedExchange (14) : : +- BroadcastExchange (25) - : : +- * ColumnarToRow (24) + : : +- * CometColumnarToRow (24) : : +- CometProject (23) : : +- CometBroadcastHashJoin (22) : : :- CometScan parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (33) - : +- * ColumnarToRow (32) + : +- * CometColumnarToRow (32) : +- CometProject (31) : +- CometFilter (30) : +- CometScan parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (39) - +- * ColumnarToRow (38) + +- * CometColumnarToRow (38) +- CometFilter (37) +- CometScan parquet spark_catalog.default.customer_demographics (36) @@ -101,7 +101,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) ColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 5] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan parquet spark_catalog.default.web_sales @@ -123,7 +123,7 @@ Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#12] (18) BroadcastExchange @@ -155,7 +155,7 @@ Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [1]: [cs_ship_customer_sk#16] (25) BroadcastExchange @@ -191,7 +191,7 @@ Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana Input [2]: [ca_address_sk#20, ca_county#21] Arguments: [ca_address_sk#20], [ca_address_sk#20] -(32) ColumnarToRow [codegen id : 3] +(32) CometColumnarToRow [codegen id : 3] Input [1]: [ca_address_sk#20] (33) BroadcastExchange @@ -219,7 +219,7 @@ ReadSchema: struct Input [2]: [d_date_sk#14, d_year#15] Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#14)) -(72) ColumnarToRow [codegen id : 1] +(72) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] (73) BroadcastExchange @@ -425,7 +425,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#31 BroadcastExchange (77) -+- * ColumnarToRow (76) ++- * CometColumnarToRow (76) +- CometFilter (75) +- CometScan parquet spark_catalog.default.date_dim (74) @@ -441,7 +441,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 2002)) AND isnotnull(d_date_sk#32)) -(76) ColumnarToRow [codegen id : 1] +(76) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] (77) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt index e509c8c3c1..8896bc14be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [customer_preferred_cust_flag] CometProject [customer_preferred_cust_flag] @@ -24,7 +24,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -47,7 +47,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt index 18abad247b..7b8864d54f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (22) +- * Project (21) +- Window (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometSort (18) +- CometExchange (17) +- CometHashAggregate (16) @@ -109,7 +109,7 @@ Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometNativeShuff Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6], [i_class#9 ASC NULLS FIRST] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] (20) Window @@ -128,7 +128,7 @@ Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_ite Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (27) -+- * ColumnarToRow (26) ++- * CometColumnarToRow (26) +- CometProject (25) +- CometFilter (24) +- CometScan parquet spark_catalog.default.date_dim (23) @@ -149,7 +149,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (27) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt index e81a507139..c4c49fe339 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] CometExchange [i_class] #1 @@ -20,7 +20,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt index e3c357b407..774d652f36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (33) +* CometColumnarToRow (33) +- CometHashAggregate (32) +- CometExchange (31) +- CometHashAggregate (30) @@ -188,14 +188,14 @@ Input [7]: [sum#23, count#24, sum#25, count#26, sum#27, count#28, sum#29] Keys: [] Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))] -(33) ColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [codegen id : 1] Input [4]: [avg(ss_quantity)#30, avg(ss_ext_sales_price)#31, avg(ss_ext_wholesale_cost)#32, sum(ss_ext_wholesale_cost)#33] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (38) -+- * ColumnarToRow (37) ++- * CometColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometScan parquet spark_catalog.default.date_dim (34) @@ -216,7 +216,7 @@ Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#16, d_year#17] Arguments: [d_date_sk#16], [d_date_sk#16] -(37) ColumnarToRow [codegen id : 1] +(37) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt index 46ed9b6365..c4d3b744d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),sum,count,sum,count,sum,count,sum,avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost))] CometExchange #1 @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt index 3aae3a13ec..868da4cd92 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (102) +* CometColumnarToRow (102) +- CometTakeOrderedAndProject (101) +- CometHashAggregate (100) +- CometExchange (99) @@ -569,13 +569,13 @@ Functions [2]: [sum(sales#49), sum(number_sales#50)] Input [6]: [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, sum(sales)#97, sum(number_sales)#98] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#89 ASC NULLS FIRST,i_brand_id#90 ASC NULLS FIRST,i_class_id#91 ASC NULLS FIRST,i_category_id#92 ASC NULLS FIRST], output=[channel#89,i_brand_id#90,i_class_id#91,i_category_id#92,sum(sales)#97,sum(number_sales)#98]), [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, sum(sales)#97, sum(number_sales)#98], 100, [channel#89 ASC NULLS FIRST, i_brand_id#90 ASC NULLS FIRST, i_class_id#91 ASC NULLS FIRST, i_category_id#92 ASC NULLS FIRST], [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, sum(sales)#97, sum(number_sales)#98] -(102) ColumnarToRow [codegen id : 1] +(102) CometColumnarToRow [codegen id : 1] Input [6]: [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, sum(sales)#97, sum(number_sales)#98] ===== Subqueries ===== Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#51, [id=#52] -* ColumnarToRow (119) +* CometColumnarToRow (119) +- CometHashAggregate (118) +- CometExchange (117) +- CometHashAggregate (116) @@ -670,7 +670,7 @@ Input [2]: [sum#120, count#121] Keys: [] Functions [1]: [avg((cast(quantity#104 as decimal(10,0)) * list_price#105))] -(119) ColumnarToRow [codegen id : 1] +(119) CometColumnarToRow [codegen id : 1] Input [1]: [average_sales#122] Subquery:2 Hosting operator id = 103 Hosting Expression = ss_sold_date_sk#101 IN dynamicpruning#12 @@ -681,7 +681,7 @@ Subquery:4 Hosting operator id = 111 Hosting Expression = ws_sold_date_sk#115 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (124) -+- * ColumnarToRow (123) ++- * CometColumnarToRow (123) +- CometProject (122) +- CometFilter (121) +- CometScan parquet spark_catalog.default.date_dim (120) @@ -702,7 +702,7 @@ Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2 Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(123) ColumnarToRow [codegen id : 1] +(123) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] (124) BroadcastExchange @@ -711,7 +711,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (129) -+- * ColumnarToRow (128) ++- * CometColumnarToRow (128) +- CometProject (127) +- CometFilter (126) +- CometScan parquet spark_catalog.default.date_dim (125) @@ -732,7 +732,7 @@ Condition : (((isnotnull(d_year#123) AND (d_year#123 >= 1999)) AND (d_year#123 < Input [2]: [d_date_sk#26, d_year#123] Arguments: [d_date_sk#26], [d_date_sk#26] -(128) ColumnarToRow [codegen id : 1] +(128) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] (129) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt index 852af8ad82..ec0b46d611 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales),spark_grouping_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)] @@ -11,7 +11,7 @@ WholeStageCodegen (1) CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] Subquery #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #15 @@ -45,7 +45,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -69,7 +69,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt index 47feaf8987..a3ed5f9e1c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (85) +* CometColumnarToRow (85) +- CometTakeOrderedAndProject (84) +- CometBroadcastHashJoin (83) :- CometFilter (64) @@ -480,13 +480,13 @@ Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#61, i_c Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#50,i_brand_id#40,i_class_id#41,i_category_id#42,sales#51,number_sales#52,channel#71,i_brand_id#61,i_class_id#62,i_category_id#63,sales#72,number_sales#73]), [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73], 100, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] -(85) ColumnarToRow [codegen id : 1] +(85) CometColumnarToRow [codegen id : 1] Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] ===== Subqueries ===== Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* ColumnarToRow (102) +* CometColumnarToRow (102) +- CometHashAggregate (101) +- CometExchange (100) +- CometHashAggregate (99) @@ -581,7 +581,7 @@ Input [2]: [sum#95, count#96] Keys: [] Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] -(102) ColumnarToRow [codegen id : 1] +(102) CometColumnarToRow [codegen id : 1] Input [1]: [average_sales#97] Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#12 @@ -592,7 +592,7 @@ Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#90 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (107) -+- * ColumnarToRow (106) ++- * CometColumnarToRow (106) +- CometProject (105) +- CometFilter (104) +- CometScan parquet spark_catalog.default.date_dim (103) @@ -613,7 +613,7 @@ Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subq Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(106) ColumnarToRow [codegen id : 1] +(106) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] (107) BroadcastExchange @@ -621,7 +621,7 @@ Input [1]: [d_date_sk#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* ColumnarToRow (111) +* CometColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometScan parquet spark_catalog.default.date_dim (108) @@ -642,12 +642,12 @@ Condition : (((((isnotnull(d_year#99) AND isnotnull(d_moy#100)) AND isnotnull(d_ Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] Arguments: [d_week_seq#98], [d_week_seq#98] -(111) ColumnarToRow [codegen id : 1] +(111) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#98] Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (116) -+- * ColumnarToRow (115) ++- * CometColumnarToRow (115) +- CometProject (114) +- CometFilter (113) +- CometScan parquet spark_catalog.default.date_dim (112) @@ -668,7 +668,7 @@ Condition : (((isnotnull(d_year#102) AND (d_year#102 >= 1999)) AND (d_year#102 < Input [2]: [d_date_sk#26, d_year#102] Arguments: [d_date_sk#26], [d_date_sk#26] -(115) ColumnarToRow [codegen id : 1] +(115) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] (116) BroadcastExchange @@ -685,7 +685,7 @@ Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquer Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 BroadcastExchange (121) -+- * ColumnarToRow (120) ++- * CometColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan parquet spark_catalog.default.date_dim (117) @@ -706,7 +706,7 @@ Condition : ((isnotnull(d_week_seq#65) AND (d_week_seq#65 = Subquery scalar-subq Input [2]: [d_date_sk#64, d_week_seq#65] Arguments: [d_date_sk#64], [d_date_sk#64] -(120) ColumnarToRow [codegen id : 1] +(120) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#64] (121) BroadcastExchange @@ -714,7 +714,7 @@ Input [1]: [d_date_sk#64] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#66, [id=#67] -* ColumnarToRow (125) +* CometColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometScan parquet spark_catalog.default.date_dim (122) @@ -735,7 +735,7 @@ Condition : (((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND isnotnull(d Input [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] Arguments: [d_week_seq#103], [d_week_seq#103] -(125) ColumnarToRow [codegen id : 1] +(125) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#103] Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt index bb4be4d3ae..befc6dadbb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt @@ -1,12 +1,12 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] CometBroadcastHashJoin [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Subquery #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #14 @@ -40,13 +40,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_week_seq] Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_week_seq,d_year,d_moy,d_dom] @@ -71,7 +71,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -132,13 +132,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_week_seq] Subquery #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_week_seq,d_year,d_moy,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt index 037c526267..3ee326ad53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (23) +* CometColumnarToRow (23) +- CometTakeOrderedAndProject (22) +- CometHashAggregate (21) +- CometExchange (20) @@ -130,14 +130,14 @@ Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] Input [2]: [ca_zip#9, sum(cs_sales_price)#14] Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_zip#9 ASC NULLS FIRST], output=[ca_zip#9,sum(cs_sales_price)#14]), [ca_zip#9, sum(cs_sales_price)#14], 100, [ca_zip#9 ASC NULLS FIRST], [ca_zip#9, sum(cs_sales_price)#14] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarToRow [codegen id : 1] Input [2]: [ca_zip#9, sum(cs_sales_price)#14] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * ColumnarToRow (27) ++- * CometColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan parquet spark_catalog.default.date_dim (24) @@ -158,7 +158,7 @@ Condition : ((((isnotnull(d_qoy#12) AND isnotnull(d_year#11)) AND (d_qoy#12 = 2) Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(27) ColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt index d417f1c682..425f20e10c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [ca_zip,sum(cs_sales_price)] CometHashAggregate [ca_zip,sum(cs_sales_price),sum,sum(UnscaledValue(cs_sales_price))] @@ -16,7 +16,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt index acd12b2771..5eeca31ae8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt @@ -3,7 +3,7 @@ +- Exchange (39) +- * HashAggregate (38) +- * HashAggregate (37) - +- * ColumnarToRow (36) + +- * CometColumnarToRow (36) +- CometHashAggregate (35) +- CometProject (34) +- CometBroadcastHashJoin (33) @@ -203,7 +203,7 @@ Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [3]: [cs_order_number#5, sum#20, sum#21] (37) HashAggregate [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt index f054ee03e9..c51841720d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt @@ -5,7 +5,7 @@ WholeStageCodegen (2) WholeStageCodegen (1) HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [cs_order_number,sum,sum,cs_ext_ship_cost,cs_net_profit] CometProject [cs_order_number,cs_ext_ship_cost,cs_net_profit] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt index 496ec2f1a1..08c00a0bc4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (42) +* CometColumnarToRow (42) +- CometTakeOrderedAndProject (41) +- CometHashAggregate (40) +- CometExchange (39) @@ -239,14 +239,14 @@ Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_qu Input [15]: [i_item_id#27, i_item_desc#28, s_state#25, store_sales_quantitycount#47, store_sales_quantityave#48, store_sales_quantitystdev#49, store_sales_quantitycov#50, as_store_returns_quantitycount#51, as_store_returns_quantityave#52, as_store_returns_quantitystdev#53, store_returns_quantitycov#54, catalog_sales_quantitycount#55, catalog_sales_quantityave#56, catalog_sales_quantitystdev#57, catalog_sales_quantitycov#58] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#27 ASC NULLS FIRST,i_item_desc#28 ASC NULLS FIRST,s_state#25 ASC NULLS FIRST], output=[i_item_id#27,i_item_desc#28,s_state#25,store_sales_quantitycount#47,store_sales_quantityave#48,store_sales_quantitystdev#49,store_sales_quantitycov#50,as_store_returns_quantitycount#51,as_store_returns_quantityave#52,as_store_returns_quantitystdev#53,store_returns_quantitycov#54,catalog_sales_quantitycount#55,catalog_sales_quantityave#56,catalog_sales_quantitystdev#57,catalog_sales_quantitycov#58]), [i_item_id#27, i_item_desc#28, s_state#25, store_sales_quantitycount#47, store_sales_quantityave#48, store_sales_quantitystdev#49, store_sales_quantitycov#50, as_store_returns_quantitycount#51, as_store_returns_quantityave#52, as_store_returns_quantitystdev#53, store_returns_quantitycov#54, catalog_sales_quantitycount#55, catalog_sales_quantityave#56, catalog_sales_quantitystdev#57, catalog_sales_quantitycov#58], 100, [i_item_id#27 ASC NULLS FIRST, i_item_desc#28 ASC NULLS FIRST, s_state#25 ASC NULLS FIRST], [i_item_id#27, i_item_desc#28, s_state#25, store_sales_quantitycount#47, store_sales_quantityave#48, store_sales_quantitystdev#49, store_sales_quantitycov#50, as_store_returns_quantitycount#51, as_store_returns_quantityave#52, as_store_returns_quantitystdev#53, store_returns_quantitycov#54, catalog_sales_quantitycount#55, catalog_sales_quantityave#56, catalog_sales_quantitystdev#57, catalog_sales_quantitycov#58] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarToRow [codegen id : 1] Input [15]: [i_item_id#27, i_item_desc#28, s_state#25, store_sales_quantitycount#47, store_sales_quantityave#48, store_sales_quantitystdev#49, store_sales_quantitycov#50, as_store_returns_quantitycount#51, as_store_returns_quantityave#52, as_store_returns_quantitystdev#53, store_returns_quantitycov#54, catalog_sales_quantitycount#55, catalog_sales_quantityave#56, catalog_sales_quantitystdev#57, catalog_sales_quantitycov#58] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (47) -+- * ColumnarToRow (46) ++- * CometColumnarToRow (46) +- CometProject (45) +- CometFilter (44) +- CometScan parquet spark_catalog.default.date_dim (43) @@ -267,7 +267,7 @@ Condition : ((isnotnull(d_quarter_name#20) AND (d_quarter_name#20 = 2001Q1)) AND Input [2]: [d_date_sk#19, d_quarter_name#20] Arguments: [d_date_sk#19], [d_date_sk#19] -(46) ColumnarToRow [codegen id : 1] +(46) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] (47) BroadcastExchange @@ -276,7 +276,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -297,7 +297,7 @@ Condition : (d_quarter_name#22 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk Input [2]: [d_date_sk#21, d_quarter_name#22] Arguments: [d_date_sk#21], [d_date_sk#21] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#21] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt index a3adaf1724..97000b91df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] CometHashAggregate [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count(ss_quantity),avg(ss_quantity),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(sr_return_quantity),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cs_quantity),stddev_samp(cast(cs_quantity as double))] @@ -24,7 +24,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_quarter_name] @@ -35,7 +35,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_quarter_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt index c41a1ef7d4..2d08a0303c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (41) +* CometColumnarToRow (41) +- CometTakeOrderedAndProject (40) +- CometHashAggregate (39) +- CometExchange (38) @@ -232,14 +232,14 @@ Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_pric Input [11]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, agg1#48, agg2#49, agg3#50, agg4#51, agg5#52, agg6#53, agg7#54] Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#30 ASC NULLS FIRST,ca_state#31 ASC NULLS FIRST,ca_county#32 ASC NULLS FIRST,i_item_id#29 ASC NULLS FIRST], output=[i_item_id#29,ca_country#30,ca_state#31,ca_county#32,agg1#48,agg2#49,agg3#50,agg4#51,agg5#52,agg6#53,agg7#54]), [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, agg1#48, agg2#49, agg3#50, agg4#51, agg5#52, agg6#53, agg7#54], 100, [ca_country#30 ASC NULLS FIRST, ca_state#31 ASC NULLS FIRST, ca_county#32 ASC NULLS FIRST, i_item_id#29 ASC NULLS FIRST], [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, agg1#48, agg2#49, agg3#50, agg4#51, agg5#52, agg6#53, agg7#54] -(41) ColumnarToRow [codegen id : 1] +(41) CometColumnarToRow [codegen id : 1] Input [11]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, agg1#48, agg2#49, agg3#50, agg4#51, agg5#52, agg6#53, agg7#54] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (46) -+- * ColumnarToRow (45) ++- * CometColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometScan parquet spark_catalog.default.date_dim (42) @@ -260,7 +260,7 @@ Condition : ((isnotnull(d_year#26) AND (d_year#26 = 1998)) AND isnotnull(d_date_ Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(45) ColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt index e26491c76a..eaf02960fe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2)))] @@ -23,7 +23,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt index b91551e576..c1bf0f14a0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (34) +* CometColumnarToRow (34) +- CometTakeOrderedAndProject (33) +- CometHashAggregate (32) +- CometExchange (31) @@ -193,6 +193,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] Input [5]: [brand_id#22, brand#23, i_manufact_id#12, i_manufact#13, ext_price#24] Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#24 DESC NULLS LAST,brand#23 ASC NULLS FIRST,brand_id#22 ASC NULLS FIRST,i_manufact_id#12 ASC NULLS FIRST,i_manufact#13 ASC NULLS FIRST], output=[brand_id#22,brand#23,i_manufact_id#12,i_manufact#13,ext_price#24]), [brand_id#22, brand#23, i_manufact_id#12, i_manufact#13, ext_price#24], 100, [ext_price#24 DESC NULLS LAST, brand#23 ASC NULLS FIRST, brand_id#22 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#13 ASC NULLS FIRST], [brand_id#22, brand#23, i_manufact_id#12, i_manufact#13, ext_price#24] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [5]: [brand_id#22, brand#23, i_manufact_id#12, i_manufact#13, ext_price#24] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt index 46b0d650a5..bfbf7fe94a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [brand_id,brand,i_manufact_id,i_manufact,ext_price] CometHashAggregate [brand_id,brand,i_manufact_id,i_manufact,ext_price,i_brand,i_brand_id,sum,sum(UnscaledValue(ss_ext_sales_price))] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt index 24c39591f5..0baaf2cdfa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (33) +* CometColumnarToRow (33) +- CometSort (32) +- CometColumnarExchange (31) +- CometProject (30) @@ -183,6 +183,6 @@ Arguments: rangepartitioning(d_week_seq1#28 ASC NULLS FIRST, 5), ENSURE_REQUIREM Input [8]: [d_week_seq1#28, round((sun_sales1 / sun_sales2), 2)#53, round((mon_sales1 / mon_sales2), 2)#54, round((tue_sales1 / tue_sales2), 2)#55, round((wed_sales1 / wed_sales2), 2)#56, round((thu_sales1 / thu_sales2), 2)#57, round((fri_sales1 / fri_sales2), 2)#58, round((sat_sales1 / sat_sales2), 2)#59] Arguments: [d_week_seq1#28, round((sun_sales1 / sun_sales2), 2)#53, round((mon_sales1 / mon_sales2), 2)#54, round((tue_sales1 / tue_sales2), 2)#55, round((wed_sales1 / wed_sales2), 2)#56, round((thu_sales1 / thu_sales2), 2)#57, round((fri_sales1 / fri_sales2), 2)#58, round((sat_sales1 / sat_sales2), 2)#59], [d_week_seq1#28 ASC NULLS FIRST] -(33) ColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [codegen id : 1] Input [8]: [d_week_seq1#28, round((sun_sales1 / sun_sales2), 2)#53, round((mon_sales1 / mon_sales2), 2)#54, round((tue_sales1 / tue_sales2), 2)#55, round((wed_sales1 / wed_sales2), 2)#56, round((thu_sales1 / thu_sales2), 2)#57, round((fri_sales1 / fri_sales2), 2)#58, round((sat_sales1 / sat_sales2), 2)#59] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt index ed772bade0..38cb758b73 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] CometColumnarExchange [d_week_seq1] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt index 909acbf7f8..6a32155103 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (22) +- * Project (21) +- Window (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometSort (18) +- CometExchange (17) +- CometHashAggregate (16) @@ -109,7 +109,7 @@ Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometNativeShuff Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6], [i_class#9 ASC NULLS FIRST] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] (20) Window @@ -128,7 +128,7 @@ Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_ite Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (27) -+- * ColumnarToRow (26) ++- * CometColumnarToRow (26) +- CometProject (25) +- CometFilter (24) +- CometScan parquet spark_catalog.default.date_dim (23) @@ -149,7 +149,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (27) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt index 9a906085b8..147d91b586 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] CometExchange [i_class] #1 @@ -20,7 +20,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt index 31a5bef868..0340ef91c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (24) +* CometColumnarToRow (24) +- CometTakeOrderedAndProject (23) +- CometFilter (22) +- CometHashAggregate (21) @@ -135,14 +135,14 @@ Condition : (CASE WHEN (inv_before#15 > 0) THEN (knownfloatingpointnormalized(no Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#15, inv_after#16] Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_warehouse_name#7 ASC NULLS FIRST,i_item_id#9 ASC NULLS FIRST], output=[w_warehouse_name#7,i_item_id#9,inv_before#15,inv_after#16]), [w_warehouse_name#7, i_item_id#9, inv_before#15, inv_after#16], 100, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST], [w_warehouse_name#7, i_item_id#9, inv_before#15, inv_after#16] -(24) ColumnarToRow [codegen id : 1] +(24) CometColumnarToRow [codegen id : 1] Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#15, inv_after#16] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (28) -+- * ColumnarToRow (27) ++- * CometColumnarToRow (27) +- CometFilter (26) +- CometScan parquet spark_catalog.default.date_dim (25) @@ -158,7 +158,7 @@ ReadSchema: struct Input [2]: [d_date_sk#11, d_date#12] Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-10)) AND (d_date#12 <= 2000-04-10)) AND isnotnull(d_date_sk#11)) -(27) ColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#11, d_date#12] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt index 6e483f6049..6ff97b53ac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] CometFilter [w_warehouse_name,i_item_id,inv_before,inv_after] @@ -17,7 +17,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt index 526dadb3c8..759689b1b8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (24) +* CometColumnarToRow (24) +- CometTakeOrderedAndProject (23) +- CometHashAggregate (22) +- CometExchange (21) @@ -135,14 +135,14 @@ Functions [1]: [avg(inv_quantity_on_hand#3)] Input [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#21] Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#21 ASC NULLS FIRST,i_product_name#14 ASC NULLS FIRST,i_brand#15 ASC NULLS FIRST,i_class#16 ASC NULLS FIRST,i_category#17 ASC NULLS FIRST], output=[i_product_name#14,i_brand#15,i_class#16,i_category#17,qoh#21]), [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#21], 100, [qoh#21 ASC NULLS FIRST, i_product_name#14 ASC NULLS FIRST, i_brand#15 ASC NULLS FIRST, i_class#16 ASC NULLS FIRST, i_category#17 ASC NULLS FIRST], [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#21] -(24) ColumnarToRow [codegen id : 1] +(24) CometColumnarToRow [codegen id : 1] Input [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (29) -+- * ColumnarToRow (28) ++- * CometColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan parquet spark_catalog.default.date_dim (25) @@ -163,7 +163,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_mont Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt index ed997b5d50..f9d4ce909d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,spark_grouping_id,sum,count,avg(inv_quantity_on_hand)] @@ -17,7 +17,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt index 45dd2f9be2..9c0af9c56d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (67) +* CometColumnarToRow (67) +- CometHashAggregate (66) +- CometExchange (65) +- CometHashAggregate (64) @@ -370,14 +370,14 @@ Input [2]: [sum#41, isEmpty#42] Keys: [] Functions [1]: [sum(sales#32)] -(67) ColumnarToRow [codegen id : 1] +(67) CometColumnarToRow [codegen id : 1] Input [1]: [sum(sales)#43] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (72) -+- * ColumnarToRow (71) ++- * CometColumnarToRow (71) +- CometProject (70) +- CometFilter (69) +- CometScan parquet spark_catalog.default.date_dim (68) @@ -398,7 +398,7 @@ Condition : ((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2 Input [3]: [d_date_sk#29, d_year#30, d_moy#31] Arguments: [d_date_sk#29], [d_date_sk#29] -(71) ColumnarToRow [codegen id : 1] +(71) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#29] (72) BroadcastExchange @@ -407,7 +407,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 2 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (77) -+- * ColumnarToRow (76) ++- * CometColumnarToRow (76) +- CometProject (75) +- CometFilter (74) +- CometScan parquet spark_catalog.default.date_dim (73) @@ -428,7 +428,7 @@ Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(76) ColumnarToRow [codegen id : 1] +(76) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] (77) BroadcastExchange @@ -436,7 +436,7 @@ Input [2]: [d_date_sk#10, d_date#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:3 Hosting operator id = 36 Hosting Expression = Subquery scalar-subquery#27, [id=#28] -* ColumnarToRow (95) +* CometColumnarToRow (95) +- CometHashAggregate (94) +- CometExchange (93) +- CometHashAggregate (92) @@ -536,12 +536,12 @@ Input [1]: [max#55] Keys: [] Functions [1]: [max(csales#54)] -(95) ColumnarToRow [codegen id : 1] +(95) CometColumnarToRow [codegen id : 1] Input [1]: [tpcds_cmax#56] Subquery:4 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#47 IN dynamicpruning#48 BroadcastExchange (100) -+- * ColumnarToRow (99) ++- * CometColumnarToRow (99) +- CometProject (98) +- CometFilter (97) +- CometScan parquet spark_catalog.default.date_dim (96) @@ -562,7 +562,7 @@ Condition : (d_year#51 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#50)) Input [2]: [d_date_sk#50, d_year#51] Arguments: [d_date_sk#50], [d_date_sk#50] -(99) ColumnarToRow [codegen id : 1] +(99) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#50] (100) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt index 7606f105f3..4ecd0dfcc5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [sum(sales),sum,isEmpty,sum(sales)] CometExchange #1 @@ -17,7 +17,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -37,7 +37,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_year] @@ -54,7 +54,7 @@ WholeStageCodegen (1) CometFilter [c_customer_sk,ssales] Subquery #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [tpcds_cmax,max,max(csales)] CometExchange #11 @@ -71,7 +71,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt index 4a4b3a41ca..056bcc611d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (87) +* CometColumnarToRow (87) +- CometTakeOrderedAndProject (86) +- CometUnion (85) :- CometHashAggregate (62) @@ -479,14 +479,14 @@ Child 1 Input [3]: [c_last_name#45, c_first_name#44, sales#50] Input [3]: [c_last_name#31, c_first_name#30, sales#49] Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#31 ASC NULLS FIRST,c_first_name#30 ASC NULLS FIRST,sales#49 ASC NULLS FIRST], output=[c_last_name#31,c_first_name#30,sales#49]), [c_last_name#31, c_first_name#30, sales#49], 100, [c_last_name#31 ASC NULLS FIRST, c_first_name#30 ASC NULLS FIRST, sales#49 ASC NULLS FIRST], [c_last_name#31, c_first_name#30, sales#49] -(87) ColumnarToRow [codegen id : 1] +(87) CometColumnarToRow [codegen id : 1] Input [3]: [c_last_name#31, c_first_name#30, sales#49] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (92) -+- * ColumnarToRow (91) ++- * CometColumnarToRow (91) +- CometProject (90) +- CometFilter (89) +- CometScan parquet spark_catalog.default.date_dim (88) @@ -507,7 +507,7 @@ Condition : ((((isnotnull(d_year#33) AND isnotnull(d_moy#34)) AND (d_year#33 = 2 Input [3]: [d_date_sk#32, d_year#33, d_moy#34] Arguments: [d_date_sk#32], [d_date_sk#32] -(91) ColumnarToRow [codegen id : 1] +(91) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#32] (92) BroadcastExchange @@ -516,7 +516,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (97) -+- * ColumnarToRow (96) ++- * CometColumnarToRow (96) +- CometProject (95) +- CometFilter (94) +- CometScan parquet spark_catalog.default.date_dim (93) @@ -537,7 +537,7 @@ Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(96) ColumnarToRow [codegen id : 1] +(96) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] (97) BroadcastExchange @@ -545,7 +545,7 @@ Input [2]: [d_date_sk#10, d_date#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] Subquery:3 Hosting operator id = 37 Hosting Expression = Subquery scalar-subquery#27, [id=#28] -* ColumnarToRow (115) +* CometColumnarToRow (115) +- CometHashAggregate (114) +- CometExchange (113) +- CometHashAggregate (112) @@ -645,12 +645,12 @@ Input [1]: [max#62] Keys: [] Functions [1]: [max(csales#61)] -(115) ColumnarToRow [codegen id : 1] +(115) CometColumnarToRow [codegen id : 1] Input [1]: [tpcds_cmax#63] Subquery:4 Hosting operator id = 98 Hosting Expression = ss_sold_date_sk#54 IN dynamicpruning#55 BroadcastExchange (120) -+- * ColumnarToRow (119) ++- * CometColumnarToRow (119) +- CometProject (118) +- CometFilter (117) +- CometScan parquet spark_catalog.default.date_dim (116) @@ -671,7 +671,7 @@ Condition : (d_year#58 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#57)) Input [2]: [d_date_sk#57, d_year#58] Arguments: [d_date_sk#57], [d_date_sk#57] -(119) ColumnarToRow [codegen id : 1] +(119) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#57] (120) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt index d503bb68af..9493bffd25 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [c_last_name,c_first_name,sales] CometUnion [c_last_name,c_first_name,sales] @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -40,7 +40,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_year] @@ -57,7 +57,7 @@ WholeStageCodegen (1) CometFilter [c_customer_sk,ssales] Subquery #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [tpcds_cmax,max,max(csales)] CometExchange #11 @@ -74,7 +74,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt index 2208ce2663..24d906d97f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt @@ -8,7 +8,7 @@ +- * HashAggregate (36) +- * Project (35) +- * BroadcastHashJoin Inner BuildRight (34) - :- * ColumnarToRow (29) + :- * CometColumnarToRow (29) : +- CometProject (28) : +- CometBroadcastHashJoin (27) : :- CometProject (23) @@ -38,7 +38,7 @@ : +- CometFilter (25) : +- CometScan parquet spark_catalog.default.customer (24) +- BroadcastExchange (33) - +- * ColumnarToRow (32) + +- * CometColumnarToRow (32) +- CometFilter (31) +- CometScan parquet spark_catalog.default.customer_address (30) @@ -174,7 +174,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Arguments: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24], [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] -(29) ColumnarToRow [codegen id : 2] +(29) CometColumnarToRow [codegen id : 2] Input [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] (30) CometScan parquet spark_catalog.default.customer_address @@ -188,7 +188,7 @@ ReadSchema: struct Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] (33) BroadcastExchange @@ -256,7 +256,7 @@ Subquery:1 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquer +- * HashAggregate (64) +- * Project (63) +- * BroadcastHashJoin Inner BuildRight (62) - :- * ColumnarToRow (60) + :- * CometColumnarToRow (60) : +- CometProject (59) : +- CometBroadcastHashJoin (58) : :- CometProject (56) @@ -348,7 +348,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Arguments: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24], [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] -(60) ColumnarToRow [codegen id : 2] +(60) CometColumnarToRow [codegen id : 2] Input [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] (61) ReusedExchange [Reuses operator id: 33] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt index bd14d82504..cd048032d9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt @@ -14,7 +14,7 @@ WholeStageCodegen (4) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] @@ -47,7 +47,7 @@ WholeStageCodegen (4) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] @@ -80,7 +80,7 @@ WholeStageCodegen (4) InputAdapter BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt index c3b9566431..600348b065 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt @@ -8,7 +8,7 @@ +- * HashAggregate (36) +- * Project (35) +- * BroadcastHashJoin Inner BuildRight (34) - :- * ColumnarToRow (29) + :- * CometColumnarToRow (29) : +- CometProject (28) : +- CometBroadcastHashJoin (27) : :- CometProject (23) @@ -38,7 +38,7 @@ : +- CometFilter (25) : +- CometScan parquet spark_catalog.default.customer (24) +- BroadcastExchange (33) - +- * ColumnarToRow (32) + +- * CometColumnarToRow (32) +- CometFilter (31) +- CometScan parquet spark_catalog.default.customer_address (30) @@ -174,7 +174,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Arguments: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24], [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] -(29) ColumnarToRow [codegen id : 2] +(29) CometColumnarToRow [codegen id : 2] Input [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] (30) CometScan parquet spark_catalog.default.customer_address @@ -188,7 +188,7 @@ ReadSchema: struct Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] (33) BroadcastExchange @@ -256,7 +256,7 @@ Subquery:1 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquer +- * HashAggregate (64) +- * Project (63) +- * BroadcastHashJoin Inner BuildRight (62) - :- * ColumnarToRow (60) + :- * CometColumnarToRow (60) : +- CometProject (59) : +- CometBroadcastHashJoin (58) : :- CometProject (56) @@ -348,7 +348,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Arguments: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24], [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] -(60) ColumnarToRow [codegen id : 2] +(60) CometColumnarToRow [codegen id : 2] Input [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] (61) ReusedExchange [Reuses operator id: 33] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt index bd14d82504..cd048032d9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt @@ -14,7 +14,7 @@ WholeStageCodegen (4) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] @@ -47,7 +47,7 @@ WholeStageCodegen (4) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] @@ -80,7 +80,7 @@ WholeStageCodegen (4) InputAdapter BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt index 6c78335481..62c54c9f56 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (42) +* CometColumnarToRow (42) +- CometTakeOrderedAndProject (41) +- CometHashAggregate (40) +- CometExchange (39) @@ -239,14 +239,14 @@ Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_lo Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#35, store_returns_loss#36, catalog_sales_profit#37] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#30 ASC NULLS FIRST,i_item_desc#31 ASC NULLS FIRST,s_store_id#27 ASC NULLS FIRST,s_store_name#28 ASC NULLS FIRST], output=[i_item_id#30,i_item_desc#31,s_store_id#27,s_store_name#28,store_sales_profit#35,store_returns_loss#36,catalog_sales_profit#37]), [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#35, store_returns_loss#36, catalog_sales_profit#37], 100, [i_item_id#30 ASC NULLS FIRST, i_item_desc#31 ASC NULLS FIRST, s_store_id#27 ASC NULLS FIRST, s_store_name#28 ASC NULLS FIRST], [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#35, store_returns_loss#36, catalog_sales_profit#37] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#35, store_returns_loss#36, catalog_sales_profit#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (47) -+- * ColumnarToRow (46) ++- * CometColumnarToRow (46) +- CometProject (45) +- CometFilter (44) +- CometScan parquet spark_catalog.default.date_dim (43) @@ -267,7 +267,7 @@ Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4) Input [3]: [d_date_sk#19, d_year#20, d_moy#21] Arguments: [d_date_sk#19], [d_date_sk#19] -(46) ColumnarToRow [codegen id : 1] +(46) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] (47) BroadcastExchange @@ -276,7 +276,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -297,7 +297,7 @@ Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= Input [3]: [d_date_sk#22, d_year#23, d_moy#24] Arguments: [d_date_sk#22], [d_date_sk#22] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt index 13cb05053c..e64e8bc840 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit,sum,sum,sum,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit))] @@ -24,7 +24,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -35,7 +35,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt index b93df3398e..525313e3f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (30) +* CometColumnarToRow (30) +- CometTakeOrderedAndProject (29) +- CometHashAggregate (28) +- CometExchange (27) @@ -169,14 +169,14 @@ Functions [4]: [avg(cs_quantity#4), avg(UnscaledValue(cs_list_price#5)), avg(Uns Input [5]: [i_item_id#17, agg1#29, agg2#30, agg3#31, agg4#32] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#17 ASC NULLS FIRST], output=[i_item_id#17,agg1#29,agg2#30,agg3#31,agg4#32]), [i_item_id#17, agg1#29, agg2#30, agg3#31, agg4#32], 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#29, agg2#30, agg3#31, agg4#32] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [5]: [i_item_id#17, agg1#29, agg2#30, agg3#31, agg4#32] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (35) -+- * ColumnarToRow (34) ++- * CometColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan parquet spark_catalog.default.date_dim (31) @@ -197,7 +197,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt index f643ff7b04..16f8d0848e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count,avg(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price))] @@ -18,7 +18,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt index 4d1b12626a..28bc38292b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (30) +* CometColumnarToRow (30) +- CometTakeOrderedAndProject (29) +- CometHashAggregate (28) +- CometExchange (27) @@ -169,14 +169,14 @@ Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(Uns Input [7]: [i_item_id#20, s_state#21, g_state#31, agg1#32, agg2#33, agg3#34, agg4#35] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#20 ASC NULLS FIRST,s_state#21 ASC NULLS FIRST], output=[i_item_id#20,s_state#21,g_state#31,agg1#32,agg2#33,agg3#34,agg4#35]), [i_item_id#20, s_state#21, g_state#31, agg1#32, agg2#33, agg3#34, agg4#35], 100, [i_item_id#20 ASC NULLS FIRST, s_state#21 ASC NULLS FIRST], [i_item_id#20, s_state#21, g_state#31, agg1#32, agg2#33, agg3#34, agg4#35] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#20, s_state#21, g_state#31, agg1#32, agg2#33, agg3#34, agg4#35] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (35) -+- * ColumnarToRow (34) ++- * CometColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan parquet spark_catalog.default.date_dim (31) @@ -197,7 +197,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt index 121f1be7ce..e65b02d4d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] CometHashAggregate [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price))] @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt index 8f0671c8f0..2e36b987c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt @@ -8,7 +8,7 @@ : : : : : +- Exchange (9) : : : : : +- * HashAggregate (8) : : : : : +- * HashAggregate (7) -: : : : : +- * ColumnarToRow (6) +: : : : : +- * CometColumnarToRow (6) : : : : : +- CometExchange (5) : : : : : +- CometHashAggregate (4) : : : : : +- CometProject (3) @@ -19,7 +19,7 @@ : : : : +- Exchange (19) : : : : +- * HashAggregate (18) : : : : +- * HashAggregate (17) -: : : : +- * ColumnarToRow (16) +: : : : +- * CometColumnarToRow (16) : : : : +- CometExchange (15) : : : : +- CometHashAggregate (14) : : : : +- CometProject (13) @@ -30,7 +30,7 @@ : : : +- Exchange (31) : : : +- * HashAggregate (30) : : : +- * HashAggregate (29) -: : : +- * ColumnarToRow (28) +: : : +- * CometColumnarToRow (28) : : : +- CometExchange (27) : : : +- CometHashAggregate (26) : : : +- CometProject (25) @@ -41,7 +41,7 @@ : : +- Exchange (43) : : +- * HashAggregate (42) : : +- * HashAggregate (41) -: : +- * ColumnarToRow (40) +: : +- * CometColumnarToRow (40) : : +- CometExchange (39) : : +- CometHashAggregate (38) : : +- CometProject (37) @@ -52,7 +52,7 @@ : +- Exchange (55) : +- * HashAggregate (54) : +- * HashAggregate (53) -: +- * ColumnarToRow (52) +: +- * CometColumnarToRow (52) : +- CometExchange (51) : +- CometHashAggregate (50) : +- CometProject (49) @@ -63,7 +63,7 @@ +- Exchange (67) +- * HashAggregate (66) +- * HashAggregate (65) - +- * ColumnarToRow (64) + +- * CometColumnarToRow (64) +- CometExchange (63) +- CometHashAggregate (62) +- CometProject (61) @@ -95,7 +95,7 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_li Input [4]: [ss_list_price#3, sum#6, count#7, count#8] Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(6) ColumnarToRow [codegen id : 1] +(6) CometColumnarToRow [codegen id : 1] Input [4]: [ss_list_price#3, sum#6, count#7, count#8] (7) HashAggregate [codegen id : 1] @@ -147,7 +147,7 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#18)), partial_count(ss_l Input [4]: [ss_list_price#18, sum#21, count#22, count#23] Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(16) ColumnarToRow [codegen id : 2] +(16) CometColumnarToRow [codegen id : 2] Input [4]: [ss_list_price#18, sum#21, count#22, count#23] (17) HashAggregate [codegen id : 2] @@ -207,7 +207,7 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#33)), partial_count(ss_l Input [4]: [ss_list_price#33, sum#36, count#37, count#38] Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(28) ColumnarToRow [codegen id : 4] +(28) CometColumnarToRow [codegen id : 4] Input [4]: [ss_list_price#33, sum#36, count#37, count#38] (29) HashAggregate [codegen id : 4] @@ -267,7 +267,7 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#48)), partial_count(ss_l Input [4]: [ss_list_price#48, sum#51, count#52, count#53] Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] -(40) ColumnarToRow [codegen id : 6] +(40) CometColumnarToRow [codegen id : 6] Input [4]: [ss_list_price#48, sum#51, count#52, count#53] (41) HashAggregate [codegen id : 6] @@ -327,7 +327,7 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#63)), partial_count(ss_l Input [4]: [ss_list_price#63, sum#66, count#67, count#68] Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(52) ColumnarToRow [codegen id : 8] +(52) CometColumnarToRow [codegen id : 8] Input [4]: [ss_list_price#63, sum#66, count#67, count#68] (53) HashAggregate [codegen id : 8] @@ -387,7 +387,7 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#78)), partial_count(ss_l Input [4]: [ss_list_price#78, sum#81, count#82, count#83] Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] -(64) ColumnarToRow [codegen id : 10] +(64) CometColumnarToRow [codegen id : 10] Input [4]: [ss_list_price#78, sum#81, count#82, count#83] (65) HashAggregate [codegen id : 10] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt index 4a547c4e04..04eaf6673f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt @@ -10,7 +10,7 @@ WholeStageCodegen (12) WholeStageCodegen (1) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [ss_list_price] #2 CometHashAggregate [ss_list_price,sum,count,count] @@ -26,7 +26,7 @@ WholeStageCodegen (12) WholeStageCodegen (2) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [ss_list_price] #5 CometHashAggregate [ss_list_price,sum,count,count] @@ -42,7 +42,7 @@ WholeStageCodegen (12) WholeStageCodegen (4) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [ss_list_price] #8 CometHashAggregate [ss_list_price,sum,count,count] @@ -58,7 +58,7 @@ WholeStageCodegen (12) WholeStageCodegen (6) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [ss_list_price] #11 CometHashAggregate [ss_list_price,sum,count,count] @@ -74,7 +74,7 @@ WholeStageCodegen (12) WholeStageCodegen (8) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [ss_list_price] #14 CometHashAggregate [ss_list_price,sum,count,count] @@ -90,7 +90,7 @@ WholeStageCodegen (12) WholeStageCodegen (10) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [ss_list_price] #17 CometHashAggregate [ss_list_price,sum,count,count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt index 71d583785d..b02bf3a4cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (45) +* CometColumnarToRow (45) +- CometTakeOrderedAndProject (44) +- CometHashAggregate (43) +- CometExchange (42) @@ -258,14 +258,14 @@ Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#11), sum(cs_quantity# Input [7]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, store_sales_quantity#36, store_returns_quantity#37, catalog_sales_quantity#38] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#31 ASC NULLS FIRST,i_item_desc#32 ASC NULLS FIRST,s_store_id#28 ASC NULLS FIRST,s_store_name#29 ASC NULLS FIRST], output=[i_item_id#31,i_item_desc#32,s_store_id#28,s_store_name#29,store_sales_quantity#36,store_returns_quantity#37,catalog_sales_quantity#38]), [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, store_sales_quantity#36, store_returns_quantity#37, catalog_sales_quantity#38], 100, [i_item_id#31 ASC NULLS FIRST, i_item_desc#32 ASC NULLS FIRST, s_store_id#28 ASC NULLS FIRST, s_store_name#29 ASC NULLS FIRST], [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, store_sales_quantity#36, store_returns_quantity#37, catalog_sales_quantity#38] -(45) ColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, store_sales_quantity#36, store_returns_quantity#37, catalog_sales_quantity#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (50) -+- * ColumnarToRow (49) ++- * CometColumnarToRow (49) +- CometProject (48) +- CometFilter (47) +- CometScan parquet spark_catalog.default.date_dim (46) @@ -286,7 +286,7 @@ Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9) Input [3]: [d_date_sk#19, d_year#20, d_moy#21] Arguments: [d_date_sk#19], [d_date_sk#19] -(49) ColumnarToRow [codegen id : 1] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] (50) BroadcastExchange @@ -295,7 +295,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (55) -+- * ColumnarToRow (54) ++- * CometColumnarToRow (54) +- CometProject (53) +- CometFilter (52) +- CometScan parquet spark_catalog.default.date_dim (51) @@ -316,7 +316,7 @@ Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= Input [3]: [d_date_sk#22, d_year#23, d_moy#24] Arguments: [d_date_sk#22], [d_date_sk#22] -(54) ColumnarToRow [codegen id : 1] +(54) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] (55) BroadcastExchange @@ -325,7 +325,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 BroadcastExchange (60) -+- * ColumnarToRow (59) ++- * CometColumnarToRow (59) +- CometProject (58) +- CometFilter (57) +- CometScan parquet spark_catalog.default.date_dim (56) @@ -346,7 +346,7 @@ Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(59) ColumnarToRow [codegen id : 1] +(59) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] (60) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt index 9398fcdf5e..9340a4e5f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum,sum,sum,sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity)] @@ -24,7 +24,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -35,7 +35,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -46,7 +46,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt index a0796a47af..e80a2eaa78 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Input [4]: [d_year#2, brand_id#12, brand#13, sum_agg#14] Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,sum_agg#14 DESC NULLS LAST,brand_id#12 ASC NULLS FIRST], output=[d_year#2,brand_id#12,brand#13,sum_agg#14]), [d_year#2, brand_id#12, brand#13, sum_agg#14], 100, [d_year#2 ASC NULLS FIRST, sum_agg#14 DESC NULLS LAST, brand_id#12 ASC NULLS FIRST], [d_year#2, brand_id#12, brand#13, sum_agg#14] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [4]: [d_year#2, brand_id#12, brand#13, sum_agg#14] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt index f82fd24bdd..7bc02651ec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [d_year,brand_id,brand,sum_agg] CometHashAggregate [d_year,brand_id,brand,sum_agg,i_brand,i_brand_id,sum,sum(UnscaledValue(ss_ext_sales_price))] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt index c0b9e17358..9930f6bbde 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (48) +* CometColumnarToRow (48) +- CometTakeOrderedAndProject (47) +- CometProject (46) +- CometBroadcastHashJoin (45) @@ -268,14 +268,14 @@ Arguments: [c_customer_id#21, c_salutation#23, c_first_name#24, c_last_name#25, Input [13]: [c_customer_id#21, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_day#27, c_birth_month#28, c_birth_year#29, c_birth_country#30, c_login#31, c_email_address#32, c_last_review_date#33, ctr_total_return#13] Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#21 ASC NULLS FIRST,c_salutation#23 ASC NULLS FIRST,c_first_name#24 ASC NULLS FIRST,c_last_name#25 ASC NULLS FIRST,c_preferred_cust_flag#26 ASC NULLS FIRST,c_birth_day#27 ASC NULLS FIRST,c_birth_month#28 ASC NULLS FIRST,c_birth_year#29 ASC NULLS FIRST,c_birth_country#30 ASC NULLS FIRST,c_login#31 ASC NULLS FIRST,c_email_address#32 ASC NULLS FIRST,c_last_review_date#33 ASC NULLS FIRST,ctr_total_return#13 ASC NULLS FIRST], output=[c_customer_id#21,c_salutation#23,c_first_name#24,c_last_name#25,c_preferred_cust_flag#26,c_birth_day#27,c_birth_month#28,c_birth_year#29,c_birth_country#30,c_login#31,c_email_address#32,c_last_review_date#33,ctr_total_return#13]), [c_customer_id#21, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_day#27, c_birth_month#28, c_birth_year#29, c_birth_country#30, c_login#31, c_email_address#32, c_last_review_date#33, ctr_total_return#13], 100, [c_customer_id#21 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_last_name#25 ASC NULLS FIRST, c_preferred_cust_flag#26 ASC NULLS FIRST, c_birth_day#27 ASC NULLS FIRST, c_birth_month#28 ASC NULLS FIRST, c_birth_year#29 ASC NULLS FIRST, c_birth_country#30 ASC NULLS FIRST, c_login#31 ASC NULLS FIRST, c_email_address#32 ASC NULLS FIRST, c_last_review_date#33 ASC NULLS FIRST, ctr_total_return#13 ASC NULLS FIRST], [c_customer_id#21, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_day#27, c_birth_month#28, c_birth_year#29, c_birth_country#30, c_login#31, c_email_address#32, c_last_review_date#33, ctr_total_return#13] -(48) ColumnarToRow [codegen id : 1] +(48) CometColumnarToRow [codegen id : 1] Input [13]: [c_customer_id#21, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_day#27, c_birth_month#28, c_birth_year#29, c_birth_country#30, c_login#31, c_email_address#32, c_last_review_date#33, ctr_total_return#13] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (53) -+- * ColumnarToRow (52) ++- * CometColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometScan parquet spark_catalog.default.date_dim (49) @@ -296,7 +296,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(52) ColumnarToRow [codegen id : 1] +(52) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt index 872bf51912..ee501650be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] @@ -21,7 +21,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt index 76a62c8926..6ce03737eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (90) +* CometColumnarToRow (90) +- CometSort (89) +- CometColumnarExchange (88) +- CometProject (87) @@ -504,14 +504,14 @@ Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENT Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9 ASC NULLS FIRST] -(90) ColumnarToRow [codegen id : 1] +(90) CometColumnarToRow [codegen id : 1] Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (94) -+- * ColumnarToRow (93) ++- * CometColumnarToRow (93) +- CometFilter (92) +- CometScan parquet spark_catalog.default.date_dim (91) @@ -527,7 +527,7 @@ ReadSchema: struct Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(93) ColumnarToRow [codegen id : 1] +(93) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] (94) BroadcastExchange @@ -536,7 +536,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (98) -+- * ColumnarToRow (97) ++- * CometColumnarToRow (97) +- CometFilter (96) +- CometScan parquet spark_catalog.default.date_dim (95) @@ -552,7 +552,7 @@ ReadSchema: struct Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) -(97) ColumnarToRow [codegen id : 1] +(97) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] (98) BroadcastExchange @@ -561,7 +561,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:3 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 BroadcastExchange (102) -+- * ColumnarToRow (101) ++- * CometColumnarToRow (101) +- CometFilter (100) +- CometScan parquet spark_catalog.default.date_dim (99) @@ -577,7 +577,7 @@ ReadSchema: struct Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) -(101) ColumnarToRow [codegen id : 1] +(101) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] (102) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt index d6615fe1c1..22635bb9f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] CometColumnarExchange [ca_county] #1 @@ -23,7 +23,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] @@ -46,7 +46,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] @@ -67,7 +67,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #12 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt index 03edc9859f..7caa3c6e24 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (30) +* CometColumnarToRow (30) +- CometHashAggregate (29) +- CometExchange (28) +- CometHashAggregate (27) @@ -168,14 +168,14 @@ Input [1]: [sum#17] Keys: [] Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [1]: [excess discount amount#18] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (35) -+- * ColumnarToRow (34) ++- * CometColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan parquet spark_catalog.default.date_dim (31) @@ -196,7 +196,7 @@ Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#1 Input [2]: [d_date_sk#16, d_date#19] Arguments: [d_date_sk#16], [d_date_sk#16] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt index 14b2a34a89..2650006a6c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [excess discount amount,sum,sum(UnscaledValue(cs_ext_discount_amt))] CometExchange #1 @@ -15,7 +15,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index 7a50fe69e9..587c91520a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (61) +* CometColumnarToRow (61) +- CometTakeOrderedAndProject (60) +- CometHashAggregate (59) +- CometExchange (58) @@ -339,14 +339,14 @@ Functions [1]: [sum(total_sales#36)] Input [2]: [i_manufact_id#12, total_sales#41] Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#41 ASC NULLS FIRST], output=[i_manufact_id#12,total_sales#41]), [i_manufact_id#12, total_sales#41], 100, [total_sales#41 ASC NULLS FIRST], [i_manufact_id#12, total_sales#41] -(61) ColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [codegen id : 1] Input [2]: [i_manufact_id#12, total_sales#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (66) -+- * ColumnarToRow (65) ++- * CometColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometScan parquet spark_catalog.default.date_dim (62) @@ -367,7 +367,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (66) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt index 9556aa785b..50c2d64a92 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_manufact_id,total_sales] CometHashAggregate [i_manufact_id,total_sales,sum,isEmpty,sum(total_sales)] @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt index 50e3800526..452f25394a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (32) +* CometColumnarToRow (32) +- CometSort (31) +- CometColumnarExchange (30) +- CometProject (29) @@ -179,14 +179,14 @@ Arguments: rangepartitioning(c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] Arguments: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (37) -+- * ColumnarToRow (36) ++- * CometColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometScan parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_do Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt index 779d4cbddb..014786e7fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt index c1e19555c9..0730e11302 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt @@ -11,7 +11,7 @@ TakeOrderedAndProject (44) : : +- * Filter (27) : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * ColumnarToRow (12) + : : : :- * CometColumnarToRow (12) : : : : +- CometBroadcastHashJoin (11) : : : : :- CometFilter (2) : : : : : +- CometScan parquet spark_catalog.default.customer (1) @@ -24,23 +24,23 @@ TakeOrderedAndProject (44) : : : : +- CometFilter (5) : : : : +- CometScan parquet spark_catalog.default.date_dim (4) : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) + : : : +- * CometColumnarToRow (17) : : : +- CometProject (16) : : : +- CometBroadcastHashJoin (15) : : : :- CometScan parquet spark_catalog.default.web_sales (13) : : : +- ReusedExchange (14) : : +- BroadcastExchange (25) - : : +- * ColumnarToRow (24) + : : +- * CometColumnarToRow (24) : : +- CometProject (23) : : +- CometBroadcastHashJoin (22) : : :- CometScan parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (32) - : +- * ColumnarToRow (31) + : +- * CometColumnarToRow (31) : +- CometFilter (30) : +- CometScan parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (38) - +- * ColumnarToRow (37) + +- * CometColumnarToRow (37) +- CometFilter (36) +- CometScan parquet spark_catalog.default.customer_demographics (35) @@ -100,7 +100,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) ColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 5] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan parquet spark_catalog.default.web_sales @@ -122,7 +122,7 @@ Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#12] (18) BroadcastExchange @@ -154,7 +154,7 @@ Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [1]: [cs_ship_customer_sk#16] (25) BroadcastExchange @@ -186,7 +186,7 @@ ReadSchema: struct Input [2]: [ca_address_sk#20, ca_state#21] Condition : isnotnull(ca_address_sk#20) -(31) ColumnarToRow [codegen id : 3] +(31) CometColumnarToRow [codegen id : 3] Input [2]: [ca_address_sk#20, ca_state#21] (32) BroadcastExchange @@ -214,7 +214,7 @@ ReadSchema: struct= 2000-02-01)) AND (d_date#1 Input [2]: [d_date_sk#10, d_date#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt index 85e86ca8ae..13fcba8592 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] CometHashAggregate [i_item_id,i_item_desc,i_current_price] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt index c0e7300dff..8139d06c08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (47) +* CometColumnarToRow (47) +- CometHashAggregate (46) +- CometExchange (45) +- CometHashAggregate (44) @@ -261,14 +261,14 @@ Input [1]: [count#26] Keys: [] Functions [1]: [count(1)] -(47) ColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [1]: [count(1)#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 BroadcastExchange (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt index 8272260f5c..5ce896e6e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [count(1),count,count(1)] CometExchange #1 @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt index 32b231d65d..0b404d5868 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (47) +* CometColumnarToRow (47) +- CometSort (46) +- CometColumnarExchange (45) +- CometBroadcastHashJoin (44) @@ -261,14 +261,14 @@ Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC N Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST] -(47) ColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_moy#11] (52) BroadcastExchange @@ -298,7 +298,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 BroadcastExchange (57) -+- * ColumnarToRow (56) ++- * CometColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan parquet spark_catalog.default.date_dim (53) @@ -319,7 +319,7 @@ Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2 Input [3]: [d_date_sk#28, d_year#29, d_moy#30] Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] -(56) ColumnarToRow [codegen id : 1] +(56) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#28, d_moy#30] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt index b789f3a8d2..9df61b1669 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] @@ -52,7 +52,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt index 0239d76784..683ec549cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (47) +* CometColumnarToRow (47) +- CometSort (46) +- CometColumnarExchange (45) +- CometBroadcastHashJoin (44) @@ -261,14 +261,14 @@ Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC N Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST] -(47) ColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_moy#11] (52) BroadcastExchange @@ -298,7 +298,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 BroadcastExchange (57) -+- * ColumnarToRow (56) ++- * CometColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan parquet spark_catalog.default.date_dim (53) @@ -319,7 +319,7 @@ Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2 Input [3]: [d_date_sk#28, d_year#29, d_moy#30] Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] -(56) ColumnarToRow [codegen id : 1] +(56) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#28, d_moy#30] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt index b789f3a8d2..9df61b1669 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] @@ -52,7 +52,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt index 0fe37749f9..457cfd8a39 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (101) +* CometColumnarToRow (101) +- CometTakeOrderedAndProject (100) +- CometProject (99) +- CometBroadcastHashJoin (98) @@ -575,14 +575,14 @@ Arguments: [customer_id#41, customer_first_name#42, customer_last_name#43, custo Input [7]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47] Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#41 ASC NULLS FIRST,customer_first_name#42 ASC NULLS FIRST,customer_last_name#43 ASC NULLS FIRST,customer_preferred_cust_flag#44 ASC NULLS FIRST,customer_birth_country#45 ASC NULLS FIRST,customer_login#46 ASC NULLS FIRST,customer_email_address#47 ASC NULLS FIRST], output=[customer_id#41,customer_first_name#42,customer_last_name#43,customer_preferred_cust_flag#44,customer_birth_country#45,customer_login#46,customer_email_address#47]), [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47], 100, [customer_id#41 ASC NULLS FIRST, customer_first_name#42 ASC NULLS FIRST, customer_last_name#43 ASC NULLS FIRST, customer_preferred_cust_flag#44 ASC NULLS FIRST, customer_birth_country#45 ASC NULLS FIRST, customer_login#46 ASC NULLS FIRST, customer_email_address#47 ASC NULLS FIRST], [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47] -(101) ColumnarToRow [codegen id : 1] +(101) CometColumnarToRow [codegen id : 1] Input [7]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#15 BroadcastExchange (105) -+- * ColumnarToRow (104) ++- * CometColumnarToRow (104) +- CometFilter (103) +- CometScan parquet spark_catalog.default.date_dim (102) @@ -598,7 +598,7 @@ ReadSchema: struct Input [2]: [d_date_sk#16, d_year#17] Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) -(104) ColumnarToRow [codegen id : 1] +(104) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_year#17] (105) BroadcastExchange @@ -607,7 +607,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 BroadcastExchange (109) -+- * ColumnarToRow (108) ++- * CometColumnarToRow (108) +- CometFilter (107) +- CometScan parquet spark_catalog.default.date_dim (106) @@ -623,7 +623,7 @@ ReadSchema: struct Input [2]: [d_date_sk#37, d_year#38] Condition : ((isnotnull(d_year#38) AND (d_year#38 = 2002)) AND isnotnull(d_date_sk#37)) -(108) ColumnarToRow [codegen id : 1] +(108) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#37, d_year#38] (109) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt index d6651fadcf..19f7ccde3a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] CometProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] @@ -27,7 +27,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -50,7 +50,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt index ef66f763d2..10640eb2bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (32) +* CometColumnarToRow (32) +- CometTakeOrderedAndProject (31) +- CometHashAggregate (30) +- CometExchange (29) @@ -179,14 +179,14 @@ Functions [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - Input [4]: [w_state#12, i_item_id#14, sales_before#22, sales_after#23] Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_state#12 ASC NULLS FIRST,i_item_id#14 ASC NULLS FIRST], output=[w_state#12,i_item_id#14,sales_before#22,sales_after#23]), [w_state#12, i_item_id#14, sales_before#22, sales_after#23], 100, [w_state#12 ASC NULLS FIRST, i_item_id#14 ASC NULLS FIRST], [w_state#12, i_item_id#14, sales_before#22, sales_after#23] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [4]: [w_state#12, i_item_id#14, sales_before#22, sales_after#23] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (36) -+- * ColumnarToRow (35) ++- * CometColumnarToRow (35) +- CometFilter (34) +- CometScan parquet spark_catalog.default.date_dim (33) @@ -202,7 +202,7 @@ ReadSchema: struct Input [2]: [d_date_sk#16, d_date#17] Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2000-02-10)) AND (d_date#17 <= 2000-04-10)) AND isnotnull(d_date_sk#16)) -(35) ColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_date#17] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt index eac6138196..6b87d9b8b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] CometHashAggregate [w_state,i_item_id,sales_before,sales_after,sum,isEmpty,sum,isEmpty,sum(CASE WHEN (d_date < 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END)] @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt index 48a1c60d16..bb3c74daeb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -103,6 +103,6 @@ Functions: [] Input [1]: [i_product_name#3] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#3 ASC NULLS FIRST], output=[i_product_name#3]), [i_product_name#3], 100, [i_product_name#3 ASC NULLS FIRST], [i_product_name#3] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [1]: [i_product_name#3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt index 007103499b..de12ce9581 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_product_name] CometHashAggregate [i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt index b58d82c785..0ad962d86a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Input [4]: [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#12] Arguments: TakeOrderedAndProject(limit=100, orderBy=[sum(ss_ext_sales_price)#12 DESC NULLS LAST,d_year#2 ASC NULLS FIRST,i_category_id#8 ASC NULLS FIRST,i_category#9 ASC NULLS FIRST], output=[d_year#2,i_category_id#8,i_category#9,sum(ss_ext_sales_price)#12]), [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#12], 100, [sum(ss_ext_sales_price)#12 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#9 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#12] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [4]: [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#12] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt index 40528f52d3..7307450677 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [d_year,i_category_id,i_category,sum(ss_ext_sales_price)] CometHashAggregate [d_year,i_category_id,i_category,sum(ss_ext_sales_price),sum,sum(UnscaledValue(ss_ext_sales_price))] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt index 2953a5f8f9..10dfceddc2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_s Input [9]: [s_store_name#9, s_store_id#8, sun_sales#18, mon_sales#19, tue_sales#20, wed_sales#21, thu_sales#22, fri_sales#23, sat_sales#24] Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#9 ASC NULLS FIRST,s_store_id#8 ASC NULLS FIRST,sun_sales#18 ASC NULLS FIRST,mon_sales#19 ASC NULLS FIRST,tue_sales#20 ASC NULLS FIRST,wed_sales#21 ASC NULLS FIRST,thu_sales#22 ASC NULLS FIRST,fri_sales#23 ASC NULLS FIRST,sat_sales#24 ASC NULLS FIRST], output=[s_store_name#9,s_store_id#8,sun_sales#18,mon_sales#19,tue_sales#20,wed_sales#21,thu_sales#22,fri_sales#23,sat_sales#24]), [s_store_name#9, s_store_id#8, sun_sales#18, mon_sales#19, tue_sales#20, wed_sales#21, thu_sales#22, fri_sales#23, sat_sales#24], 100, [s_store_name#9 ASC NULLS FIRST, s_store_id#8 ASC NULLS FIRST, sun_sales#18 ASC NULLS FIRST, mon_sales#19 ASC NULLS FIRST, tue_sales#20 ASC NULLS FIRST, wed_sales#21 ASC NULLS FIRST, thu_sales#22 ASC NULLS FIRST, fri_sales#23 ASC NULLS FIRST, sat_sales#24 ASC NULLS FIRST], [s_store_name#9, s_store_id#8, sun_sales#18, mon_sales#19, tue_sales#20, wed_sales#21, thu_sales#22, fri_sales#23, sat_sales#24] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [9]: [s_store_name#9, s_store_id#8, sun_sales#18, mon_sales#19, tue_sales#20, wed_sales#21, thu_sales#22, fri_sales#23, sat_sales#24] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt index 447131b776..08b394e62e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] CometHashAggregate [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt index 6e85991e4a..956ca43dbf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject (33) : : : +- * Project (13) : : : +- * Filter (12) : : : +- Window (11) - : : : +- * ColumnarToRow (10) + : : : +- * CometColumnarToRow (10) : : : +- CometSort (9) : : : +- CometExchange (8) : : : +- CometFilter (7) @@ -24,11 +24,11 @@ TakeOrderedAndProject (33) : : +- * Project (20) : : +- * Filter (19) : : +- Window (18) - : : +- * ColumnarToRow (17) + : : +- * CometColumnarToRow (17) : : +- CometSort (16) : : +- ReusedExchange (15) : +- BroadcastExchange (27) - : +- * ColumnarToRow (26) + : +- * CometColumnarToRow (26) : +- CometFilter (25) : +- CometScan parquet spark_catalog.default.item (24) +- ReusedExchange (30) @@ -75,7 +75,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [2]: [item_sk#7, rank_col#8] Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] -(10) ColumnarToRow [codegen id : 1] +(10) CometColumnarToRow [codegen id : 1] Input [2]: [item_sk#7, rank_col#8] (11) Window @@ -101,7 +101,7 @@ Output [2]: [item_sk#12, rank_col#13] Input [2]: [item_sk#12, rank_col#13] Arguments: [item_sk#12, rank_col#13], [rank_col#13 DESC NULLS LAST] -(17) ColumnarToRow [codegen id : 3] +(17) CometColumnarToRow [codegen id : 3] Input [2]: [item_sk#12, rank_col#13] (18) Window @@ -141,7 +141,7 @@ ReadSchema: struct Input [2]: [i_item_sk#15, i_product_name#16] Condition : isnotnull(i_item_sk#15) -(26) ColumnarToRow [codegen id : 5] +(26) CometColumnarToRow [codegen id : 5] Input [2]: [i_item_sk#15, i_product_name#16] (27) BroadcastExchange @@ -178,7 +178,7 @@ Arguments: 100, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#19, worst_per ===== Subqueries ===== Subquery:1 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* ColumnarToRow (40) +* CometColumnarToRow (40) +- CometHashAggregate (39) +- CometExchange (38) +- CometHashAggregate (37) @@ -216,7 +216,7 @@ Input [3]: [ss_store_sk#22, sum#25, count#26] Keys [1]: [ss_store_sk#22] Functions [1]: [avg(UnscaledValue(ss_net_profit#23))] -(40) ColumnarToRow [codegen id : 1] +(40) CometColumnarToRow [codegen id : 1] Input [1]: [rank_col#27] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt index 6ae18d8792..3230b1456a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt @@ -14,14 +14,14 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] InputAdapter Window [rank_col] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item_sk,rank_col] CometExchange #1 CometFilter [item_sk,rank_col] Subquery #1 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [rank_col,ss_store_sk,sum,count,avg(UnscaledValue(ss_net_profit))] CometExchange [ss_store_sk] #3 @@ -43,14 +43,14 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] InputAdapter Window [rank_col] WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item_sk,rank_col] ReusedExchange [item_sk,rank_col] #1 InputAdapter BroadcastExchange #4 WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [i_item_sk,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt index 962a51203b..d9213e07f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject (36) +- * Project (32) +- * Filter (31) +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (30) - :- * ColumnarToRow (24) + :- * CometColumnarToRow (24) : +- CometProject (23) : +- CometBroadcastHashJoin (22) : :- CometProject (18) @@ -31,7 +31,7 @@ TakeOrderedAndProject (36) : +- CometFilter (20) : +- CometScan parquet spark_catalog.default.item (19) +- BroadcastExchange (29) - +- * ColumnarToRow (28) + +- * CometColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan parquet spark_catalog.default.item (25) @@ -149,7 +149,7 @@ Arguments: [ws_item_sk#2], [i_item_sk#15], Inner, BuildRight Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#11, i_item_sk#15, i_item_id#16] Arguments: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16], [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [4]: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16] (25) CometScan parquet spark_catalog.default.item @@ -167,7 +167,7 @@ Condition : i_item_sk#17 IN (2,3,5,7,11,13,17,19,23,29) Input [2]: [i_item_sk#17, i_item_id#18] Arguments: [i_item_id#18], [i_item_id#18] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [i_item_id#18] (29) BroadcastExchange @@ -214,7 +214,7 @@ Arguments: 100, [ca_zip#11 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (41) -+- * ColumnarToRow (40) ++- * CometColumnarToRow (40) +- CometProject (39) +- CometFilter (38) +- CometScan parquet spark_catalog.default.date_dim (37) @@ -235,7 +235,7 @@ Condition : ((((isnotnull(d_qoy#14) AND isnotnull(d_year#13)) AND (d_qoy#14 = 2) Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] Arguments: [d_date_sk#12], [d_date_sk#12] -(40) ColumnarToRow [codegen id : 1] +(40) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#12] (41) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt index f60fdb18a1..a325fb95d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] Project [ws_sales_price,ca_city,ca_zip] Filter [ca_zip,exists] BroadcastHashJoin [i_item_id,i_item_id] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ca_city,ca_zip,i_item_sk,i_item_id] @@ -23,7 +23,7 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] @@ -44,7 +44,7 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] InputAdapter BroadcastExchange #7 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [i_item_id] CometFilter [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt index 7bc10ce88e..406d3a6730 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (38) +* CometColumnarToRow (38) +- CometTakeOrderedAndProject (37) +- CometProject (36) +- CometBroadcastHashJoin (35) @@ -213,14 +213,14 @@ Arguments: [c_last_name#25, c_first_name#24, ca_city#30, bought_city#26, ss_tick Input [7]: [c_last_name#25, c_first_name#24, ca_city#30, bought_city#26, ss_ticket_number#5, amt#27, profit#28] Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#25 ASC NULLS FIRST,c_first_name#24 ASC NULLS FIRST,ca_city#30 ASC NULLS FIRST,bought_city#26 ASC NULLS FIRST,ss_ticket_number#5 ASC NULLS FIRST], output=[c_last_name#25,c_first_name#24,ca_city#30,bought_city#26,ss_ticket_number#5,amt#27,profit#28]), [c_last_name#25, c_first_name#24, ca_city#30, bought_city#26, ss_ticket_number#5, amt#27, profit#28], 100, [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, ca_city#30 ASC NULLS FIRST, bought_city#26 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#25, c_first_name#24, ca_city#30, bought_city#26, ss_ticket_number#5, amt#27, profit#28] -(38) ColumnarToRow [codegen id : 1] +(38) CometColumnarToRow [codegen id : 1] Input [7]: [c_last_name#25, c_first_name#24, ca_city#30, bought_city#26, ss_ticket_number#5, amt#27, profit#28] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (43) -+- * ColumnarToRow (42) ++- * CometColumnarToRow (42) +- CometProject (41) +- CometFilter (40) +- CometScan parquet spark_catalog.default.date_dim (39) @@ -241,7 +241,7 @@ Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (43) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt index e1b53bf4d0..25e38a548e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] CometProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dow] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt index 32126320ff..e39a63cc69 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject (47) : : +- Window (26) : : +- * Filter (25) : : +- Window (24) - : : +- * ColumnarToRow (23) + : : +- * CometColumnarToRow (23) : : +- CometSort (22) : : +- CometExchange (21) : : +- CometHashAggregate (20) @@ -35,7 +35,7 @@ TakeOrderedAndProject (47) : +- BroadcastExchange (36) : +- * Project (35) : +- Window (34) - : +- * ColumnarToRow (33) + : +- * CometColumnarToRow (33) : +- CometSort (32) : +- CometExchange (31) : +- CometHashAggregate (30) @@ -43,7 +43,7 @@ TakeOrderedAndProject (47) +- BroadcastExchange (44) +- * Project (43) +- Window (42) - +- * ColumnarToRow (41) + +- * CometColumnarToRow (41) +- CometSort (40) +- ReusedExchange (39) @@ -154,7 +154,7 @@ Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_ Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17] Arguments: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17], [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarToRow [codegen id : 1] Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17] (24) Window @@ -193,7 +193,7 @@ Arguments: hashpartitioning(i_category#20, i_brand#21, s_store_name#22, s_compan Input [7]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16] Arguments: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16], [i_category#20 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(33) ColumnarToRow [codegen id : 3] +(33) CometColumnarToRow [codegen id : 3] Input [7]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16] (34) Window @@ -225,7 +225,7 @@ Output [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_ye Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#16] Arguments: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#16], [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] -(41) ColumnarToRow [codegen id : 5] +(41) CometColumnarToRow [codegen id : 5] Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#16] (42) Window @@ -258,7 +258,7 @@ Arguments: 100, [(sum_sales#16 - avg_monthly_sales#19) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (51) -+- * ColumnarToRow (50) ++- * CometColumnarToRow (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -274,7 +274,7 @@ ReadSchema: struct Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (51) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt index cf4556774d..26c6cd226a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 @@ -34,7 +34,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] @@ -51,7 +51,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 @@ -64,7 +64,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt index 70c7dc75f5..451f4d9646 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (28) +* CometColumnarToRow (28) +- CometHashAggregate (27) +- CometExchange (26) +- CometHashAggregate (25) @@ -159,14 +159,14 @@ Input [1]: [sum#18] Keys: [] Functions [1]: [sum(ss_quantity#4)] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [sum(ss_quantity)#19] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (33) -+- * ColumnarToRow (32) ++- * CometColumnarToRow (32) +- CometProject (31) +- CometFilter (30) +- CometScan parquet spark_catalog.default.date_dim (29) @@ -187,7 +187,7 @@ Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#16, d_year#17] Arguments: [d_date_sk#16], [d_date_sk#16] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] (33) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt index a46d8d6037..60c611beca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [sum(ss_quantity),sum,sum(ss_quantity)] CometExchange #1 @@ -17,7 +17,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt index 2def4544c3..c1371c0e8a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject (77) : +- Window (24) : +- * Sort (23) : +- Window (22) - : +- * ColumnarToRow (21) + : +- * CometColumnarToRow (21) : +- CometSort (20) : +- CometExchange (19) : +- CometHashAggregate (18) @@ -35,7 +35,7 @@ TakeOrderedAndProject (77) : +- Window (47) : +- * Sort (46) : +- Window (45) - : +- * ColumnarToRow (44) + : +- * CometColumnarToRow (44) : +- CometSort (43) : +- CometExchange (42) : +- CometHashAggregate (41) @@ -58,7 +58,7 @@ TakeOrderedAndProject (77) +- Window (70) +- * Sort (69) +- Window (68) - +- * ColumnarToRow (67) + +- * CometColumnarToRow (67) +- CometSort (66) +- CometExchange (65) +- CometHashAggregate (64) @@ -172,7 +172,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [3]: [item#22, return_ratio#23, currency_ratio#24] Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) ColumnarToRow [codegen id : 1] +(21) CometColumnarToRow [codegen id : 1] Input [3]: [item#22, return_ratio#23, currency_ratio#24] (22) Window @@ -273,7 +273,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] -(44) ColumnarToRow [codegen id : 4] +(44) CometColumnarToRow [codegen id : 4] Input [3]: [item#47, return_ratio#48, currency_ratio#49] (45) Window @@ -374,7 +374,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] -(67) ColumnarToRow [codegen id : 7] +(67) CometColumnarToRow [codegen id : 7] Input [3]: [item#72, return_ratio#73, currency_ratio#74] (68) Window @@ -425,7 +425,7 @@ Arguments: 100, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, cur Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (82) -+- * ColumnarToRow (81) ++- * CometColumnarToRow (81) +- CometProject (80) +- CometFilter (79) +- CometScan parquet spark_catalog.default.date_dim (78) @@ -446,7 +446,7 @@ Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2 Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(81) ColumnarToRow [codegen id : 1] +(81) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] (82) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt index d98b2b0a66..aaf75c375c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] InputAdapter Window [return_ratio] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item,return_ratio,currency_ratio] CometExchange #2 @@ -35,7 +35,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -57,7 +57,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] InputAdapter Window [return_ratio] WholeStageCodegen (4) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item,return_ratio,currency_ratio] CometExchange #7 @@ -87,7 +87,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] InputAdapter Window [return_ratio] WholeStageCodegen (7) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item,return_ratio,currency_ratio] CometExchange #10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt index 02ad2f3575..1b4ffcc441 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject (70) +- * Expand (66) +- Union (65) :- * HashAggregate (22) - : +- * ColumnarToRow (21) + : +- * CometColumnarToRow (21) : +- CometExchange (20) : +- CometHashAggregate (19) : +- CometProject (18) @@ -28,7 +28,7 @@ TakeOrderedAndProject (70) : +- CometFilter (15) : +- CometScan parquet spark_catalog.default.store (14) :- * HashAggregate (41) - : +- * ColumnarToRow (40) + : +- * CometColumnarToRow (40) : +- CometExchange (39) : +- CometHashAggregate (38) : +- CometProject (37) @@ -47,7 +47,7 @@ TakeOrderedAndProject (70) : +- CometFilter (34) : +- CometScan parquet spark_catalog.default.catalog_page (33) +- * HashAggregate (64) - +- * ColumnarToRow (63) + +- * CometColumnarToRow (63) +- CometExchange (62) +- CometHashAggregate (61) +- CometProject (60) @@ -168,7 +168,7 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(21) ColumnarToRow [codegen id : 1] +(21) CometColumnarToRow [codegen id : 1] Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] (22) HashAggregate [codegen id : 1] @@ -259,7 +259,7 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#46)), partial_sum(Unscaled Input [5]: [cp_catalog_page_id#62, sum#63, sum#64, sum#65, sum#66] Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(40) ColumnarToRow [codegen id : 2] +(40) CometColumnarToRow [codegen id : 2] Input [5]: [cp_catalog_page_id#62, sum#63, sum#64, sum#65, sum#66] (41) HashAggregate [codegen id : 2] @@ -369,7 +369,7 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#83)), partial_sum(Unscaled Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(63) ColumnarToRow [codegen id : 3] +(63) CometColumnarToRow [codegen id : 3] Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] (64) HashAggregate [codegen id : 3] @@ -411,7 +411,7 @@ Arguments: 100, [channel#118 ASC NULLS FIRST, id#119 ASC NULLS FIRST], [channel# Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (75) -+- * ColumnarToRow (74) ++- * CometColumnarToRow (74) +- CometProject (73) +- CometFilter (72) +- CometScan parquet spark_catalog.default.date_dim (71) @@ -432,7 +432,7 @@ Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#2 Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(74) ColumnarToRow [codegen id : 1] +(74) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] (75) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt index c6f9f60af4..30bdf3edea 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Union WholeStageCodegen (1) HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [s_store_id] #2 CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] @@ -25,7 +25,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] @@ -43,7 +43,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (2) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [cp_catalog_page_id] #6 CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] @@ -66,7 +66,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (3) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [web_site_id] #8 CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt index 9b1efc7ada..ded5068bb0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (28) +* CometColumnarToRow (28) +- CometTakeOrderedAndProject (27) +- CometHashAggregate (26) +- CometExchange (25) @@ -160,14 +160,14 @@ Functions [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30 Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#12 ASC NULLS FIRST,s_company_id#13 ASC NULLS FIRST,s_street_number#14 ASC NULLS FIRST,s_street_name#15 ASC NULLS FIRST,s_street_type#16 ASC NULLS FIRST,s_suite_number#17 ASC NULLS FIRST,s_city#18 ASC NULLS FIRST,s_county#19 ASC NULLS FIRST,s_state#20 ASC NULLS FIRST,s_zip#21 ASC NULLS FIRST], output=[s_store_name#12,s_company_id#13,s_street_number#14,s_street_name#15,s_street_type#16,s_suite_number#17,s_city#18,s_county#19,s_state#20,s_zip#21,30 days #31,31 - 60 days #32,61 - 90 days #33,91 - 120 days #34,>120 days #35]), [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35], 100, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRST, s_street_number#14 ASC NULLS FIRST, s_street_name#15 ASC NULLS FIRST, s_street_type#16 ASC NULLS FIRST, s_suite_number#17 ASC NULLS FIRST, s_city#18 ASC NULLS FIRST, s_county#19 ASC NULLS FIRST, s_state#20 ASC NULLS FIRST, s_zip#21 ASC NULLS FIRST], [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 BroadcastExchange (33) -+- * ColumnarToRow (32) ++- * CometColumnarToRow (32) +- CometProject (31) +- CometFilter (30) +- CometScan parquet spark_catalog.default.date_dim (29) @@ -188,7 +188,7 @@ Condition : ((((isnotnull(d_year#24) AND isnotnull(d_moy#25)) AND (d_year#24 = 2 Input [3]: [d_date_sk#23, d_year#24, d_moy#25] Arguments: [d_date_sk#23], [d_date_sk#23] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#23] (33) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt index 5333344640..c0a0b6d39c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum,sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 30) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 60) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 90) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) > 120) THEN 1 ELSE 0 END)] @@ -21,7 +21,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt index 8779bd70e7..36d0ef2338 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject (40) : +- Exchange (17) : +- * Project (16) : +- Window (15) - : +- * ColumnarToRow (14) + : +- * CometColumnarToRow (14) : +- CometSort (13) : +- CometExchange (12) : +- CometHashAggregate (11) @@ -28,7 +28,7 @@ TakeOrderedAndProject (40) +- Exchange (32) +- * Project (31) +- Window (30) - +- * ColumnarToRow (29) + +- * CometColumnarToRow (29) +- CometSort (28) +- CometExchange (27) +- CometHashAggregate (26) @@ -103,7 +103,7 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeSh Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] Arguments: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] (15) Window @@ -168,7 +168,7 @@ Arguments: hashpartitioning(ss_item_sk#12, 5), ENSURE_REQUIREMENTS, CometNativeS Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] Arguments: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12], [ss_item_sk#12 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] -(29) ColumnarToRow [codegen id : 4] +(29) CometColumnarToRow [codegen id : 4] Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] (30) Window @@ -221,7 +221,7 @@ Arguments: 100, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_s Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (45) -+- * ColumnarToRow (44) ++- * CometColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan parquet spark_catalog.default.date_dim (41) @@ -242,7 +242,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_mont Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(44) ColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt index f972c31c17..bddc3a125a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store InputAdapter Window [_w0,ws_item_sk,d_date] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item_sk,d_date,_w0,ws_item_sk] CometExchange [ws_item_sk] #3 @@ -34,7 +34,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] @@ -53,7 +53,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store InputAdapter Window [_w0,ss_item_sk,d_date] WholeStageCodegen (4) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item_sk,d_date,_w0,ss_item_sk] CometExchange [ss_item_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt index 42b974e53f..c7a400159f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Input [4]: [d_year#2, brand_id#12, brand#13, ext_price#14] Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,ext_price#14 DESC NULLS LAST,brand_id#12 ASC NULLS FIRST], output=[d_year#2,brand_id#12,brand#13,ext_price#14]), [d_year#2, brand_id#12, brand#13, ext_price#14], 100, [d_year#2 ASC NULLS FIRST, ext_price#14 DESC NULLS LAST, brand_id#12 ASC NULLS FIRST], [d_year#2, brand_id#12, brand#13, ext_price#14] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [4]: [d_year#2, brand_id#12, brand#13, ext_price#14] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt index af5223b69e..1e7168862a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [d_year,brand_id,brand,ext_price] CometHashAggregate [d_year,brand_id,brand,ext_price,i_brand,i_brand_id,sum,sum(UnscaledValue(ss_ext_sales_price))] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt index 9b71fa4009..c1448cf812 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (29) +- * Project (28) +- * Filter (27) +- Window (26) - +- * ColumnarToRow (25) + +- * CometColumnarToRow (25) +- CometSort (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -144,7 +144,7 @@ Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometNativ Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] Arguments: [i_manufact_id#5, sum_sales#20, _w0#21], [i_manufact_id#5 ASC NULLS FIRST] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] (26) Window @@ -167,7 +167,7 @@ Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (34) -+- * ColumnarToRow (33) ++- * CometColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] -(33) ColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#15, d_qoy#17] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt index 363b84b354..96a2eec50e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] InputAdapter Window [_w0,i_manufact_id] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_manufact_id,sum_sales,_w0] CometExchange [i_manufact_id] #1 @@ -27,7 +27,7 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_qoy] CometFilter [d_date_sk,d_month_seq,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt index 73422b2923..72ba214250 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (56) +* CometColumnarToRow (56) +- CometTakeOrderedAndProject (55) +- CometHashAggregate (54) +- CometExchange (53) @@ -320,14 +320,14 @@ Functions [1]: [count(1)] Input [3]: [segment#38, num_customers#40, segment_base#41] Arguments: TakeOrderedAndProject(limit=100, orderBy=[segment#38 ASC NULLS FIRST,num_customers#40 ASC NULLS FIRST], output=[segment#38,num_customers#40,segment_base#41]), [segment#38, num_customers#40, segment_base#41], 100, [segment#38 ASC NULLS FIRST, num_customers#40 ASC NULLS FIRST], [segment#38, num_customers#40, segment_base#41] -(56) ColumnarToRow [codegen id : 1] +(56) CometColumnarToRow [codegen id : 1] Input [3]: [segment#38, num_customers#40, segment_base#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (61) -+- * ColumnarToRow (60) ++- * CometColumnarToRow (60) +- CometProject (59) +- CometFilter (58) +- CometScan parquet spark_catalog.default.date_dim (57) @@ -348,7 +348,7 @@ Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12 Input [3]: [d_date_sk#17, d_year#18, d_moy#19] Arguments: [d_date_sk#17], [d_date_sk#17] -(60) ColumnarToRow [codegen id : 1] +(60) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#17] (61) BroadcastExchange @@ -359,7 +359,7 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dy Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 BroadcastExchange (66) -+- * ColumnarToRow (65) ++- * CometColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometScan parquet spark_catalog.default.date_dim (62) @@ -380,7 +380,7 @@ Condition : (((isnotnull(d_month_seq#32) AND (d_month_seq#32 >= Subquery scalar- Input [2]: [d_date_sk#31, d_month_seq#32] Arguments: [d_date_sk#31], [d_date_sk#31] -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#31] (66) BroadcastExchange @@ -388,7 +388,7 @@ Input [1]: [d_date_sk#31] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:4 Hosting operator id = 63 Hosting Expression = Subquery scalar-subquery#33, [id=#34] -* ColumnarToRow (73) +* CometColumnarToRow (73) +- CometHashAggregate (72) +- CometExchange (71) +- CometHashAggregate (70) @@ -426,11 +426,11 @@ Input [1]: [(d_month_seq + 1)#45] Keys [1]: [(d_month_seq + 1)#45] Functions: [] -(73) ColumnarToRow [codegen id : 1] +(73) CometColumnarToRow [codegen id : 1] Input [1]: [(d_month_seq + 1)#45] Subquery:5 Hosting operator id = 63 Hosting Expression = Subquery scalar-subquery#35, [id=#36] -* ColumnarToRow (80) +* CometColumnarToRow (80) +- CometHashAggregate (79) +- CometExchange (78) +- CometHashAggregate (77) @@ -468,7 +468,7 @@ Input [1]: [(d_month_seq + 3)#49] Keys [1]: [(d_month_seq + 3)#49] Functions: [] -(80) ColumnarToRow [codegen id : 1] +(80) CometColumnarToRow [codegen id : 1] Input [1]: [(d_month_seq + 3)#49] Subquery:6 Hosting operator id = 44 Hosting Expression = ReusedSubquery Subquery scalar-subquery#33, [id=#34] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt index 1290186685..e41bbe85b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [segment,num_customers,segment_base] CometHashAggregate [segment,num_customers,segment_base,count,count(1)] @@ -32,7 +32,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -58,13 +58,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] Subquery #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [(d_month_seq + 1)] CometExchange [(d_month_seq + 1)] #10 @@ -74,7 +74,7 @@ WholeStageCodegen (1) CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] Subquery #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [(d_month_seq + 3)] CometExchange [(d_month_seq + 3)] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt index 46240a3c02..4549080d86 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Input [3]: [brand_id#12, brand#13, ext_price#14] Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#14 DESC NULLS LAST,brand_id#12 ASC NULLS FIRST], output=[brand_id#12,brand#13,ext_price#14]), [brand_id#12, brand#13, ext_price#14], 100, [ext_price#14 DESC NULLS LAST, brand_id#12 ASC NULLS FIRST], [brand_id#12, brand#13, ext_price#14] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [3]: [brand_id#12, brand#13, ext_price#14] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt index 999c8a6c4c..5a5f7ee21e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [brand_id,brand,ext_price] CometHashAggregate [brand_id,brand,ext_price,i_brand,i_brand_id,sum,sum(UnscaledValue(ss_ext_sales_price))] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt index c56eb8287f..e3977ede3f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (61) +* CometColumnarToRow (61) +- CometTakeOrderedAndProject (60) +- CometHashAggregate (59) +- CometExchange (58) @@ -339,14 +339,14 @@ Functions [1]: [sum(total_sales#36)] Input [2]: [i_item_id#12, total_sales#41] Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#41 ASC NULLS FIRST], output=[i_item_id#12,total_sales#41]), [i_item_id#12, total_sales#41], 100, [total_sales#41 ASC NULLS FIRST], [i_item_id#12, total_sales#41] -(61) ColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [codegen id : 1] Input [2]: [i_item_id#12, total_sales#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (66) -+- * ColumnarToRow (65) ++- * CometColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometScan parquet spark_catalog.default.date_dim (62) @@ -367,7 +367,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (66) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt index 2603a2781f..95ff3d709a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,total_sales] CometHashAggregate [i_item_id,total_sales,sum,isEmpty,sum(total_sales)] @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt index 78df07e8fb..8f45480fc8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject (47) : : +- Window (26) : : +- * Filter (25) : : +- Window (24) - : : +- * ColumnarToRow (23) + : : +- * CometColumnarToRow (23) : : +- CometSort (22) : : +- CometExchange (21) : : +- CometHashAggregate (20) @@ -35,7 +35,7 @@ TakeOrderedAndProject (47) : +- BroadcastExchange (36) : +- * Project (35) : +- Window (34) - : +- * ColumnarToRow (33) + : +- * CometColumnarToRow (33) : +- CometSort (32) : +- CometExchange (31) : +- CometHashAggregate (30) @@ -43,7 +43,7 @@ TakeOrderedAndProject (47) +- BroadcastExchange (44) +- * Project (43) +- Window (42) - +- * ColumnarToRow (41) + +- * CometColumnarToRow (41) +- CometSort (40) +- ReusedExchange (39) @@ -154,7 +154,7 @@ Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQU Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16] Arguments: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16], [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarToRow [codegen id : 1] Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16] (24) Window @@ -193,7 +193,7 @@ Arguments: hashpartitioning(i_category#19, i_brand#20, cc_name#21, 5), ENSURE_RE Input [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] Arguments: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15], [i_category#19 ASC NULLS FIRST, i_brand#20 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] -(33) ColumnarToRow [codegen id : 3] +(33) CometColumnarToRow [codegen id : 3] Input [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] (34) Window @@ -225,7 +225,7 @@ Output [6]: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sal Input [6]: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sales#15] Arguments: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sales#15], [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] -(41) ColumnarToRow [codegen id : 5] +(41) CometColumnarToRow [codegen id : 5] Input [6]: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sales#15] (42) Window @@ -258,7 +258,7 @@ Arguments: 100, [(sum_sales#15 - avg_monthly_sales#18) ASC NULLS FIRST, cc_name# Subquery:1 Hosting operator id = 3 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (51) -+- * ColumnarToRow (50) ++- * CometColumnarToRow (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -274,7 +274,7 @@ ReadSchema: struct Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (51) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt index 39a7ce9d67..10cd515da7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] CometExchange [i_category,i_brand,cc_name] #1 @@ -34,7 +34,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] @@ -51,7 +51,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] CometExchange [i_category,i_brand,cc_name] #8 @@ -64,7 +64,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index cd1e112f69..18736927ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (51) +* CometColumnarToRow (51) +- CometTakeOrderedAndProject (50) +- CometProject (49) +- CometBroadcastHashJoin (48) @@ -280,14 +280,14 @@ Arguments: [item_id#11, ss_item_rev#12, ss_dev#33, cs_item_rev#22, cs_dev#34, ws Input [8]: [item_id#11, ss_item_rev#12, ss_dev#33, cs_item_rev#22, cs_dev#34, ws_item_rev#32, ws_dev#35, average#36] Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_id#11 ASC NULLS FIRST,ss_item_rev#12 ASC NULLS FIRST], output=[item_id#11,ss_item_rev#12,ss_dev#33,cs_item_rev#22,cs_dev#34,ws_item_rev#32,ws_dev#35,average#36]), [item_id#11, ss_item_rev#12, ss_dev#33, cs_item_rev#22, cs_dev#34, ws_item_rev#32, ws_dev#35, average#36], 100, [item_id#11 ASC NULLS FIRST, ss_item_rev#12 ASC NULLS FIRST], [item_id#11, ss_item_rev#12, ss_dev#33, cs_item_rev#22, cs_dev#34, ws_item_rev#32, ws_dev#35, average#36] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [8]: [item_id#11, ss_item_rev#12, ss_dev#33, cs_item_rev#22, cs_dev#34, ws_item_rev#32, ws_dev#35, average#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (61) -+- * ColumnarToRow (60) ++- * CometColumnarToRow (60) +- CometProject (59) +- CometBroadcastHashJoin (58) :- CometFilter (53) @@ -337,7 +337,7 @@ Arguments: [d_date#8], [d_date#9], LeftSemi, BuildRight Input [2]: [d_date_sk#7, d_date#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(60) ColumnarToRow [codegen id : 1] +(60) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (61) BroadcastExchange @@ -345,7 +345,7 @@ Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] Subquery:2 Hosting operator id = 55 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* ColumnarToRow (65) +* CometColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometScan parquet spark_catalog.default.date_dim (62) @@ -366,7 +366,7 @@ Condition : (isnotnull(d_date#40) AND (d_date#40 = 2000-01-03)) Input [2]: [d_date#40, d_week_seq#41] Arguments: [d_week_seq#41], [d_week_seq#41] -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#41] Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt index 36fc8ac573..db48fd9775 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] CometProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometBroadcastHashJoin [d_date_sk,d_date,d_date] @@ -30,7 +30,7 @@ WholeStageCodegen (1) CometFilter [d_date,d_week_seq] Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_date,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt index acdbfd1716..1091631a21 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (39) +* CometColumnarToRow (39) +- CometTakeOrderedAndProject (38) +- CometProject (37) +- CometBroadcastHashJoin (36) @@ -219,6 +219,6 @@ Arguments: [s_store_name1#26, s_store_id1#28, d_week_seq1#27, (sun_sales1 / sun_ Input [10]: [s_store_name1#26, s_store_id1#28, d_week_seq1#27, (sun_sales1 / sun_sales2)#56, (mon_sales1 / mon_sales2)#57, (tue_sales1 / tue_sales2)#58, (wed_sales1 / wed_sales2)#59, (thu_sales1 / thu_sales2)#60, (fri_sales1 / fri_sales2)#61, (sat_sales1 / sat_sales2)#62] Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name1#26 ASC NULLS FIRST,s_store_id1#28 ASC NULLS FIRST,d_week_seq1#27 ASC NULLS FIRST], output=[s_store_name1#26,s_store_id1#28,d_week_seq1#27,(sun_sales1 / sun_sales2)#56,(mon_sales1 / mon_sales2)#57,(tue_sales1 / tue_sales2)#58,(wed_sales1 / wed_sales2)#59,(thu_sales1 / thu_sales2)#60,(fri_sales1 / fri_sales2)#61,(sat_sales1 / sat_sales2)#62]), [s_store_name1#26, s_store_id1#28, d_week_seq1#27, (sun_sales1 / sun_sales2)#56, (mon_sales1 / mon_sales2)#57, (tue_sales1 / tue_sales2)#58, (wed_sales1 / wed_sales2)#59, (thu_sales1 / thu_sales2)#60, (fri_sales1 / fri_sales2)#61, (sat_sales1 / sat_sales2)#62], 100, [s_store_name1#26 ASC NULLS FIRST, s_store_id1#28 ASC NULLS FIRST, d_week_seq1#27 ASC NULLS FIRST], [s_store_name1#26, s_store_id1#28, d_week_seq1#27, (sun_sales1 / sun_sales2)#56, (mon_sales1 / mon_sales2)#57, (tue_sales1 / tue_sales2)#58, (wed_sales1 / wed_sales2)#59, (thu_sales1 / thu_sales2)#60, (fri_sales1 / fri_sales2)#61, (sat_sales1 / sat_sales2)#62] -(39) ColumnarToRow [codegen id : 1] +(39) CometColumnarToRow [codegen id : 1] Input [10]: [s_store_name1#26, s_store_id1#28, d_week_seq1#27, (sun_sales1 / sun_sales2)#56, (mon_sales1 / mon_sales2)#57, (tue_sales1 / tue_sales2)#58, (wed_sales1 / wed_sales2)#59, (thu_sales1 / thu_sales2)#60, (fri_sales1 / fri_sales2)#61, (sat_sales1 / sat_sales2)#62] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt index 140a764391..15f2e45693 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt index e87f6ce763..542920b01c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (38) +* CometColumnarToRow (38) +- CometTakeOrderedAndProject (37) +- CometFilter (36) +- CometHashAggregate (35) @@ -215,14 +215,14 @@ Condition : (cnt#23 >= 10) Input [2]: [state#22, cnt#23] Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#23 ASC NULLS FIRST], output=[state#22,cnt#23]), [state#22, cnt#23], 100, [cnt#23 ASC NULLS FIRST], [state#22, cnt#23] -(38) ColumnarToRow [codegen id : 1] +(38) CometColumnarToRow [codegen id : 1] Input [2]: [state#22, cnt#23] ===== Subqueries ===== Subquery:1 Hosting operator id = 8 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (43) -+- * ColumnarToRow (42) ++- * CometColumnarToRow (42) +- CometProject (41) +- CometFilter (40) +- CometScan parquet spark_catalog.default.date_dim (39) @@ -243,7 +243,7 @@ Condition : ((isnotnull(d_month_seq#10) AND (d_month_seq#10 = Subquery scalar-su Input [2]: [d_date_sk#9, d_month_seq#10] Arguments: [d_date_sk#9], [d_date_sk#9] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] (43) BroadcastExchange @@ -251,7 +251,7 @@ Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] Subquery:2 Hosting operator id = 40 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* ColumnarToRow (50) +* CometColumnarToRow (50) +- CometHashAggregate (49) +- CometExchange (48) +- CometHashAggregate (47) @@ -289,7 +289,7 @@ Input [1]: [d_month_seq#24] Keys [1]: [d_month_seq#24] Functions: [] -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [1]: [d_month_seq#24] Subquery:3 Hosting operator id = 14 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt index 05566cbca5..1ba3d55d23 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [state,cnt] CometFilter [state,cnt] @@ -25,13 +25,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [d_month_seq] CometExchange [d_month_seq] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt index 0d64004d9a..5596623c61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (61) +* CometColumnarToRow (61) +- CometTakeOrderedAndProject (60) +- CometHashAggregate (59) +- CometExchange (58) @@ -339,14 +339,14 @@ Functions [1]: [sum(total_sales#36)] Input [2]: [i_item_id#12, total_sales#41] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#12 ASC NULLS FIRST,total_sales#41 ASC NULLS FIRST], output=[i_item_id#12,total_sales#41]), [i_item_id#12, total_sales#41], 100, [i_item_id#12 ASC NULLS FIRST, total_sales#41 ASC NULLS FIRST], [i_item_id#12, total_sales#41] -(61) ColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [codegen id : 1] Input [2]: [i_item_id#12, total_sales#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (66) -+- * ColumnarToRow (65) ++- * CometColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometScan parquet spark_catalog.default.date_dim (62) @@ -367,7 +367,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (66) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt index 4dd8b3b5bd..ea973a4810 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,total_sales] CometHashAggregate [i_item_id,total_sales,sum,isEmpty,sum(total_sales)] @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt index 4a157e64e8..9656e93142 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * Project (65) +- * BroadcastNestedLoopJoin Inner BuildRight (64) - :- * ColumnarToRow (41) + :- * CometColumnarToRow (41) : +- CometHashAggregate (40) : +- CometExchange (39) : +- CometHashAggregate (38) @@ -43,7 +43,7 @@ : +- CometFilter (33) : +- CometScan parquet spark_catalog.default.item (32) +- BroadcastExchange (63) - +- * ColumnarToRow (62) + +- * CometColumnarToRow (62) +- CometHashAggregate (61) +- CometExchange (60) +- CometHashAggregate (59) @@ -256,7 +256,7 @@ Input [1]: [sum#23] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -(41) ColumnarToRow [codegen id : 2] +(41) CometColumnarToRow [codegen id : 2] Input [1]: [promotions#24] (42) CometScan parquet spark_catalog.default.store_sales @@ -345,7 +345,7 @@ Input [1]: [sum#37] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] -(62) ColumnarToRow [codegen id : 1] +(62) CometColumnarToRow [codegen id : 1] Input [1]: [total#38] (63) BroadcastExchange @@ -364,7 +364,7 @@ Input [2]: [promotions#24, total#38] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (70) -+- * ColumnarToRow (69) ++- * CometColumnarToRow (69) +- CometProject (68) +- CometFilter (67) +- CometScan parquet spark_catalog.default.date_dim (66) @@ -385,7 +385,7 @@ Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1 Input [3]: [d_date_sk#14, d_year#15, d_moy#16] Arguments: [d_date_sk#14], [d_date_sk#14] -(69) ColumnarToRow [codegen id : 1] +(69) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] (70) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt index 548614416e..30c16b696d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (2) Project [promotions,total] BroadcastNestedLoopJoin - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [promotions,sum,sum(UnscaledValue(ss_ext_sales_price))] CometExchange #1 @@ -23,7 +23,7 @@ WholeStageCodegen (2) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -54,7 +54,7 @@ WholeStageCodegen (2) InputAdapter BroadcastExchange #9 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [total,sum,sum(UnscaledValue(ss_ext_sales_price))] CometExchange #10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt index c174784d38..b80eab371d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (28) +* CometColumnarToRow (28) +- CometTakeOrderedAndProject (27) +- CometHashAggregate (26) +- CometExchange (25) @@ -158,6 +158,6 @@ Functions [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) TH Input [8]: [substr(w_warehouse_name, 1, 20)#20, sm_type#9, web_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25] Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#20 ASC NULLS FIRST,sm_type#9 ASC NULLS FIRST,web_name#11 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#20,sm_type#9,web_name#11,30 days #21,31 - 60 days #22,61 - 90 days #23,91 - 120 days #24,>120 days #25]), [substr(w_warehouse_name, 1, 20)#20, sm_type#9, web_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25], 100, [substr(w_warehouse_name, 1, 20)#20 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, web_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#20, sm_type#9, web_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [8]: [substr(w_warehouse_name, 1, 20)#20, sm_type#9, web_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt index 6a9b1ced35..002eedf08e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] CometHashAggregate [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,sum,sum,sum,sum,sum,sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 30) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 60) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 90) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) > 120) THEN 1 ELSE 0 END)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt index 2a5ae3d1f4..08e1a99a58 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (29) +- * Project (28) +- * Filter (27) +- Window (26) - +- * ColumnarToRow (25) + +- * CometColumnarToRow (25) +- CometSort (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -144,7 +144,7 @@ Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometNative Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] Arguments: [i_manager_id#5, sum_sales#20, _w0#21], [i_manager_id#5 ASC NULLS FIRST] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] (26) Window @@ -167,7 +167,7 @@ Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (34) -+- * ColumnarToRow (33) ++- * CometColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] -(33) ColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#15, d_moy#17] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt index 8ea16ea740..8e1ce8c997 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_manager_id] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_manager_id,sum_sales,_w0] CometExchange [i_manager_id] #1 @@ -27,7 +27,7 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_month_seq,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt index 2dcffc7a77..8fd7206781 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (173) +* CometColumnarToRow (173) +- CometSort (172) +- CometColumnarExchange (171) +- CometProject (170) @@ -938,14 +938,14 @@ Arguments: rangepartitioning(product_name#76 ASC NULLS FIRST, store_name#78 ASC Input [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] Arguments: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162], [product_name#76 ASC NULLS FIRST, store_name#78 ASC NULLS FIRST, cnt#162 ASC NULLS FIRST] -(173) ColumnarToRow [codegen id : 1] +(173) CometColumnarToRow [codegen id : 1] Input [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (177) -+- * ColumnarToRow (176) ++- * CometColumnarToRow (176) +- CometFilter (175) +- CometScan parquet spark_catalog.default.date_dim (174) @@ -961,7 +961,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(176) ColumnarToRow [codegen id : 1] +(176) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] (177) BroadcastExchange @@ -970,7 +970,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#104 IN dynamicpruning#105 BroadcastExchange (181) -+- * ColumnarToRow (180) ++- * CometColumnarToRow (180) +- CometFilter (179) +- CometScan parquet spark_catalog.default.date_dim (178) @@ -986,7 +986,7 @@ ReadSchema: struct Input [2]: [d_date_sk#117, d_year#118] Condition : ((isnotnull(d_year#118) AND (d_year#118 = 2000)) AND isnotnull(d_date_sk#117)) -(180) ColumnarToRow [codegen id : 1] +(180) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#117, d_year#118] (181) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt index 49a08d6f8b..0ce55b5714 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] CometColumnarExchange [product_name,store_name,cnt] #1 @@ -51,7 +51,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -158,7 +158,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #22 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt index e0f1b47992..09ddb55e37 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (39) +* CometColumnarToRow (39) +- CometTakeOrderedAndProject (38) +- CometProject (37) +- CometBroadcastHashJoin (36) @@ -219,14 +219,14 @@ Arguments: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wh Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16] Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#2 ASC NULLS FIRST,i_item_desc#13 ASC NULLS FIRST], output=[s_store_name#2,i_item_desc#13,revenue#11,i_current_price#14,i_wholesale_cost#15,i_brand#16]), [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16], 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#13 ASC NULLS FIRST], [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16] -(39) ColumnarToRow [codegen id : 1] +(39) CometColumnarToRow [codegen id : 1] Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (44) -+- * ColumnarToRow (43) ++- * CometColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan parquet spark_catalog.default.date_dim (40) @@ -247,7 +247,7 @@ Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_mont Input [2]: [d_date_sk#8, d_month_seq#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(43) ColumnarToRow [codegen id : 1] +(43) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#8] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt index be08ee23a0..28412503a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] CometProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt index 693eb5662d..58f093ae40 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (50) +* CometColumnarToRow (50) +- CometTakeOrderedAndProject (49) +- CometHashAggregate (48) +- CometExchange (47) @@ -277,14 +277,14 @@ Functions [36]: [sum(jan_sales#141), sum(feb_sales#142), sum(mar_sales#143), sum Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#139, year#140, jan_sales#263, feb_sales#264, mar_sales#265, apr_sales#266, may_sales#267, jun_sales#268, jul_sales#269, aug_sales#270, sep_sales#271, oct_sales#272, nov_sales#273, dec_sales#274, jan_sales_per_sq_foot#275, feb_sales_per_sq_foot#276, mar_sales_per_sq_foot#277, apr_sales_per_sq_foot#278, may_sales_per_sq_foot#279, jun_sales_per_sq_foot#280, jul_sales_per_sq_foot#281, aug_sales_per_sq_foot#282, sep_sales_per_sq_foot#283, oct_sales_per_sq_foot#284, nov_sales_per_sq_foot#285, dec_sales_per_sq_foot#286, jan_net#287, feb_net#288, mar_net#289, apr_net#290, may_net#291, jun_net#292, jul_net#293, aug_net#294, sep_net#295, oct_net#296, nov_net#297, dec_net#298] Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_warehouse_name#10 ASC NULLS FIRST], output=[w_warehouse_name#10,w_warehouse_sq_ft#11,w_city#12,w_county#13,w_state#14,w_country#15,ship_carriers#139,year#140,jan_sales#263,feb_sales#264,mar_sales#265,apr_sales#266,may_sales#267,jun_sales#268,jul_sales#269,aug_sales#270,sep_sales#271,oct_sales#272,nov_sales#273,dec_sales#274,jan_sales_per_sq_foot#275,feb_sales_per_sq_foot#276,mar_sales_per_sq_foot#277,apr_sales_per_sq_foot#278,may_sales_per_sq_foot#279,jun_sales_per_sq_foot#280,jul_sales_per_sq_foot#281,aug_sales_per_sq_foot#282,sep_sales_per_sq_foot#283,oct_sales_per_sq_foot#284,nov_sales_per_sq_foot#285,dec_sales_per_sq_foot#286,jan_net#287,feb_net#288,mar_net#289,apr_net#290,may_net#291,jun_net#292,jul_net#293,aug_net#294,sep_net#295,oct_net#296,nov_net#297,dec_net#298]), [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#139, year#140, jan_sales#263, feb_sales#264, mar_sales#265, apr_sales#266, may_sales#267, jun_sales#268, jul_sales#269, aug_sales#270, sep_sales#271, oct_sales#272, nov_sales#273, dec_sales#274, jan_sales_per_sq_foot#275, feb_sales_per_sq_foot#276, mar_sales_per_sq_foot#277, apr_sales_per_sq_foot#278, may_sales_per_sq_foot#279, jun_sales_per_sq_foot#280, jul_sales_per_sq_foot#281, aug_sales_per_sq_foot#282, sep_sales_per_sq_foot#283, oct_sales_per_sq_foot#284, nov_sales_per_sq_foot#285, dec_sales_per_sq_foot#286, jan_net#287, feb_net#288, mar_net#289, apr_net#290, may_net#291, jun_net#292, jul_net#293, aug_net#294, sep_net#295, oct_net#296, nov_net#297, dec_net#298], 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#139, year#140, jan_sales#263, feb_sales#264, mar_sales#265, apr_sales#266, may_sales#267, jun_sales#268, jul_sales#269, aug_sales#270, sep_sales#271, oct_sales#272, nov_sales#273, dec_sales#274, jan_sales_per_sq_foot#275, feb_sales_per_sq_foot#276, mar_sales_per_sq_foot#277, apr_sales_per_sq_foot#278, may_sales_per_sq_foot#279, jun_sales_per_sq_foot#280, jul_sales_per_sq_foot#281, aug_sales_per_sq_foot#282, sep_sales_per_sq_foot#283, oct_sales_per_sq_foot#284, nov_sales_per_sq_foot#285, dec_sales_per_sq_foot#286, jan_net#287, feb_net#288, mar_net#289, apr_net#290, may_net#291, jun_net#292, jul_net#293, aug_net#294, sep_net#295, oct_net#296, nov_net#297, dec_net#298] -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#139, year#140, jan_sales#263, feb_sales#264, mar_sales#265, apr_sales#266, may_sales#267, jun_sales#268, jul_sales#269, aug_sales#270, sep_sales#271, oct_sales#272, nov_sales#273, dec_sales#274, jan_sales_per_sq_foot#275, feb_sales_per_sq_foot#276, mar_sales_per_sq_foot#277, apr_sales_per_sq_foot#278, may_sales_per_sq_foot#279, jun_sales_per_sq_foot#280, jul_sales_per_sq_foot#281, aug_sales_per_sq_foot#282, sep_sales_per_sq_foot#283, oct_sales_per_sq_foot#284, nov_sales_per_sq_foot#285, dec_sales_per_sq_foot#286, jan_net#287, feb_net#288, mar_net#289, apr_net#290, may_net#291, jun_net#292, jul_net#293, aug_net#294, sep_net#295, oct_net#296, nov_net#297, dec_net#298] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (54) -+- * ColumnarToRow (53) ++- * CometColumnarToRow (53) +- CometFilter (52) +- CometScan parquet spark_catalog.default.date_dim (51) @@ -300,7 +300,7 @@ ReadSchema: struct Input [3]: [d_date_sk#16, d_year#17, d_moy#18] Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) -(53) ColumnarToRow [codegen id : 1] +(53) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#16, d_year#17, d_moy#18] (54) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt index a2b5963e13..3529a210a6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum(jan_sales),sum(feb_sales),sum(mar_sales),sum(apr_sales),sum(may_sales),sum(jun_sales),sum(jul_sales),sum(aug_sales),sum(sep_sales),sum(oct_sales),sum(nov_sales),sum(dec_sales),sum((jan_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((feb_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((mar_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((apr_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((may_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jun_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jul_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((aug_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((sep_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((oct_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((nov_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((dec_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum(jan_net),sum(feb_net),sum(mar_net),sum(apr_net),sum(may_net),sum(jun_net),sum(jul_net),sum(aug_net),sum(sep_net),sum(oct_net),sum(nov_net),sum(dec_net)] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt index 4e78dad21d..487789ad10 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (28) +- * Filter (27) +- Window (26) - +- * ColumnarToRow (25) + +- * CometColumnarToRow (25) +- CometSort (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -143,7 +143,7 @@ Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, CometNativeS Input [9]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, sumsales#30] Arguments: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, sumsales#30], [i_category#19 ASC NULLS FIRST, sumsales#30 DESC NULLS LAST] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [9]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, sumsales#30] (26) Window @@ -162,7 +162,7 @@ Arguments: 100, [i_category#19 ASC NULLS FIRST, i_class#20 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (33) -+- * ColumnarToRow (32) ++- * CometColumnarToRow (32) +- CometProject (31) +- CometFilter (30) +- CometScan parquet spark_catalog.default.date_dim (29) @@ -183,7 +183,7 @@ Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_mont Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] (33) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt index 4825ab898b..3a51784865 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ InputAdapter Window [sumsales,i_category] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] CometExchange [i_category] #1 @@ -23,7 +23,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt index 121987260e..00c0db86ec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (38) +* CometColumnarToRow (38) +- CometTakeOrderedAndProject (37) +- CometProject (36) +- CometBroadcastHashJoin (35) @@ -213,14 +213,14 @@ Arguments: [c_last_name#27, c_first_name#26, ca_city#33, bought_city#28, ss_tick Input [8]: [c_last_name#27, c_first_name#26, ca_city#33, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#27 ASC NULLS FIRST,ss_ticket_number#5 ASC NULLS FIRST], output=[c_last_name#27,c_first_name#26,ca_city#33,bought_city#28,ss_ticket_number#5,extended_price#29,extended_tax#31,list_price#30]), [c_last_name#27, c_first_name#26, ca_city#33, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30], 100, [c_last_name#27 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#27, c_first_name#26, ca_city#33, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] -(38) ColumnarToRow [codegen id : 1] +(38) CometColumnarToRow [codegen id : 1] Input [8]: [c_last_name#27, c_first_name#26, ca_city#33, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (43) -+- * ColumnarToRow (42) ++- * CometColumnarToRow (42) +- CometProject (41) +- CometFilter (40) +- CometScan parquet spark_catalog.default.date_dim (39) @@ -241,7 +241,7 @@ Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AN Input [3]: [d_date_sk#11, d_year#12, d_dom#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (43) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt index 568fd734b9..986918c8b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] CometProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt index 5386a3d166..eaf61eb735 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject (44) : :- * Project (27) : : +- * BroadcastHashJoin LeftAnti BuildRight (26) : : :- * BroadcastHashJoin LeftAnti BuildRight (19) - : : : :- * ColumnarToRow (12) + : : : :- * CometColumnarToRow (12) : : : : +- CometBroadcastHashJoin (11) : : : : :- CometFilter (2) : : : : : +- CometScan parquet spark_catalog.default.customer (1) @@ -23,24 +23,24 @@ TakeOrderedAndProject (44) : : : : +- CometFilter (5) : : : : +- CometScan parquet spark_catalog.default.date_dim (4) : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) + : : : +- * CometColumnarToRow (17) : : : +- CometProject (16) : : : +- CometBroadcastHashJoin (15) : : : :- CometScan parquet spark_catalog.default.web_sales (13) : : : +- ReusedExchange (14) : : +- BroadcastExchange (25) - : : +- * ColumnarToRow (24) + : : +- * CometColumnarToRow (24) : : +- CometProject (23) : : +- CometBroadcastHashJoin (22) : : :- CometScan parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (32) - : +- * ColumnarToRow (31) + : +- * CometColumnarToRow (31) : +- CometProject (30) : +- CometFilter (29) : +- CometScan parquet spark_catalog.default.customer_address (28) +- BroadcastExchange (38) - +- * ColumnarToRow (37) + +- * CometColumnarToRow (37) +- CometFilter (36) +- CometScan parquet spark_catalog.default.customer_demographics (35) @@ -100,7 +100,7 @@ Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Right output [1]: [ss_customer_sk#4] Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight -(12) ColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 5] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (13) CometScan parquet spark_catalog.default.web_sales @@ -122,7 +122,7 @@ Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] Arguments: [ws_bill_customer_sk#10], [ws_bill_customer_sk#10] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#10] (18) BroadcastExchange @@ -154,7 +154,7 @@ Arguments: [cs_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] Arguments: [cs_ship_customer_sk#14], [cs_ship_customer_sk#14] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [1]: [cs_ship_customer_sk#14] (25) BroadcastExchange @@ -186,7 +186,7 @@ Condition : (ca_state#19 IN (KY,GA,NM) AND isnotnull(ca_address_sk#18)) Input [2]: [ca_address_sk#18, ca_state#19] Arguments: [ca_address_sk#18], [ca_address_sk#18] -(31) ColumnarToRow [codegen id : 3] +(31) CometColumnarToRow [codegen id : 3] Input [1]: [ca_address_sk#18] (32) BroadcastExchange @@ -214,7 +214,7 @@ ReadSchema: struct Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) ColumnarToRow [codegen id : 3] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) CometScan parquet spark_catalog.default.store_sales @@ -168,7 +168,7 @@ Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] Input [3]: [s_state#15, _w0#18, s_state#15] Arguments: [s_state#15, _w0#18, s_state#15], [s_state#15 ASC NULLS FIRST, _w0#18 DESC NULLS LAST] -(27) ColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [3]: [s_state#15, _w0#18, s_state#15] (28) Window @@ -253,7 +253,7 @@ Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (49) -+- * ColumnarToRow (48) ++- * CometColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometScan parquet spark_catalog.default.date_dim (45) @@ -274,7 +274,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(48) ColumnarToRow [codegen id : 1] +(48) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (49) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt index 704213148f..7287426993 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count Expand [ss_net_profit,s_state,s_county] Project [ss_net_profit,s_state,s_county] BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] @@ -25,7 +25,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -38,7 +38,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #5 WholeStageCodegen (3) BroadcastHashJoin [s_state,s_state] - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] @@ -50,7 +50,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count InputAdapter Window [_w0,s_state] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [s_state,_w0] CometHashAggregate [s_state,_w0,sum,sum(UnscaledValue(ss_net_profit))] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt index 79a71af03a..fe8f7c5f76 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (37) +* CometColumnarToRow (37) +- CometSort (36) +- CometColumnarExchange (35) +- CometHashAggregate (34) @@ -209,14 +209,14 @@ Arguments: rangepartitioning(ext_price#41 DESC NULLS LAST, brand_id#39 ASC NULLS Input [5]: [brand_id#39, brand#40, t_hour#35, t_minute#36, ext_price#41] Arguments: [brand_id#39, brand#40, t_hour#35, t_minute#36, ext_price#41], [ext_price#41 DESC NULLS LAST, brand_id#39 ASC NULLS FIRST] -(37) ColumnarToRow [codegen id : 1] +(37) CometColumnarToRow [codegen id : 1] Input [5]: [brand_id#39, brand#40, t_hour#35, t_minute#36, ext_price#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (42) -+- * ColumnarToRow (41) ++- * CometColumnarToRow (41) +- CometProject (40) +- CometFilter (39) +- CometScan parquet spark_catalog.default.date_dim (38) @@ -237,7 +237,7 @@ Condition : ((((isnotnull(d_moy#12) AND isnotnull(d_year#11)) AND (d_moy#12 = 11 Input [3]: [d_date_sk#10, d_year#11, d_moy#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(41) ColumnarToRow [codegen id : 1] +(41) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (42) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt index 88ca8b67df..7214ec67a5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [brand_id,brand,t_hour,t_minute,ext_price] CometColumnarExchange [ext_price,brand_id] #1 @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt index 6aae88fff3..4179f3b1d6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (64) +* CometColumnarToRow (64) +- CometTakeOrderedAndProject (63) +- CometHashAggregate (62) +- CometExchange (61) @@ -364,14 +364,14 @@ Functions [1]: [count(1)] Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, [total_cnt#37 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] -(64) ColumnarToRow [codegen id : 1] +(64) CometColumnarToRow [codegen id : 1] Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (69) -+- * ColumnarToRow (68) ++- * CometColumnarToRow (68) +- CometProject (67) +- CometFilter (66) +- CometScan parquet spark_catalog.default.date_dim (65) @@ -392,7 +392,7 @@ Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_dat Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(68) ColumnarToRow [codegen id : 1] +(68) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (69) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt index ccd1b66fa2..fe26a3f5cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count,count(1)] @@ -32,7 +32,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] CometFilter [d_date_sk,d_date,d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt index b0ca5520f2..a7c93664a7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (32) +* CometColumnarToRow (32) +- CometSort (31) +- CometColumnarExchange (30) +- CometProject (29) @@ -179,14 +179,14 @@ Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, Co Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] Arguments: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [cnt#17 DESC NULLS LAST] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (37) -+- * ColumnarToRow (36) ++- * CometColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometScan parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt index 4417365325..eaede7232d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometColumnarExchange [cnt] #1 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt index c7a559c8ef..6358cf7e31 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (68) +* CometColumnarToRow (68) +- CometTakeOrderedAndProject (67) +- CometProject (66) +- CometBroadcastHashJoin (65) @@ -388,14 +388,14 @@ Arguments: [customer_id#25, customer_first_name#26, customer_last_name#27], [cus Input [3]: [customer_id#25, customer_first_name#26, customer_last_name#27] Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#25 ASC NULLS FIRST,customer_id#25 ASC NULLS FIRST,customer_id#25 ASC NULLS FIRST], output=[customer_id#25,customer_first_name#26,customer_last_name#27]), [customer_id#25, customer_first_name#26, customer_last_name#27], 100, [customer_id#25 ASC NULLS FIRST, customer_id#25 ASC NULLS FIRST, customer_id#25 ASC NULLS FIRST], [customer_id#25, customer_first_name#26, customer_last_name#27] -(68) ColumnarToRow [codegen id : 1] +(68) CometColumnarToRow [codegen id : 1] Input [3]: [customer_id#25, customer_first_name#26, customer_last_name#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (72) -+- * ColumnarToRow (71) ++- * CometColumnarToRow (71) +- CometFilter (70) +- CometScan parquet spark_catalog.default.date_dim (69) @@ -411,7 +411,7 @@ ReadSchema: struct Input [2]: [d_date_sk#9, d_year#10] Condition : (((isnotnull(d_year#10) AND (d_year#10 = 2001)) AND d_year#10 IN (2001,2002)) AND isnotnull(d_date_sk#9)) -(71) ColumnarToRow [codegen id : 1] +(71) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] (72) BroadcastExchange @@ -420,7 +420,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 BroadcastExchange (76) -+- * ColumnarToRow (75) ++- * CometColumnarToRow (75) +- CometFilter (74) +- CometScan parquet spark_catalog.default.date_dim (73) @@ -436,7 +436,7 @@ ReadSchema: struct Input [2]: [d_date_sk#22, d_year#23] Condition : (((isnotnull(d_year#23) AND (d_year#23 = 2002)) AND d_year#23 IN (2001,2002)) AND isnotnull(d_date_sk#22)) -(75) ColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#22, d_year#23] (76) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt index 631a82f1e3..9bde9ac863 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] CometProject [customer_id,customer_first_name,customer_last_name] @@ -23,7 +23,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -46,7 +46,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt index 2f33f50a8b..9d68c7a66c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (124) +* CometColumnarToRow (124) +- CometTakeOrderedAndProject (123) +- CometProject (122) +- CometSortMergeJoin (121) @@ -676,14 +676,14 @@ Arguments: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#1 Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#124 ASC NULLS FIRST], output=[prev_year#120,year#121,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#122,curr_yr_cnt#123,sales_cnt_diff#124,sales_amt_diff#125]), [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], 100, [sales_cnt_diff#124 ASC NULLS FIRST], [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] -(124) ColumnarToRow [codegen id : 1] +(124) CometColumnarToRow [codegen id : 1] Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (128) -+- * ColumnarToRow (127) ++- * CometColumnarToRow (127) +- CometFilter (126) +- CometScan parquet spark_catalog.default.date_dim (125) @@ -699,7 +699,7 @@ ReadSchema: struct Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(127) ColumnarToRow [codegen id : 1] +(127) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] (128) BroadcastExchange @@ -712,7 +712,7 @@ Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN d Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 BroadcastExchange (132) -+- * ColumnarToRow (131) ++- * CometColumnarToRow (131) +- CometFilter (130) +- CometScan parquet spark_catalog.default.date_dim (129) @@ -728,7 +728,7 @@ ReadSchema: struct Input [2]: [d_date_sk#77, d_year#78] Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(131) ColumnarToRow [codegen id : 1] +(131) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#77, d_year#78] (132) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt index 9e340130a5..5a8af58abc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] @@ -27,7 +27,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -102,7 +102,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt index a2a17ccfce..24ed6809af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (34) +* CometColumnarToRow (34) +- CometTakeOrderedAndProject (33) +- CometHashAggregate (32) +- CometExchange (31) @@ -190,6 +190,6 @@ Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#12))] Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#39, sales_amt#40] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#10 ASC NULLS FIRST,col_name#11 ASC NULLS FIRST,d_year#8 ASC NULLS FIRST,d_qoy#9 ASC NULLS FIRST,i_category#6 ASC NULLS FIRST], output=[channel#10,col_name#11,d_year#8,d_qoy#9,i_category#6,sales_cnt#39,sales_amt#40]), [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#39, sales_amt#40], 100, [channel#10 ASC NULLS FIRST, col_name#11 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#9 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#39, sales_amt#40] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#39, sales_amt#40] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt index eb3e14d5ec..3e89713523 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt,count,sum,count(1),sum(UnscaledValue(ext_sales_price))] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt index 661b23c47b..dccd7c8e56 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject (84) +- * HashAggregate (81) +- * Expand (80) +- Union (79) - :- * ColumnarToRow (31) + :- * CometColumnarToRow (31) : +- CometProject (30) : +- CometBroadcastHashJoin (29) : :- CometHashAggregate (16) @@ -39,7 +39,7 @@ TakeOrderedAndProject (84) :- * Project (50) : +- * BroadcastNestedLoopJoin Inner BuildLeft (49) : :- BroadcastExchange (40) - : : +- * ColumnarToRow (39) + : : +- * CometColumnarToRow (39) : : +- CometHashAggregate (38) : : +- CometExchange (37) : : +- CometHashAggregate (36) @@ -47,7 +47,7 @@ TakeOrderedAndProject (84) : : +- CometBroadcastHashJoin (34) : : :- CometScan parquet spark_catalog.default.catalog_sales (32) : : +- ReusedExchange (33) - : +- * ColumnarToRow (48) + : +- * CometColumnarToRow (48) : +- CometHashAggregate (47) : +- CometExchange (46) : +- CometHashAggregate (45) @@ -55,7 +55,7 @@ TakeOrderedAndProject (84) : +- CometBroadcastHashJoin (43) : :- CometScan parquet spark_catalog.default.catalog_returns (41) : +- ReusedExchange (42) - +- * ColumnarToRow (78) + +- * CometColumnarToRow (78) +- CometProject (77) +- CometBroadcastHashJoin (76) :- CometHashAggregate (63) @@ -226,7 +226,7 @@ Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] Arguments: [sales#22, returns#24, profit#25, channel#26, id#27], [sales#22, coalesce(returns#20, 0.00) AS returns#24, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#25, store channel AS channel#26, s_store_sk#8 AS id#27] -(31) ColumnarToRow [codegen id : 1] +(31) CometColumnarToRow [codegen id : 1] Input [5]: [sales#22, returns#24, profit#25, channel#26, id#27] (32) CometScan parquet spark_catalog.default.catalog_sales @@ -262,7 +262,7 @@ Input [3]: [cs_call_center_sk#28, sum#34, sum#35] Keys [1]: [cs_call_center_sk#28] Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#29)), sum(UnscaledValue(cs_net_profit#30))] -(39) ColumnarToRow [codegen id : 2] +(39) CometColumnarToRow [codegen id : 2] Input [3]: [cs_call_center_sk#28, sales#36, profit#37] (40) BroadcastExchange @@ -302,7 +302,7 @@ Input [2]: [sum#43, sum#44] Keys: [] Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] -(48) ColumnarToRow +(48) CometColumnarToRow Input [2]: [returns#45, profit_loss#46] (49) BroadcastNestedLoopJoin [codegen id : 3] @@ -438,7 +438,7 @@ Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] Arguments: [sales#70, returns#72, profit#73, channel#74, id#75], [sales#70, coalesce(returns#68, 0.00) AS returns#72, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#73, web channel AS channel#74, wp_web_page_sk#56 AS id#75] -(78) ColumnarToRow [codegen id : 4] +(78) CometColumnarToRow [codegen id : 4] Input [5]: [sales#70, returns#72, profit#73, channel#74, id#75] (79) Union @@ -473,7 +473,7 @@ Arguments: 100, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (89) -+- * ColumnarToRow (88) ++- * CometColumnarToRow (88) +- CometProject (87) +- CometFilter (86) +- CometScan parquet spark_catalog.default.date_dim (85) @@ -494,7 +494,7 @@ Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 < Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(88) ColumnarToRow [codegen id : 1] +(88) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (89) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt index e8226ab1fa..60fcf20e24 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter Union WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [returns,profit,profit_loss,s_store_sk] [sales,returns,profit,channel,id] CometBroadcastHashJoin [s_store_sk,sales,profit,s_store_sk,returns,profit_loss] @@ -25,7 +25,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] @@ -56,7 +56,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter BroadcastExchange #8 WholeStageCodegen (2) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [cs_call_center_sk,sales,profit,sum,sum,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] CometExchange [cs_call_center_sk] #9 @@ -66,7 +66,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [returns,profit_loss,sum,sum,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] CometExchange #10 @@ -77,7 +77,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 WholeStageCodegen (4) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [returns,profit,profit_loss,wp_web_page_sk] [sales,returns,profit,channel,id] CometBroadcastHashJoin [wp_web_page_sk,sales,profit,wp_web_page_sk,returns,profit_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt index b8b24b9d0b..41ea1060b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (67) +- * Project (66) - +- * ColumnarToRow (65) + +- * CometColumnarToRow (65) +- CometSortMergeJoin (64) :- CometProject (43) : +- CometSortMergeJoin (42) @@ -360,7 +360,7 @@ Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss Right output [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59], Inner -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] (66) Project [codegen id : 1] @@ -375,7 +375,7 @@ Arguments: 100, [ratio#63 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 D Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (71) -+- * ColumnarToRow (70) ++- * CometColumnarToRow (70) +- CometFilter (69) +- CometScan parquet spark_catalog.default.date_dim (68) @@ -391,7 +391,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(70) ColumnarToRow [codegen id : 1] +(70) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] (71) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt index 3318702eff..6b85e03ae9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] WholeStageCodegen (1) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] - ColumnarToRow + CometColumnarToRow InputAdapter CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] @@ -22,7 +22,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt index e697c9038b..c4b54c4ed5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (30) +* CometColumnarToRow (30) +- CometTakeOrderedAndProject (29) +- CometProject (28) +- CometBroadcastHashJoin (27) @@ -169,14 +169,14 @@ Arguments: [c_last_name#23, c_first_name#22, substr(s_city, 1, 30)#26, ss_ticket Input [7]: [c_last_name#23, c_first_name#22, substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#24, profit#25, s_city#15] Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#23 ASC NULLS FIRST,c_first_name#22 ASC NULLS FIRST,substr(s_city#15, 1, 30) ASC NULLS FIRST,profit#25 ASC NULLS FIRST], output=[c_last_name#23,c_first_name#22,substr(s_city, 1, 30)#26,ss_ticket_number#5,amt#24,profit#25]), [c_last_name#23, c_first_name#22, substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#24, profit#25], 100, [c_last_name#23 ASC NULLS FIRST, c_first_name#22 ASC NULLS FIRST, substr(s_city#15, 1, 30) ASC NULLS FIRST, profit#25 ASC NULLS FIRST], [c_last_name#23, c_first_name#22, substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#24, profit#25] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [6]: [c_last_name#23, c_first_name#22, substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#24, profit#25] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (35) -+- * ColumnarToRow (34) ++- * CometColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan parquet spark_catalog.default.date_dim (31) @@ -197,7 +197,7 @@ Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,20 Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt index b69b0cd2e9..d5514ba8a7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit,s_city] CometProject [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit,s_city] @@ -18,7 +18,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dow] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt index f02edb1912..c6f5a077fa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (42) +* CometColumnarToRow (42) +- CometTakeOrderedAndProject (41) +- CometHashAggregate (40) +- CometExchange (39) @@ -236,14 +236,14 @@ Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] Input [2]: [s_store_name#9, sum(ss_net_profit)#21] Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#9 ASC NULLS FIRST], output=[s_store_name#9,sum(ss_net_profit)#21]), [s_store_name#9, sum(ss_net_profit)#21], 100, [s_store_name#9 ASC NULLS FIRST], [s_store_name#9, sum(ss_net_profit)#21] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarToRow [codegen id : 1] Input [2]: [s_store_name#9, sum(ss_net_profit)#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (47) -+- * ColumnarToRow (46) ++- * CometColumnarToRow (46) +- CometProject (45) +- CometFilter (44) +- CometScan parquet spark_catalog.default.date_dim (43) @@ -264,7 +264,7 @@ Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) A Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Arguments: [d_date_sk#5], [d_date_sk#5] -(46) ColumnarToRow [codegen id : 1] +(46) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (47) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt index 91d98aca9b..7beec478b3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [s_store_name,sum(ss_net_profit)] CometHashAggregate [s_store_name,sum(ss_net_profit),sum,sum(UnscaledValue(ss_net_profit))] @@ -16,7 +16,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt index 65fe1a28b0..acdf464198 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject (102) +- * Expand (98) +- Union (97) :- * HashAggregate (38) - : +- * ColumnarToRow (37) + : +- * CometColumnarToRow (37) : +- CometExchange (36) : +- CometHashAggregate (35) : +- CometProject (34) @@ -44,7 +44,7 @@ TakeOrderedAndProject (102) : +- CometFilter (30) : +- CometScan parquet spark_catalog.default.promotion (29) :- * HashAggregate (67) - : +- * ColumnarToRow (66) + : +- * CometColumnarToRow (66) : +- CometExchange (65) : +- CometHashAggregate (64) : +- CometProject (63) @@ -73,7 +73,7 @@ TakeOrderedAndProject (102) : : +- ReusedExchange (58) : +- ReusedExchange (61) +- * HashAggregate (96) - +- * ColumnarToRow (95) + +- * CometColumnarToRow (95) +- CometExchange (94) +- CometHashAggregate (93) +- CometProject (92) @@ -272,7 +272,7 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(co Input [6]: [s_store_id#17, sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] Arguments: hashpartitioning(s_store_id#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(37) ColumnarToRow [codegen id : 1] +(37) CometColumnarToRow [codegen id : 1] Input [6]: [s_store_id#17, sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] (38) HashAggregate [codegen id : 1] @@ -403,7 +403,7 @@ Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#39)), partial_sum(c Input [6]: [cp_catalog_page_id#50, sum#53, sum#54, isEmpty#55, sum#56, isEmpty#57] Arguments: hashpartitioning(cp_catalog_page_id#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(66) ColumnarToRow [codegen id : 2] +(66) CometColumnarToRow [codegen id : 2] Input [6]: [cp_catalog_page_id#50, sum#53, sum#54, isEmpty#55, sum#56, isEmpty#57] (67) HashAggregate [codegen id : 2] @@ -534,7 +534,7 @@ Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(c Input [6]: [web_site_id#81, sum#84, sum#85, isEmpty#86, sum#87, isEmpty#88] Arguments: hashpartitioning(web_site_id#81, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] -(95) ColumnarToRow [codegen id : 3] +(95) CometColumnarToRow [codegen id : 3] Input [6]: [web_site_id#81, sum#84, sum#85, isEmpty#86, sum#87, isEmpty#88] (96) HashAggregate [codegen id : 3] @@ -576,7 +576,7 @@ Arguments: 100, [channel#97 ASC NULLS FIRST, id#98 ASC NULLS FIRST], [channel#97 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (107) -+- * ColumnarToRow (106) ++- * CometColumnarToRow (106) +- CometProject (105) +- CometFilter (104) +- CometScan parquet spark_catalog.default.date_dim (103) @@ -597,7 +597,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(106) ColumnarToRow [codegen id : 1] +(106) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] (107) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt index 012c711bfe..a1ab21e6b5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Union WholeStageCodegen (1) HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [s_store_id] #2 CometHashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] @@ -31,7 +31,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] @@ -58,7 +58,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (2) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [cp_catalog_page_id] #10 CometHashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] @@ -90,7 +90,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [p_promo_sk] #9 WholeStageCodegen (3) HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [web_site_id] #14 CometHashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt index 8ae0dae9ac..86f7932331 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (47) +* CometColumnarToRow (47) +- CometTakeOrderedAndProject (46) +- CometProject (45) +- CometBroadcastHashJoin (44) @@ -263,14 +263,14 @@ Arguments: [c_customer_id#21, c_salutation#23, c_first_name#24, c_last_name#25, Input [16]: [c_customer_id#21, c_salutation#23, c_first_name#24, c_last_name#25, ca_street_number#27, ca_street_name#28, ca_street_type#29, ca_suite_number#30, ca_city#31, ca_county#32, ca_state#33, ca_zip#34, ca_country#35, ca_gmt_offset#36, ca_location_type#37, ctr_total_return#13] Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#21 ASC NULLS FIRST,c_salutation#23 ASC NULLS FIRST,c_first_name#24 ASC NULLS FIRST,c_last_name#25 ASC NULLS FIRST,ca_street_number#27 ASC NULLS FIRST,ca_street_name#28 ASC NULLS FIRST,ca_street_type#29 ASC NULLS FIRST,ca_suite_number#30 ASC NULLS FIRST,ca_city#31 ASC NULLS FIRST,ca_county#32 ASC NULLS FIRST,ca_state#33 ASC NULLS FIRST,ca_zip#34 ASC NULLS FIRST,ca_country#35 ASC NULLS FIRST,ca_gmt_offset#36 ASC NULLS FIRST,ca_location_type#37 ASC NULLS FIRST,ctr_total_return#13 ASC NULLS FIRST], output=[c_customer_id#21,c_salutation#23,c_first_name#24,c_last_name#25,ca_street_number#27,ca_street_name#28,ca_street_type#29,ca_suite_number#30,ca_city#31,ca_county#32,ca_state#33,ca_zip#34,ca_country#35,ca_gmt_offset#36,ca_location_type#37,ctr_total_return#13]), [c_customer_id#21, c_salutation#23, c_first_name#24, c_last_name#25, ca_street_number#27, ca_street_name#28, ca_street_type#29, ca_suite_number#30, ca_city#31, ca_county#32, ca_state#33, ca_zip#34, ca_country#35, ca_gmt_offset#36, ca_location_type#37, ctr_total_return#13], 100, [c_customer_id#21 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_last_name#25 ASC NULLS FIRST, ca_street_number#27 ASC NULLS FIRST, ca_street_name#28 ASC NULLS FIRST, ca_street_type#29 ASC NULLS FIRST, ca_suite_number#30 ASC NULLS FIRST, ca_city#31 ASC NULLS FIRST, ca_county#32 ASC NULLS FIRST, ca_state#33 ASC NULLS FIRST, ca_zip#34 ASC NULLS FIRST, ca_country#35 ASC NULLS FIRST, ca_gmt_offset#36 ASC NULLS FIRST, ca_location_type#37 ASC NULLS FIRST, ctr_total_return#13 ASC NULLS FIRST], [c_customer_id#21, c_salutation#23, c_first_name#24, c_last_name#25, ca_street_number#27, ca_street_name#28, ca_street_type#29, ca_suite_number#30, ca_city#31, ca_county#32, ca_state#33, ca_zip#34, ca_country#35, ca_gmt_offset#36, ca_location_type#37, ctr_total_return#13] -(47) ColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [16]: [c_customer_id#21, c_salutation#23, c_first_name#24, c_last_name#25, ca_street_number#27, ca_street_name#28, ca_street_type#29, ca_suite_number#30, ca_city#31, ca_county#32, ca_state#33, ca_zip#34, ca_country#35, ca_gmt_offset#36, ca_location_type#37, ctr_total_return#13] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -291,7 +291,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt index b5d2c83218..4887a152c5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] @@ -21,7 +21,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt index 34319a8fc3..6117d4a921 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (26) +* CometColumnarToRow (26) +- CometTakeOrderedAndProject (25) +- CometHashAggregate (24) +- CometExchange (23) @@ -145,14 +145,14 @@ Functions: [] Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#2 ASC NULLS FIRST], output=[i_item_id#2,i_item_desc#3,i_current_price#4]), [i_item_id#2, i_item_desc#3, i_current_price#4], 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#8 IN dynamicpruning#9 BroadcastExchange (31) -+- * ColumnarToRow (30) ++- * CometColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_date#11) AND (d_date#11 >= 2000-05-25)) AND (d_date#1 Input [2]: [d_date_sk#10, d_date#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt index 42c3f40e68..755128d2c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] CometHashAggregate [i_item_id,i_item_desc,i_current_price] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt index c0dfa3fdf0..820bc9bd5c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (48) +* CometColumnarToRow (48) +- CometTakeOrderedAndProject (47) +- CometProject (46) +- CometBroadcastHashJoin (45) @@ -265,14 +265,14 @@ Arguments: [item_id#21, sr_item_qty#22, sr_dev#33, cr_item_qty#20, cr_dev#34, wr Input [8]: [item_id#21, sr_item_qty#22, sr_dev#33, cr_item_qty#20, cr_dev#34, wr_item_qty#32, wr_dev#35, average#36] Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_id#21 ASC NULLS FIRST,sr_item_qty#22 ASC NULLS FIRST], output=[item_id#21,sr_item_qty#22,sr_dev#33,cr_item_qty#20,cr_dev#34,wr_item_qty#32,wr_dev#35,average#36]), [item_id#21, sr_item_qty#22, sr_dev#33, cr_item_qty#20, cr_dev#34, wr_item_qty#32, wr_dev#35, average#36], 100, [item_id#21 ASC NULLS FIRST, sr_item_qty#22 ASC NULLS FIRST], [item_id#21, sr_item_qty#22, sr_dev#33, cr_item_qty#20, cr_dev#34, wr_item_qty#32, wr_dev#35, average#36] -(48) ColumnarToRow [codegen id : 1] +(48) CometColumnarToRow [codegen id : 1] Input [8]: [item_id#21, sr_item_qty#22, sr_dev#33, cr_item_qty#20, cr_dev#34, wr_item_qty#32, wr_dev#35, average#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 BroadcastExchange (62) -+- * ColumnarToRow (61) ++- * CometColumnarToRow (61) +- CometProject (60) +- CometBroadcastHashJoin (59) :- CometFilter (50) @@ -344,7 +344,7 @@ Arguments: [d_date#8], [d_date#9], LeftSemi, BuildRight Input [2]: [d_date_sk#7, d_date#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(61) ColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (62) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt index 77345d28a7..0530aad2f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] CometProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] @@ -18,7 +18,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometBroadcastHashJoin [d_date_sk,d_date,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt index 572fd7a66f..ca784ef3e7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (32) +- * Project (31) - +- * ColumnarToRow (30) + +- * CometColumnarToRow (30) +- CometBroadcastHashJoin (29) :- CometBroadcastExchange (25) : +- CometProject (24) @@ -172,7 +172,7 @@ Left output [4]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9] Right output [1]: [sr_cdemo_sk#15] Arguments: [cd_demo_sk#9], [sr_cdemo_sk#15], Inner, BuildLeft -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [5]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9, sr_cdemo_sk#15] (31) Project [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt index 646285a082..3101b29dd5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] WholeStageCodegen (1) Project [c_customer_id,c_last_name,c_first_name] - ColumnarToRow + CometColumnarToRow InputAdapter CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt index c03da66a07..020414c4d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (45) +* CometColumnarToRow (45) +- CometTakeOrderedAndProject (44) +- CometHashAggregate (43) +- CometExchange (42) @@ -256,14 +256,14 @@ Functions [3]: [avg(ws_quantity#4), avg(UnscaledValue(wr_refunded_cash#16)), avg Input [4]: [substr(r_reason_desc, 1, 20)#38, avg(ws_quantity)#39, avg(wr_refunded_cash)#40, avg(wr_fee)#41] Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(r_reason_desc, 1, 20)#38 ASC NULLS FIRST,avg(ws_quantity)#39 ASC NULLS FIRST,avg(wr_refunded_cash)#40 ASC NULLS FIRST,avg(wr_fee)#41 ASC NULLS FIRST], output=[substr(r_reason_desc, 1, 20)#38,avg(ws_quantity)#39,avg(wr_refunded_cash)#40,avg(wr_fee)#41]), [substr(r_reason_desc, 1, 20)#38, avg(ws_quantity)#39, avg(wr_refunded_cash)#40, avg(wr_fee)#41], 100, [substr(r_reason_desc, 1, 20)#38 ASC NULLS FIRST, avg(ws_quantity)#39 ASC NULLS FIRST, avg(wr_refunded_cash)#40 ASC NULLS FIRST, avg(wr_fee)#41 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#38, avg(ws_quantity)#39, avg(wr_refunded_cash)#40, avg(wr_fee)#41] -(45) ColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [4]: [substr(r_reason_desc, 1, 20)#38, avg(ws_quantity)#39, avg(wr_refunded_cash)#40, avg(wr_fee)#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (50) -+- * ColumnarToRow (49) ++- * CometColumnarToRow (49) +- CometProject (48) +- CometFilter (47) +- CometScan parquet spark_catalog.default.date_dim (46) @@ -284,7 +284,7 @@ Condition : ((isnotnull(d_year#29) AND (d_year#29 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#28, d_year#29] Arguments: [d_date_sk#28], [d_date_sk#28] -(49) ColumnarToRow [codegen id : 1] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#28] (50) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt index 4e4ece1bda..28752f3783 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] CometHashAggregate [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),r_reason_desc,sum,count,sum,count,sum,count,avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee))] @@ -25,7 +25,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt index cf40b3e8d5..e7ce31dd2b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (23) +- * Project (22) +- Window (21) - +- * ColumnarToRow (20) + +- * CometColumnarToRow (20) +- CometSort (19) +- CometExchange (18) +- CometHashAggregate (17) @@ -114,7 +114,7 @@ Arguments: hashpartitioning(_w1#17, _w2#18, 5), ENSURE_REQUIREMENTS, CometNative Input [7]: [total_sum#14, i_category#10, i_class#11, lochierarchy#15, _w0#16, _w1#17, _w2#18] Arguments: [total_sum#14, i_category#10, i_class#11, lochierarchy#15, _w0#16, _w1#17, _w2#18], [_w1#17 ASC NULLS FIRST, _w2#18 ASC NULLS FIRST, _w0#16 DESC NULLS LAST] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarToRow [codegen id : 1] Input [7]: [total_sum#14, i_category#10, i_class#11, lochierarchy#15, _w0#16, _w1#17, _w2#18] (21) Window @@ -133,7 +133,7 @@ Arguments: 100, [lochierarchy#15 DESC NULLS LAST, CASE WHEN (lochierarchy#15 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * ColumnarToRow (27) ++- * CometColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(27) ColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt index 86480d58c6..b1bf4c2e38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl InputAdapter Window [_w0,_w1,_w2] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] CometExchange [_w1,_w2] #1 @@ -21,7 +21,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt index 28b942e2f7..5081f68284 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt @@ -5,7 +5,7 @@ +- * Project (46) +- * BroadcastHashJoin LeftAnti BuildRight (45) :- * BroadcastHashJoin LeftAnti BuildRight (31) - : :- * ColumnarToRow (17) + : :- * CometColumnarToRow (17) : : +- CometHashAggregate (16) : : +- CometExchange (15) : : +- CometHashAggregate (14) @@ -23,7 +23,7 @@ : : +- CometFilter (10) : : +- CometScan parquet spark_catalog.default.customer (9) : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) + : +- * CometColumnarToRow (29) : +- CometHashAggregate (28) : +- CometExchange (27) : +- CometHashAggregate (26) @@ -36,7 +36,7 @@ : : +- ReusedExchange (20) : +- ReusedExchange (23) +- BroadcastExchange (44) - +- * ColumnarToRow (43) + +- * CometColumnarToRow (43) +- CometHashAggregate (42) +- CometExchange (41) +- CometHashAggregate (40) @@ -128,7 +128,7 @@ Input [3]: [c_last_name#9, c_first_name#8, d_date#5] Keys [3]: [c_last_name#9, c_first_name#8, d_date#5] Functions: [] -(17) ColumnarToRow [codegen id : 3] +(17) CometColumnarToRow [codegen id : 3] Input [3]: [c_last_name#9, c_first_name#8, d_date#5] (18) CometScan parquet spark_catalog.default.catalog_sales @@ -181,7 +181,7 @@ Input [3]: [c_last_name#17, c_first_name#16, d_date#14] Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] Functions: [] -(29) ColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [3]: [c_last_name#17, c_first_name#16, d_date#14] (30) BroadcastExchange @@ -244,7 +244,7 @@ Input [3]: [c_last_name#25, c_first_name#24, d_date#22] Keys [3]: [c_last_name#25, c_first_name#24, d_date#22] Functions: [] -(43) ColumnarToRow [codegen id : 2] +(43) CometColumnarToRow [codegen id : 2] Input [3]: [c_last_name#25, c_first_name#24, d_date#22] (44) BroadcastExchange @@ -283,7 +283,7 @@ Results [1]: [count(1)#28 AS count(1)#29] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 BroadcastExchange (54) -+- * ColumnarToRow (53) ++- * CometColumnarToRow (53) +- CometProject (52) +- CometFilter (51) +- CometScan parquet spark_catalog.default.date_dim (50) @@ -304,7 +304,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(53) ColumnarToRow [codegen id : 1] +(53) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] (54) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt index 6e3328ae1d..81cd6f40be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt @@ -7,7 +7,7 @@ WholeStageCodegen (4) Project BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometExchange [c_last_name,c_first_name,d_date] #2 @@ -21,7 +21,7 @@ WholeStageCodegen (4) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] @@ -36,7 +36,7 @@ WholeStageCodegen (4) InputAdapter BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometExchange [c_last_name,c_first_name,d_date] #7 @@ -53,7 +53,7 @@ WholeStageCodegen (4) InputAdapter BroadcastExchange #8 WholeStageCodegen (2) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometExchange [c_last_name,c_first_name,d_date] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt index 592e23cd27..6682e151f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt @@ -6,7 +6,7 @@ : : : :- * BroadcastNestedLoopJoin Inner BuildRight (88) : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (67) : : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (46) -: : : : : : :- * ColumnarToRow (25) +: : : : : : :- * CometColumnarToRow (25) : : : : : : : +- CometHashAggregate (24) : : : : : : : +- CometExchange (23) : : : : : : : +- CometHashAggregate (22) @@ -32,7 +32,7 @@ : : : : : : : +- CometFilter (17) : : : : : : : +- CometScan parquet spark_catalog.default.store (16) : : : : : : +- BroadcastExchange (45) -: : : : : : +- * ColumnarToRow (44) +: : : : : : +- * CometColumnarToRow (44) : : : : : : +- CometHashAggregate (43) : : : : : : +- CometExchange (42) : : : : : : +- CometHashAggregate (41) @@ -52,7 +52,7 @@ : : : : : : : +- CometScan parquet spark_catalog.default.time_dim (32) : : : : : : +- ReusedExchange (38) : : : : : +- BroadcastExchange (66) -: : : : : +- * ColumnarToRow (65) +: : : : : +- * CometColumnarToRow (65) : : : : : +- CometHashAggregate (64) : : : : : +- CometExchange (63) : : : : : +- CometHashAggregate (62) @@ -72,7 +72,7 @@ : : : : : : +- CometScan parquet spark_catalog.default.time_dim (53) : : : : : +- ReusedExchange (59) : : : : +- BroadcastExchange (87) -: : : : +- * ColumnarToRow (86) +: : : : +- * CometColumnarToRow (86) : : : : +- CometHashAggregate (85) : : : : +- CometExchange (84) : : : : +- CometHashAggregate (83) @@ -92,7 +92,7 @@ : : : : : +- CometScan parquet spark_catalog.default.time_dim (74) : : : : +- ReusedExchange (80) : : : +- BroadcastExchange (108) -: : : +- * ColumnarToRow (107) +: : : +- * CometColumnarToRow (107) : : : +- CometHashAggregate (106) : : : +- CometExchange (105) : : : +- CometHashAggregate (104) @@ -112,7 +112,7 @@ : : : : +- CometScan parquet spark_catalog.default.time_dim (95) : : : +- ReusedExchange (101) : : +- BroadcastExchange (129) -: : +- * ColumnarToRow (128) +: : +- * CometColumnarToRow (128) : : +- CometHashAggregate (127) : : +- CometExchange (126) : : +- CometHashAggregate (125) @@ -132,7 +132,7 @@ : : : +- CometScan parquet spark_catalog.default.time_dim (116) : : +- ReusedExchange (122) : +- BroadcastExchange (150) -: +- * ColumnarToRow (149) +: +- * CometColumnarToRow (149) : +- CometHashAggregate (148) : +- CometExchange (147) : +- CometHashAggregate (146) @@ -152,7 +152,7 @@ : : +- CometScan parquet spark_catalog.default.time_dim (137) : +- ReusedExchange (143) +- BroadcastExchange (171) - +- * ColumnarToRow (170) + +- * CometColumnarToRow (170) +- CometHashAggregate (169) +- CometExchange (168) +- CometHashAggregate (167) @@ -285,7 +285,7 @@ Input [1]: [count#13] Keys: [] Functions [1]: [count(1)] -(25) ColumnarToRow [codegen id : 8] +(25) CometColumnarToRow [codegen id : 8] Input [1]: [h8_30_to_9#14] (26) CometScan parquet spark_catalog.default.store_sales @@ -368,7 +368,7 @@ Input [1]: [count#24] Keys: [] Functions [1]: [count(1)] -(44) ColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [h9_to_9_30#25] (45) BroadcastExchange @@ -459,7 +459,7 @@ Input [1]: [count#35] Keys: [] Functions [1]: [count(1)] -(65) ColumnarToRow [codegen id : 2] +(65) CometColumnarToRow [codegen id : 2] Input [1]: [h9_30_to_10#36] (66) BroadcastExchange @@ -550,7 +550,7 @@ Input [1]: [count#46] Keys: [] Functions [1]: [count(1)] -(86) ColumnarToRow [codegen id : 3] +(86) CometColumnarToRow [codegen id : 3] Input [1]: [h10_to_10_30#47] (87) BroadcastExchange @@ -641,7 +641,7 @@ Input [1]: [count#57] Keys: [] Functions [1]: [count(1)] -(107) ColumnarToRow [codegen id : 4] +(107) CometColumnarToRow [codegen id : 4] Input [1]: [h10_30_to_11#58] (108) BroadcastExchange @@ -732,7 +732,7 @@ Input [1]: [count#68] Keys: [] Functions [1]: [count(1)] -(128) ColumnarToRow [codegen id : 5] +(128) CometColumnarToRow [codegen id : 5] Input [1]: [h11_to_11_30#69] (129) BroadcastExchange @@ -823,7 +823,7 @@ Input [1]: [count#79] Keys: [] Functions [1]: [count(1)] -(149) ColumnarToRow [codegen id : 6] +(149) CometColumnarToRow [codegen id : 6] Input [1]: [h11_30_to_12#80] (150) BroadcastExchange @@ -914,7 +914,7 @@ Input [1]: [count#90] Keys: [] Functions [1]: [count(1)] -(170) ColumnarToRow [codegen id : 7] +(170) CometColumnarToRow [codegen id : 7] Input [1]: [h12_to_12_30#91] (171) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt index 1bb61b6c92..21b72d36a7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt @@ -6,7 +6,7 @@ WholeStageCodegen (8) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h8_30_to_9,count,count(1)] CometExchange #1 @@ -35,7 +35,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h9_to_9_30,count,count(1)] CometExchange #6 @@ -58,7 +58,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #8 WholeStageCodegen (2) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h9_30_to_10,count,count(1)] CometExchange #9 @@ -81,7 +81,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #11 WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h10_to_10_30,count,count(1)] CometExchange #12 @@ -104,7 +104,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #14 WholeStageCodegen (4) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h10_30_to_11,count,count(1)] CometExchange #15 @@ -127,7 +127,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #17 WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h11_to_11_30,count,count(1)] CometExchange #18 @@ -150,7 +150,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #20 WholeStageCodegen (6) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h11_30_to_12,count,count(1)] CometExchange #21 @@ -173,7 +173,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #23 WholeStageCodegen (7) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h12_to_12_30,count,count(1)] CometExchange #24 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt index 143ff08683..8346f9b848 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (28) +- * Project (27) +- * Filter (26) +- Window (25) - +- * ColumnarToRow (24) + +- * CometColumnarToRow (24) +- CometSort (23) +- CometExchange (22) +- CometHashAggregate (21) @@ -139,7 +139,7 @@ Arguments: hashpartitioning(i_category#4, i_brand#2, s_store_name#14, s_company_ Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#17, _w0#18] Arguments: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#17, _w0#18], [i_category#4 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST, s_company_name#15 ASC NULLS FIRST] -(24) ColumnarToRow [codegen id : 1] +(24) CometColumnarToRow [codegen id : 1] Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#17, _w0#18] (25) Window @@ -162,7 +162,7 @@ Arguments: 100, [(sum_sales#17 - avg_monthly_sales#19) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (33) -+- * ColumnarToRow (32) ++- * CometColumnarToRow (32) +- CometProject (31) +- CometFilter (30) +- CometScan parquet spark_catalog.default.date_dim (29) @@ -183,7 +183,7 @@ Condition : ((isnotnull(d_year#11) AND (d_year#11 = 1999)) AND isnotnull(d_date_ Input [3]: [d_date_sk#10, d_year#11, d_moy#12] Arguments: [d_date_sk#10, d_moy#12], [d_date_sk#10, d_moy#12] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_moy#12] (33) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt index 97b9563be6..58bb7e979e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 @@ -26,7 +26,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt index 7cc8c153c4..d2061bd1d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == * Project (4) -+- * ColumnarToRow (3) ++- * CometColumnarToRow (3) +- CometFilter (2) +- CometScan parquet spark_catalog.default.reason (1) @@ -16,7 +16,7 @@ ReadSchema: struct Input [1]: [r_reason_sk#1] Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) -(3) ColumnarToRow [codegen id : 1] +(3) CometColumnarToRow [codegen id : 1] Input [1]: [r_reason_sk#1] (4) Project [codegen id : 1] @@ -26,7 +26,7 @@ Input [1]: [r_reason_sk#1] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#2, [id=#3] -* ColumnarToRow (12) +* CometColumnarToRow (12) +- CometProject (11) +- CometHashAggregate (10) +- CometExchange (9) @@ -69,7 +69,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(Unscal Input [3]: [count(1)#26, avg(ss_ext_discount_amt)#27, avg(ss_net_paid)#28] Arguments: [mergedValue#29], [named_struct(count(1), count(1)#26, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#27, avg(ss_net_paid), avg(ss_net_paid)#28) AS mergedValue#29] -(12) ColumnarToRow [codegen id : 1] +(12) CometColumnarToRow [codegen id : 1] Input [1]: [mergedValue#29] Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] @@ -77,7 +77,7 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] -* ColumnarToRow (20) +* CometColumnarToRow (20) +- CometProject (19) +- CometHashAggregate (18) +- CometExchange (17) @@ -120,7 +120,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#31)), avg(Unscal Input [3]: [count(1)#39, avg(ss_ext_discount_amt)#40, avg(ss_net_paid)#41] Arguments: [mergedValue#42], [named_struct(count(1), count(1)#39, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#40, avg(ss_net_paid), avg(ss_net_paid)#41) AS mergedValue#42] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarToRow [codegen id : 1] Input [1]: [mergedValue#42] Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] @@ -128,7 +128,7 @@ Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* ColumnarToRow (28) +* CometColumnarToRow (28) +- CometProject (27) +- CometHashAggregate (26) +- CometExchange (25) @@ -171,7 +171,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#44)), avg(Unscal Input [3]: [count(1)#52, avg(ss_ext_discount_amt)#53, avg(ss_net_paid)#54] Arguments: [mergedValue#55], [named_struct(count(1), count(1)#52, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#53, avg(ss_net_paid), avg(ss_net_paid)#54) AS mergedValue#55] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [mergedValue#55] Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] @@ -179,7 +179,7 @@ Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* ColumnarToRow (36) +* CometColumnarToRow (36) +- CometProject (35) +- CometHashAggregate (34) +- CometExchange (33) @@ -222,7 +222,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#57)), avg(Unscal Input [3]: [count(1)#65, avg(ss_ext_discount_amt)#66, avg(ss_net_paid)#67] Arguments: [mergedValue#68], [named_struct(count(1), count(1)#65, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#66, avg(ss_net_paid), avg(ss_net_paid)#67) AS mergedValue#68] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [1]: [mergedValue#68] Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] @@ -230,7 +230,7 @@ Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] -* ColumnarToRow (44) +* CometColumnarToRow (44) +- CometProject (43) +- CometHashAggregate (42) +- CometExchange (41) @@ -273,7 +273,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#70)), avg(Unscal Input [3]: [count(1)#78, avg(ss_ext_discount_amt)#79, avg(ss_net_paid)#80] Arguments: [mergedValue#81], [named_struct(count(1), count(1)#78, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#79, avg(ss_net_paid), avg(ss_net_paid)#80) AS mergedValue#81] -(44) ColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [mergedValue#81] Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt index 3d69c60d08..68961382c0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) Project Subquery #1 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] CometHashAggregate [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count,count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] @@ -15,7 +15,7 @@ WholeStageCodegen (1) ReusedSubquery [mergedValue] #1 Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] CometHashAggregate [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count,count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] @@ -28,7 +28,7 @@ WholeStageCodegen (1) ReusedSubquery [mergedValue] #2 Subquery #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] CometHashAggregate [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count,count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] @@ -41,7 +41,7 @@ WholeStageCodegen (1) ReusedSubquery [mergedValue] #3 Subquery #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] CometHashAggregate [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count,count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] @@ -54,7 +54,7 @@ WholeStageCodegen (1) ReusedSubquery [mergedValue] #4 Subquery #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] CometHashAggregate [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count,count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] @@ -65,7 +65,7 @@ WholeStageCodegen (1) CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [r_reason_sk] CometScan parquet spark_catalog.default.reason [r_reason_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt index a6ec6f4b96..38379fb61c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * Project (47) +- * BroadcastNestedLoopJoin Inner BuildRight (46) - :- * ColumnarToRow (25) + :- * CometColumnarToRow (25) : +- CometHashAggregate (24) : +- CometExchange (23) : +- CometHashAggregate (22) @@ -27,7 +27,7 @@ : +- CometFilter (17) : +- CometScan parquet spark_catalog.default.web_page (16) +- BroadcastExchange (45) - +- * ColumnarToRow (44) + +- * CometColumnarToRow (44) +- CometHashAggregate (43) +- CometExchange (42) +- CometHashAggregate (41) @@ -160,7 +160,7 @@ Input [1]: [count#11] Keys: [] Functions [1]: [count(1)] -(25) ColumnarToRow [codegen id : 2] +(25) CometColumnarToRow [codegen id : 2] Input [1]: [amc#12] (26) CometScan parquet spark_catalog.default.web_sales @@ -243,7 +243,7 @@ Input [1]: [count#21] Keys: [] Functions [1]: [count(1)] -(44) ColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [pmc#22] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt index 95fd73d86e..72e45a9ad1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (2) Project [amc,pmc] BroadcastNestedLoopJoin - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [amc,count,count(1)] CometExchange #1 @@ -30,7 +30,7 @@ WholeStageCodegen (2) InputAdapter BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [pmc,count,count(1)] CometExchange #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt index ede0953066..26272decdb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (41) +* CometColumnarToRow (41) +- CometSort (40) +- CometColumnarExchange (39) +- CometHashAggregate (38) @@ -232,14 +232,14 @@ Arguments: rangepartitioning(Returns_Loss#28 DESC NULLS LAST, 5), ENSURE_REQUIRE Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] Arguments: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28], [Returns_Loss#28 DESC NULLS LAST] -(41) ColumnarToRow [codegen id : 1] +(41) CometColumnarToRow [codegen id : 1] Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = cr_returned_date_sk#8 IN dynamicpruning#9 BroadcastExchange (46) -+- * ColumnarToRow (45) ++- * CometColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometScan parquet spark_catalog.default.date_dim (42) @@ -260,7 +260,7 @@ Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#12)) AND (d_year#11 = 1 Input [3]: [d_date_sk#10, d_year#11, d_moy#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(45) ColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt index 0dc197b1b4..1847051c7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] CometColumnarExchange [Returns_Loss] #1 @@ -26,7 +26,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt index 93bb03fae1..25045775b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (30) +* CometColumnarToRow (30) +- CometHashAggregate (29) +- CometExchange (28) +- CometHashAggregate (27) @@ -168,14 +168,14 @@ Input [1]: [sum#17] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [1]: [Excess Discount Amount #18] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (35) -+- * ColumnarToRow (34) ++- * CometColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan parquet spark_catalog.default.date_dim (31) @@ -196,7 +196,7 @@ Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#1 Input [2]: [d_date_sk#16, d_date#19] Arguments: [d_date_sk#16], [d_date_sk#16] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt index f49dd1ea30..4ed5b0b8ef 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [Excess Discount Amount ,sum,sum(UnscaledValue(ws_ext_discount_amt))] CometExchange #1 @@ -15,7 +15,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt index dc64f3c4ce..ae826e9288 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (22) +* CometColumnarToRow (22) +- CometTakeOrderedAndProject (21) +- CometHashAggregate (20) +- CometExchange (19) @@ -119,6 +119,6 @@ Functions [1]: [sum(act_sales#14)] Input [2]: [ss_customer_sk#2, sumsales#17] Arguments: TakeOrderedAndProject(limit=100, orderBy=[sumsales#17 ASC NULLS FIRST,ss_customer_sk#2 ASC NULLS FIRST], output=[ss_customer_sk#2,sumsales#17]), [ss_customer_sk#2, sumsales#17], 100, [sumsales#17 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#17] -(22) ColumnarToRow [codegen id : 1] +(22) CometColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#2, sumsales#17] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt index 6795d7e399..9580e56671 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [ss_customer_sk,sumsales] CometHashAggregate [ss_customer_sk,sumsales,sum,isEmpty,sum(act_sales)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt index 4dd9246cd7..338558d002 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt @@ -3,7 +3,7 @@ +- Exchange (39) +- * HashAggregate (38) +- * HashAggregate (37) - +- * ColumnarToRow (36) + +- * CometColumnarToRow (36) +- CometHashAggregate (35) +- CometProject (34) +- CometBroadcastHashJoin (33) @@ -203,7 +203,7 @@ Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [3]: [ws_order_number#5, sum#20, sum#21] (37) HashAggregate [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt index 601f577da9..8b7b457074 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt @@ -5,7 +5,7 @@ WholeStageCodegen (2) WholeStageCodegen (1) HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [ws_order_number,sum,sum,ws_ext_ship_cost,ws_net_profit] CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt index 2e951a408c..88c16efa42 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt @@ -3,7 +3,7 @@ +- Exchange (52) +- * HashAggregate (51) +- * HashAggregate (50) - +- * ColumnarToRow (49) + +- * CometColumnarToRow (49) +- CometHashAggregate (48) +- CometProject (47) +- CometBroadcastHashJoin (46) @@ -270,7 +270,7 @@ Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Keys [1]: [ws_order_number#4] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] -(49) ColumnarToRow [codegen id : 1] +(49) CometColumnarToRow [codegen id : 1] Input [3]: [ws_order_number#4, sum#21, sum#22] (50) HashAggregate [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt index 168f353a7b..021211d58f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt @@ -5,7 +5,7 @@ WholeStageCodegen (2) WholeStageCodegen (1) HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [ws_order_number,sum,sum,ws_ext_ship_cost,ws_net_profit] CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/explain.txt index d2e63bee25..677abdadae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (25) +* CometColumnarToRow (25) +- CometHashAggregate (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -138,6 +138,6 @@ Input [1]: [count#12] Keys: [] Functions [1]: [count(1)] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [1]: [count(1)#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt index d9a87aa3c4..9e86a025ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [count(1),count,count(1)] CometExchange #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt index 1c6e9b78c0..f23cf90e66 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (25) +* CometColumnarToRow (25) +- CometHashAggregate (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -139,14 +139,14 @@ Input [3]: [sum#16, sum#17, sum#18] Keys: [] Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [3]: [store_only#19, catalog_only#20, store_and_catalog#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (30) -+- * ColumnarToRow (29) ++- * CometColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan parquet spark_catalog.default.date_dim (26) @@ -167,7 +167,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(29) ColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt index 0036a4bd84..aa5be145e9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [store_only,catalog_only,store_and_catalog,sum,sum,sum,sum(CASE WHEN (isnotnull(customer_sk) AND isnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnotnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END)] CometExchange #1 @@ -16,7 +16,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt index 593065ba0b..8c2521b90c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt @@ -1,11 +1,11 @@ == Physical Plan == -* ColumnarToRow (25) +* CometColumnarToRow (25) +- CometProject (24) +- CometSort (23) +- CometColumnarExchange (22) +- * Project (21) +- Window (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometSort (18) +- CometExchange (17) +- CometHashAggregate (16) @@ -112,7 +112,7 @@ Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometNativeShuff Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6], [i_class#9 ASC NULLS FIRST] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] (20) Window @@ -135,14 +135,14 @@ Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemreve Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, i_item_id#6] Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] -(25) ColumnarToRow [codegen id : 3] +(25) CometColumnarToRow [codegen id : 3] Input [6]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (30) -+- * ColumnarToRow (29) ++- * CometColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan parquet spark_catalog.default.date_dim (26) @@ -163,7 +163,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(29) ColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt index 01aa117344..a77e84b60b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] @@ -9,7 +9,7 @@ WholeStageCodegen (3) InputAdapter Window [_w0,i_class] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] CometExchange [i_class] #2 @@ -25,7 +25,7 @@ WholeStageCodegen (3) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt index 6dfcf8b322..b4c43d5d01 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (28) +* CometColumnarToRow (28) +- CometTakeOrderedAndProject (27) +- CometHashAggregate (26) +- CometExchange (25) @@ -158,6 +158,6 @@ Functions [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) TH Input [8]: [substr(w_warehouse_name, 1, 20)#20, sm_type#9, cc_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25] Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#20 ASC NULLS FIRST,sm_type#9 ASC NULLS FIRST,cc_name#11 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#20,sm_type#9,cc_name#11,30 days #21,31 - 60 days #22,61 - 90 days #23,91 - 120 days #24,>120 days #25]), [substr(w_warehouse_name, 1, 20)#20, sm_type#9, cc_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25], 100, [substr(w_warehouse_name, 1, 20)#20 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#20, sm_type#9, cc_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [8]: [substr(w_warehouse_name, 1, 20)#20, sm_type#9, cc_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt index 51599575db..e112972b12 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] CometHashAggregate [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,sum,sum,sum,sum,sum,sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 30) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 60) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 90) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) > 120) THEN 1 ELSE 0 END)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/explain.txt index 868b2f4822..c2b7f49205 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (39) +* CometColumnarToRow (39) +- CometTakeOrderedAndProject (38) +- CometHashAggregate (37) +- CometExchange (36) @@ -220,14 +220,14 @@ Functions [1]: [count(1)] Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34, cd_dep_count#28, cnt4#35, cd_dep_employed_count#29, cnt5#36, cd_dep_college_count#30, cnt6#37] Arguments: TakeOrderedAndProject(limit=100, orderBy=[cd_gender#23 ASC NULLS FIRST,cd_marital_status#24 ASC NULLS FIRST,cd_education_status#25 ASC NULLS FIRST,cd_purchase_estimate#26 ASC NULLS FIRST,cd_credit_rating#27 ASC NULLS FIRST,cd_dep_count#28 ASC NULLS FIRST,cd_dep_employed_count#29 ASC NULLS FIRST,cd_dep_college_count#30 ASC NULLS FIRST], output=[cd_gender#23,cd_marital_status#24,cd_education_status#25,cnt1#32,cd_purchase_estimate#26,cnt2#33,cd_credit_rating#27,cnt3#34,cd_dep_count#28,cnt4#35,cd_dep_employed_count#29,cnt5#36,cd_dep_college_count#30,cnt6#37]), [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34, cd_dep_count#28, cnt4#35, cd_dep_employed_count#29, cnt5#36, cd_dep_college_count#30, cnt6#37], 100, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34, cd_dep_count#28, cnt4#35, cd_dep_employed_count#29, cnt5#36, cd_dep_college_count#30, cnt6#37] -(39) ColumnarToRow [codegen id : 1] +(39) CometColumnarToRow [codegen id : 1] Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34, cd_dep_count#28, cnt4#35, cd_dep_employed_count#29, cnt5#36, cd_dep_college_count#30, cnt6#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (44) -+- * ColumnarToRow (43) ++- * CometColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan parquet spark_catalog.default.date_dim (40) @@ -248,7 +248,7 @@ Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 200 Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(43) ColumnarToRow [codegen id : 1] +(43) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt index 2e8f2022cf..b570b700ac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6] CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6,count,count(1)] @@ -21,7 +21,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/explain.txt index 689697306d..0bcd9798e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (68) +* CometColumnarToRow (68) +- CometTakeOrderedAndProject (67) +- CometProject (66) +- CometBroadcastHashJoin (65) @@ -388,14 +388,14 @@ Arguments: [customer_id#35, customer_first_name#36, customer_last_name#37, custo Input [4]: [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38] Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#35 ASC NULLS FIRST,customer_first_name#36 ASC NULLS FIRST,customer_last_name#37 ASC NULLS FIRST,customer_email_address#38 ASC NULLS FIRST], output=[customer_id#35,customer_first_name#36,customer_last_name#37,customer_email_address#38]), [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38], 100, [customer_id#35 ASC NULLS FIRST, customer_first_name#36 ASC NULLS FIRST, customer_last_name#37 ASC NULLS FIRST, customer_email_address#38 ASC NULLS FIRST], [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38] -(68) ColumnarToRow [codegen id : 1] +(68) CometColumnarToRow [codegen id : 1] Input [4]: [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (72) -+- * ColumnarToRow (71) ++- * CometColumnarToRow (71) +- CometFilter (70) +- CometScan parquet spark_catalog.default.date_dim (69) @@ -411,7 +411,7 @@ ReadSchema: struct Input [2]: [d_date_sk#14, d_year#15] Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#14)) -(71) ColumnarToRow [codegen id : 1] +(71) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] (72) BroadcastExchange @@ -420,7 +420,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#31 BroadcastExchange (76) -+- * ColumnarToRow (75) ++- * CometColumnarToRow (75) +- CometFilter (74) +- CometScan parquet spark_catalog.default.date_dim (73) @@ -436,7 +436,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 2002)) AND isnotnull(d_date_sk#32)) -(75) ColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] (76) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt index 8743dd42ef..987c965c03 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] CometProject [customer_id,customer_first_name,customer_last_name,customer_email_address] @@ -23,7 +23,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -46,7 +46,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/explain.txt index a428886298..b8d68b0a31 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (22) +- * Project (21) +- Window (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometSort (18) +- CometExchange (17) +- CometHashAggregate (16) @@ -109,7 +109,7 @@ Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometNativeShuff Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15], [i_class#9 ASC NULLS FIRST] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] (20) Window @@ -128,7 +128,7 @@ Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_ite Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (27) -+- * ColumnarToRow (26) ++- * CometColumnarToRow (26) +- CometProject (25) +- CometFilter (24) +- CometScan parquet spark_catalog.default.date_dim (23) @@ -149,7 +149,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (27) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt index 6f51be0c5b..4a829e8612 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] CometExchange [i_class] #1 @@ -20,7 +20,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/explain.txt index 130726664c..a315b85659 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (85) +* CometColumnarToRow (85) +- CometTakeOrderedAndProject (84) +- CometBroadcastHashJoin (83) :- CometFilter (64) @@ -480,13 +480,13 @@ Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#62, i_c Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#50,i_brand_id#40,i_class_id#41,i_category_id#42,sales#51,number_sales#52,channel#72,i_brand_id#62,i_class_id#63,i_category_id#64,sales#73,number_sales#74]), [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74], 100, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -(85) ColumnarToRow [codegen id : 1] +(85) CometColumnarToRow [codegen id : 1] Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] ===== Subqueries ===== Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* ColumnarToRow (102) +* CometColumnarToRow (102) +- CometHashAggregate (101) +- CometExchange (100) +- CometHashAggregate (99) @@ -581,7 +581,7 @@ Input [2]: [sum#96, count#97] Keys: [] Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] -(102) ColumnarToRow [codegen id : 1] +(102) CometColumnarToRow [codegen id : 1] Input [1]: [average_sales#98] Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 @@ -592,7 +592,7 @@ Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (107) -+- * ColumnarToRow (106) ++- * CometColumnarToRow (106) +- CometProject (105) +- CometFilter (104) +- CometScan parquet spark_catalog.default.date_dim (103) @@ -613,7 +613,7 @@ Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subq Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(106) ColumnarToRow [codegen id : 1] +(106) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] (107) BroadcastExchange @@ -621,7 +621,7 @@ Input [1]: [d_date_sk#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* ColumnarToRow (111) +* CometColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometScan parquet spark_catalog.default.date_dim (108) @@ -642,12 +642,12 @@ Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] Arguments: [d_week_seq#99], [d_week_seq#99] -(111) ColumnarToRow [codegen id : 1] +(111) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#99] Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (116) -+- * ColumnarToRow (115) ++- * CometColumnarToRow (115) +- CometProject (114) +- CometFilter (113) +- CometScan parquet spark_catalog.default.date_dim (112) @@ -668,7 +668,7 @@ Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1998)) AND (d_year#103 < Input [2]: [d_date_sk#26, d_year#103] Arguments: [d_date_sk#26], [d_date_sk#26] -(115) ColumnarToRow [codegen id : 1] +(115) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] (116) BroadcastExchange @@ -685,7 +685,7 @@ Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquer Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 BroadcastExchange (121) -+- * ColumnarToRow (120) ++- * CometColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan parquet spark_catalog.default.date_dim (117) @@ -706,7 +706,7 @@ Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = Subquery scalar-subq Input [2]: [d_date_sk#65, d_week_seq#66] Arguments: [d_date_sk#65], [d_date_sk#65] -(120) ColumnarToRow [codegen id : 1] +(120) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#65] (121) BroadcastExchange @@ -714,7 +714,7 @@ Input [1]: [d_date_sk#65] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#67, [id=#68] -* ColumnarToRow (125) +* CometColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometScan parquet spark_catalog.default.date_dim (122) @@ -735,7 +735,7 @@ Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] Arguments: [d_week_seq#104], [d_week_seq#104] -(125) ColumnarToRow [codegen id : 1] +(125) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#104] Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt index bb4be4d3ae..befc6dadbb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt @@ -1,12 +1,12 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] CometBroadcastHashJoin [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Subquery #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #14 @@ -40,13 +40,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_week_seq] Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_week_seq,d_year,d_moy,d_dom] @@ -71,7 +71,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -132,13 +132,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_week_seq] Subquery #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_week_seq,d_year,d_moy,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/explain.txt index c5c211461a..9dd52b68f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (122) +* CometColumnarToRow (122) +- CometTakeOrderedAndProject (121) +- CometHashAggregate (120) +- CometExchange (119) @@ -682,13 +682,13 @@ Functions: [] Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#49 ASC NULLS FIRST,i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#49,i_brand_id#40,i_class_id#41,i_category_id#42,sum_sales#123,number_sales#124]), [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124], 100, [channel#49 ASC NULLS FIRST, i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -(122) ColumnarToRow [codegen id : 1] +(122) CometColumnarToRow [codegen id : 1] Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] ===== Subqueries ===== Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* ColumnarToRow (142) +* CometColumnarToRow (142) +- CometHashAggregate (141) +- CometExchange (140) +- CometHashAggregate (139) @@ -802,14 +802,14 @@ Input [2]: [sum#165, count#166] Keys: [] Functions [1]: [avg((cast(quantity#148 as decimal(10,0)) * list_price#149))] -(142) ColumnarToRow [codegen id : 1] +(142) CometColumnarToRow [codegen id : 1] Input [1]: [average_sales#167] Subquery:2 Hosting operator id = 123 Hosting Expression = ss_sold_date_sk#145 IN dynamicpruning#12 Subquery:3 Hosting operator id = 127 Hosting Expression = cs_sold_date_sk#152 IN dynamicpruning#153 BroadcastExchange (147) -+- * ColumnarToRow (146) ++- * CometColumnarToRow (146) +- CometProject (145) +- CometFilter (144) +- CometScan parquet spark_catalog.default.date_dim (143) @@ -830,7 +830,7 @@ Condition : (((isnotnull(d_year#155) AND (d_year#155 >= 1998)) AND (d_year#155 < Input [2]: [d_date_sk#154, d_year#155] Arguments: [d_date_sk#154], [d_date_sk#154] -(146) ColumnarToRow [codegen id : 1] +(146) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#154] (147) BroadcastExchange @@ -841,7 +841,7 @@ Subquery:4 Hosting operator id = 134 Hosting Expression = ws_sold_date_sk#160 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (152) -+- * ColumnarToRow (151) ++- * CometColumnarToRow (151) +- CometProject (150) +- CometFilter (149) +- CometScan parquet spark_catalog.default.date_dim (148) @@ -862,7 +862,7 @@ Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2 Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(151) ColumnarToRow [codegen id : 1] +(151) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] (152) BroadcastExchange @@ -871,7 +871,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (157) -+- * ColumnarToRow (156) ++- * CometColumnarToRow (156) +- CometProject (155) +- CometFilter (154) +- CometScan parquet spark_catalog.default.date_dim (153) @@ -892,7 +892,7 @@ Condition : (((isnotnull(d_year#168) AND (d_year#168 >= 1999)) AND (d_year#168 < Input [2]: [d_date_sk#26, d_year#168] Arguments: [d_date_sk#26], [d_date_sk#26] -(156) ColumnarToRow [codegen id : 1] +(156) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] (157) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt index 760d0963ed..162f4f2447 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] @@ -13,7 +13,7 @@ WholeStageCodegen (1) CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Subquery #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #16 @@ -30,7 +30,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -57,7 +57,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -81,7 +81,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/explain.txt index e227c81b82..8eeb8487de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (143) +* CometColumnarToRow (143) +- CometTakeOrderedAndProject (142) +- CometUnion (141) :- CometHashAggregate (38) @@ -785,14 +785,14 @@ Child 4 Input [11]: [i_item_id#256, ca_country#257, ca_state#258, county#259, ag Input [11]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#222, agg2#223, agg3#224, agg4#225, agg5#226, agg6#227, agg7#228] Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#24 ASC NULLS FIRST,ca_state#23 ASC NULLS FIRST,ca_county#22 ASC NULLS FIRST,i_item_id#28 ASC NULLS FIRST], output=[i_item_id#28,ca_country#24,ca_state#23,ca_county#22,agg1#222,agg2#223,agg3#224,agg4#225,agg5#226,agg6#227,agg7#228]), [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#222, agg2#223, agg3#224, agg4#225, agg5#226, agg6#227, agg7#228], 100, [ca_country#24 ASC NULLS FIRST, ca_state#23 ASC NULLS FIRST, ca_county#22 ASC NULLS FIRST, i_item_id#28 ASC NULLS FIRST], [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#222, agg2#223, agg3#224, agg4#225, agg5#226, agg6#227, agg7#228] -(143) ColumnarToRow [codegen id : 1] +(143) CometColumnarToRow [codegen id : 1] Input [11]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#222, agg2#223, agg3#224, agg4#225, agg5#226, agg6#227, agg7#228] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (148) -+- * ColumnarToRow (147) ++- * CometColumnarToRow (147) +- CometProject (146) +- CometFilter (145) +- CometScan parquet spark_catalog.default.date_dim (144) @@ -813,7 +813,7 @@ Condition : ((isnotnull(d_year#26) AND (d_year#26 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(147) ColumnarToRow [codegen id : 1] +(147) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] (148) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt index add76afad3..b974a7afce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometUnion [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] @@ -23,7 +23,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/explain.txt index a0c02e00e9..0610ff5ce3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (22) +- * Project (21) +- Window (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometSort (18) +- CometExchange (17) +- CometHashAggregate (16) @@ -109,7 +109,7 @@ Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometNativeShuff Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15], [i_class#9 ASC NULLS FIRST] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] (20) Window @@ -128,7 +128,7 @@ Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_ite Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (27) -+- * ColumnarToRow (26) ++- * CometColumnarToRow (26) +- CometProject (25) +- CometFilter (24) +- CometScan parquet spark_catalog.default.date_dim (23) @@ -149,7 +149,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (27) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt index c4d5e48ae5..8e639b6fa0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] CometExchange [i_class] #1 @@ -20,7 +20,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/explain.txt index 34d510fc71..5be661c351 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/explain.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject (24) +- * Expand (20) +- * Project (19) +- * BroadcastNestedLoopJoin Inner BuildRight (18) - :- * ColumnarToRow (14) + :- * CometColumnarToRow (14) : +- CometProject (13) : +- CometBroadcastHashJoin (12) : :- CometProject (8) @@ -21,7 +21,7 @@ TakeOrderedAndProject (24) : +- CometFilter (10) : +- CometScan parquet spark_catalog.default.item (9) +- BroadcastExchange (17) - +- * ColumnarToRow (16) + +- * CometColumnarToRow (16) +- CometScan parquet spark_catalog.default.warehouse (15) @@ -89,7 +89,7 @@ Arguments: [inv_item_sk#1], [i_item_sk#7], Inner, BuildRight Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] Arguments: [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11], [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11] -(14) ColumnarToRow [codegen id : 2] +(14) CometColumnarToRow [codegen id : 2] Input [5]: [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11] (15) CometScan parquet spark_catalog.default.warehouse @@ -98,7 +98,7 @@ Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] ReadSchema: struct<> -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarToRow [codegen id : 1] Input: [] (17) BroadcastExchange @@ -143,7 +143,7 @@ Arguments: 100, [qoh#22 ASC NULLS FIRST, i_product_name#12 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * ColumnarToRow (28) ++- * CometColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan parquet spark_catalog.default.date_dim (25) @@ -164,7 +164,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt index 0e864ab701..027836b68c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] BroadcastNestedLoopJoin - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] @@ -19,7 +19,7 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -34,6 +34,6 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometScan parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/explain.txt index deae565119..4457b74a5a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (46) +* CometColumnarToRow (46) +- CometTakeOrderedAndProject (45) +- CometUnion (44) :- CometHashAggregate (23) @@ -258,14 +258,14 @@ Child 4 Input [5]: [i_product_name#69, i_brand#70, i_class#71, i_category#72, qo Input [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#59] Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#59 ASC NULLS FIRST,i_product_name#12 ASC NULLS FIRST,i_brand#9 ASC NULLS FIRST,i_class#10 ASC NULLS FIRST,i_category#11 ASC NULLS FIRST], output=[i_product_name#12,i_brand#9,i_class#10,i_category#11,qoh#59]), [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#59], 100, [qoh#59 ASC NULLS FIRST, i_product_name#12 ASC NULLS FIRST, i_brand#9 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#59] -(46) ColumnarToRow [codegen id : 1] +(46) CometColumnarToRow [codegen id : 1] Input [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#59] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (51) -+- * ColumnarToRow (50) ++- * CometColumnarToRow (50) +- CometProject (49) +- CometFilter (48) +- CometScan parquet spark_catalog.default.date_dim (47) @@ -286,7 +286,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_mont Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (51) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt index 63eda27a6b..02f615b128 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] CometUnion [i_product_name,i_brand,i_class,i_category,qoh] @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/explain.txt index 522e028b5a..165a4b6099 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (45) +* CometColumnarToRow (45) +- CometSort (44) +- CometColumnarExchange (43) +- * Filter (42) @@ -11,7 +11,7 @@ +- * HashAggregate (36) +- * Project (35) +- * BroadcastHashJoin Inner BuildRight (34) - :- * ColumnarToRow (29) + :- * CometColumnarToRow (29) : +- CometProject (28) : +- CometBroadcastHashJoin (27) : :- CometProject (23) @@ -41,7 +41,7 @@ : +- CometFilter (25) : +- CometScan parquet spark_catalog.default.customer (24) +- BroadcastExchange (33) - +- * ColumnarToRow (32) + +- * CometColumnarToRow (32) +- CometFilter (31) +- CometScan parquet spark_catalog.default.customer_address (30) @@ -177,7 +177,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] Arguments: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25], [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -(29) ColumnarToRow [codegen id : 2] +(29) CometColumnarToRow [codegen id : 2] Input [13]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] (30) CometScan parquet spark_catalog.default.customer_address @@ -191,7 +191,7 @@ ReadSchema: struct= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_do Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt index 7da6076f5b..e1fc6bf43f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/explain.txt index 5933ea5fad..fbf3086106 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/explain.txt @@ -11,7 +11,7 @@ TakeOrderedAndProject (44) : : +- * Filter (27) : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * ColumnarToRow (12) + : : : :- * CometColumnarToRow (12) : : : : +- CometBroadcastHashJoin (11) : : : : :- CometFilter (2) : : : : : +- CometScan parquet spark_catalog.default.customer (1) @@ -24,23 +24,23 @@ TakeOrderedAndProject (44) : : : : +- CometFilter (5) : : : : +- CometScan parquet spark_catalog.default.date_dim (4) : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) + : : : +- * CometColumnarToRow (17) : : : +- CometProject (16) : : : +- CometBroadcastHashJoin (15) : : : :- CometScan parquet spark_catalog.default.web_sales (13) : : : +- ReusedExchange (14) : : +- BroadcastExchange (25) - : : +- * ColumnarToRow (24) + : : +- * CometColumnarToRow (24) : : +- CometProject (23) : : +- CometBroadcastHashJoin (22) : : :- CometScan parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (32) - : +- * ColumnarToRow (31) + : +- * CometColumnarToRow (31) : +- CometFilter (30) : +- CometScan parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (38) - +- * ColumnarToRow (37) + +- * CometColumnarToRow (37) +- CometFilter (36) +- CometScan parquet spark_catalog.default.customer_demographics (35) @@ -100,7 +100,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) ColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 5] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan parquet spark_catalog.default.web_sales @@ -122,7 +122,7 @@ Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#12] (18) BroadcastExchange @@ -154,7 +154,7 @@ Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [1]: [cs_ship_customer_sk#16] (25) BroadcastExchange @@ -186,7 +186,7 @@ ReadSchema: struct Input [2]: [ca_address_sk#20, ca_state#21] Condition : isnotnull(ca_address_sk#20) -(31) ColumnarToRow [codegen id : 3] +(31) CometColumnarToRow [codegen id : 3] Input [2]: [ca_address_sk#20, ca_state#21] (32) BroadcastExchange @@ -214,7 +214,7 @@ ReadSchema: struct Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (51) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt index a18f68d837..6355321c9b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 @@ -34,7 +34,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] @@ -51,7 +51,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 @@ -64,7 +64,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/explain.txt index 685f048dea..fbd4aea034 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/explain.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject (77) : +- Window (24) : +- * Sort (23) : +- Window (22) - : +- * ColumnarToRow (21) + : +- * CometColumnarToRow (21) : +- CometSort (20) : +- CometExchange (19) : +- CometHashAggregate (18) @@ -35,7 +35,7 @@ TakeOrderedAndProject (77) : +- Window (47) : +- * Sort (46) : +- Window (45) - : +- * ColumnarToRow (44) + : +- * CometColumnarToRow (44) : +- CometSort (43) : +- CometExchange (42) : +- CometHashAggregate (41) @@ -58,7 +58,7 @@ TakeOrderedAndProject (77) +- Window (70) +- * Sort (69) +- Window (68) - +- * ColumnarToRow (67) + +- * CometColumnarToRow (67) +- CometSort (66) +- CometExchange (65) +- CometHashAggregate (64) @@ -172,7 +172,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [3]: [item#22, return_ratio#23, currency_ratio#24] Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) ColumnarToRow [codegen id : 1] +(21) CometColumnarToRow [codegen id : 1] Input [3]: [item#22, return_ratio#23, currency_ratio#24] (22) Window @@ -273,7 +273,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] -(44) ColumnarToRow [codegen id : 4] +(44) CometColumnarToRow [codegen id : 4] Input [3]: [item#47, return_ratio#48, currency_ratio#49] (45) Window @@ -374,7 +374,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] -(67) ColumnarToRow [codegen id : 7] +(67) CometColumnarToRow [codegen id : 7] Input [3]: [item#72, return_ratio#73, currency_ratio#74] (68) Window @@ -425,7 +425,7 @@ Arguments: 100, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, cur Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (82) -+- * ColumnarToRow (81) ++- * CometColumnarToRow (81) +- CometProject (80) +- CometFilter (79) +- CometScan parquet spark_catalog.default.date_dim (78) @@ -446,7 +446,7 @@ Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2 Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(81) ColumnarToRow [codegen id : 1] +(81) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] (82) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt index d98b2b0a66..aaf75c375c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] InputAdapter Window [return_ratio] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item,return_ratio,currency_ratio] CometExchange #2 @@ -35,7 +35,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -57,7 +57,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] InputAdapter Window [return_ratio] WholeStageCodegen (4) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item,return_ratio,currency_ratio] CometExchange #7 @@ -87,7 +87,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] InputAdapter Window [return_ratio] WholeStageCodegen (7) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item,return_ratio,currency_ratio] CometExchange #10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/explain.txt index 3211e46f6d..725868a338 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/explain.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject (72) : : +- * BroadcastHashJoin Inner BuildRight (23) : : :- * Project (16) : : : +- Window (15) - : : : +- * ColumnarToRow (14) + : : : +- * CometColumnarToRow (14) : : : +- CometSort (13) : : : +- CometExchange (12) : : : +- CometHashAggregate (11) @@ -37,7 +37,7 @@ TakeOrderedAndProject (72) : : +- BroadcastExchange (22) : : +- * Project (21) : : +- Window (20) - : : +- * ColumnarToRow (19) + : : +- * CometColumnarToRow (19) : : +- CometSort (18) : : +- ReusedExchange (17) : +- * Sort (55) @@ -49,7 +49,7 @@ TakeOrderedAndProject (72) : +- * BroadcastHashJoin Inner BuildRight (49) : :- * Project (42) : : +- Window (41) - : : +- * ColumnarToRow (40) + : : +- * CometColumnarToRow (40) : : +- CometSort (39) : : +- CometExchange (38) : : +- CometHashAggregate (37) @@ -63,7 +63,7 @@ TakeOrderedAndProject (72) : +- BroadcastExchange (48) : +- * Project (47) : +- Window (46) - : +- * ColumnarToRow (45) + : +- * CometColumnarToRow (45) : +- CometSort (44) : +- ReusedExchange (43) +- BroadcastExchange (66) @@ -135,7 +135,7 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeSh Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] Arguments: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] (15) Window @@ -153,7 +153,7 @@ Output [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] Arguments: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13], [ws_item_sk#13 ASC NULLS FIRST, d_date#12 ASC NULLS FIRST] -(19) ColumnarToRow [codegen id : 2] +(19) CometColumnarToRow [codegen id : 2] Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] (20) Window @@ -250,7 +250,7 @@ Arguments: hashpartitioning(ss_item_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeS Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] Arguments: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23], [ss_item_sk#23 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] -(40) ColumnarToRow [codegen id : 7] +(40) CometColumnarToRow [codegen id : 7] Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] (41) Window @@ -268,7 +268,7 @@ Output [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] Arguments: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34], [ss_item_sk#34 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST] -(45) ColumnarToRow [codegen id : 8] +(45) CometColumnarToRow [codegen id : 8] Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] (46) Window @@ -400,7 +400,7 @@ Arguments: 100, [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], [item_s Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (77) -+- * ColumnarToRow (76) ++- * CometColumnarToRow (76) +- CometProject (75) +- CometFilter (74) +- CometScan parquet spark_catalog.default.date_dim (73) @@ -421,7 +421,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_mont Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(76) ColumnarToRow [codegen id : 1] +(76) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] (77) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt index f8e1ba2353..f592008d93 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store InputAdapter Window [ws_item_sk,d_date] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item_sk,d_date,sumws,ws_item_sk] CometExchange [ws_item_sk] #4 @@ -46,7 +46,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] @@ -62,7 +62,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store InputAdapter Window [ws_item_sk,d_date] WholeStageCodegen (2) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item_sk,d_date,sumws,ws_item_sk] ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 @@ -83,7 +83,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store InputAdapter Window [ss_item_sk,d_date] WholeStageCodegen (7) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item_sk,d_date,sumss,ss_item_sk] CometExchange [ss_item_sk] #11 @@ -103,7 +103,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store InputAdapter Window [ss_item_sk,d_date] WholeStageCodegen (8) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item_sk,d_date,sumss,ss_item_sk] ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/explain.txt index 4b3a3c576e..84c04cfce2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/explain.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject (47) : : +- Window (26) : : +- * Filter (25) : : +- Window (24) - : : +- * ColumnarToRow (23) + : : +- * CometColumnarToRow (23) : : +- CometSort (22) : : +- CometExchange (21) : : +- CometHashAggregate (20) @@ -35,7 +35,7 @@ TakeOrderedAndProject (47) : +- BroadcastExchange (36) : +- * Project (35) : +- Window (34) - : +- * ColumnarToRow (33) + : +- * CometColumnarToRow (33) : +- CometSort (32) : +- CometExchange (31) : +- CometHashAggregate (30) @@ -43,7 +43,7 @@ TakeOrderedAndProject (47) +- BroadcastExchange (44) +- * Project (43) +- Window (42) - +- * ColumnarToRow (41) + +- * CometColumnarToRow (41) +- CometSort (40) +- ReusedExchange (39) @@ -154,7 +154,7 @@ Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQU Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16] Arguments: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16], [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarToRow [codegen id : 1] Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16] (24) Window @@ -193,7 +193,7 @@ Arguments: hashpartitioning(i_category#19, i_brand#20, cc_name#21, 5), ENSURE_RE Input [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] Arguments: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15], [i_category#19 ASC NULLS FIRST, i_brand#20 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] -(33) ColumnarToRow [codegen id : 3] +(33) CometColumnarToRow [codegen id : 3] Input [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] (34) Window @@ -225,7 +225,7 @@ Output [6]: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sal Input [6]: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sales#15] Arguments: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sales#15], [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] -(41) ColumnarToRow [codegen id : 5] +(41) CometColumnarToRow [codegen id : 5] Input [6]: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sales#15] (42) Window @@ -258,7 +258,7 @@ Arguments: 100, [(sum_sales#15 - avg_monthly_sales#18) ASC NULLS FIRST, d_year#1 Subquery:1 Hosting operator id = 3 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (51) -+- * ColumnarToRow (50) ++- * CometColumnarToRow (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -274,7 +274,7 @@ ReadSchema: struct Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (51) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt index 3ea3165fc7..e9ee530494 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] CometExchange [i_category,i_brand,cc_name] #1 @@ -34,7 +34,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] @@ -51,7 +51,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] CometExchange [i_category,i_brand,cc_name] #8 @@ -64,7 +64,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt index 3e1b88c26e..10122f2f79 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject (83) : +- * HashAggregate (66) : +- Union (65) : :- * HashAggregate (22) - : : +- * ColumnarToRow (21) + : : +- * CometColumnarToRow (21) : : +- CometExchange (20) : : +- CometHashAggregate (19) : : +- CometProject (18) @@ -31,7 +31,7 @@ TakeOrderedAndProject (83) : : +- CometFilter (15) : : +- CometScan parquet spark_catalog.default.store (14) : :- * HashAggregate (41) - : : +- * ColumnarToRow (40) + : : +- * CometColumnarToRow (40) : : +- CometExchange (39) : : +- CometHashAggregate (38) : : +- CometProject (37) @@ -50,7 +50,7 @@ TakeOrderedAndProject (83) : : +- CometFilter (34) : : +- CometScan parquet spark_catalog.default.catalog_page (33) : +- * HashAggregate (64) - : +- * ColumnarToRow (63) + : +- * CometColumnarToRow (63) : +- CometExchange (62) : +- CometHashAggregate (61) : +- CometProject (60) @@ -181,7 +181,7 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(21) ColumnarToRow [codegen id : 1] +(21) CometColumnarToRow [codegen id : 1] Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] (22) HashAggregate [codegen id : 1] @@ -272,7 +272,7 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#46)), partial_sum(Unscaled Input [5]: [cp_catalog_page_id#62, sum#63, sum#64, sum#65, sum#66] Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(40) ColumnarToRow [codegen id : 2] +(40) CometColumnarToRow [codegen id : 2] Input [5]: [cp_catalog_page_id#62, sum#63, sum#64, sum#65, sum#66] (41) HashAggregate [codegen id : 2] @@ -382,7 +382,7 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#83)), partial_sum(Unscaled Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(63) ColumnarToRow [codegen id : 3] +(63) CometColumnarToRow [codegen id : 3] Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] (64) HashAggregate [codegen id : 3] @@ -496,7 +496,7 @@ Arguments: 100, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (88) -+- * ColumnarToRow (87) ++- * CometColumnarToRow (87) +- CometProject (86) +- CometFilter (85) +- CometScan parquet spark_catalog.default.date_dim (84) @@ -517,7 +517,7 @@ Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#2 Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(87) ColumnarToRow [codegen id : 1] +(87) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] (88) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt index 751b3dc240..15b14cb16f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Union WholeStageCodegen (1) HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [s_store_id] #3 CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] @@ -32,7 +32,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] @@ -50,7 +50,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (2) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [cp_catalog_page_id] #7 CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] @@ -73,7 +73,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (3) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [web_site_id] #9 CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/explain.txt index 54cbf8326b..77ad43a121 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (38) +* CometColumnarToRow (38) +- CometTakeOrderedAndProject (37) +- CometFilter (36) +- CometHashAggregate (35) @@ -215,14 +215,14 @@ Condition : (cnt#23 >= 10) Input [3]: [state#22, cnt#23, ca_state#2] Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#23 ASC NULLS FIRST,ca_state#2 ASC NULLS FIRST], output=[state#22,cnt#23]), [state#22, cnt#23], 100, [cnt#23 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#22, cnt#23] -(38) ColumnarToRow [codegen id : 1] +(38) CometColumnarToRow [codegen id : 1] Input [2]: [state#22, cnt#23] ===== Subqueries ===== Subquery:1 Hosting operator id = 8 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (43) -+- * ColumnarToRow (42) ++- * CometColumnarToRow (42) +- CometProject (41) +- CometFilter (40) +- CometScan parquet spark_catalog.default.date_dim (39) @@ -243,7 +243,7 @@ Condition : ((isnotnull(d_month_seq#10) AND (d_month_seq#10 = Subquery scalar-su Input [2]: [d_date_sk#9, d_month_seq#10] Arguments: [d_date_sk#9], [d_date_sk#9] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] (43) BroadcastExchange @@ -251,7 +251,7 @@ Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] Subquery:2 Hosting operator id = 40 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* ColumnarToRow (50) +* CometColumnarToRow (50) +- CometHashAggregate (49) +- CometExchange (48) +- CometHashAggregate (47) @@ -289,7 +289,7 @@ Input [1]: [d_month_seq#24] Keys [1]: [d_month_seq#24] Functions: [] -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [1]: [d_month_seq#24] Subquery:3 Hosting operator id = 14 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt index 580f668ea8..266060bf09 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [state,cnt,ca_state] CometFilter [state,cnt,ca_state] @@ -25,13 +25,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [d_month_seq] CometExchange [d_month_seq] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt index 4d8ac469c3..7252446de1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (173) +* CometColumnarToRow (173) +- CometSort (172) +- CometColumnarExchange (171) +- CometProject (170) @@ -938,14 +938,14 @@ Arguments: rangepartitioning(product_name#76 ASC NULLS FIRST, store_name#78 ASC Input [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] Arguments: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162], [product_name#76 ASC NULLS FIRST, store_name#78 ASC NULLS FIRST, cnt#162 ASC NULLS FIRST, s1#90 ASC NULLS FIRST, s1#163 ASC NULLS FIRST] -(173) ColumnarToRow [codegen id : 1] +(173) CometColumnarToRow [codegen id : 1] Input [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (177) -+- * ColumnarToRow (176) ++- * CometColumnarToRow (176) +- CometFilter (175) +- CometScan parquet spark_catalog.default.date_dim (174) @@ -961,7 +961,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(176) ColumnarToRow [codegen id : 1] +(176) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] (177) BroadcastExchange @@ -970,7 +970,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#104 IN dynamicpruning#105 BroadcastExchange (181) -+- * ColumnarToRow (180) ++- * CometColumnarToRow (180) +- CometFilter (179) +- CometScan parquet spark_catalog.default.date_dim (178) @@ -986,7 +986,7 @@ ReadSchema: struct Input [2]: [d_date_sk#117, d_year#118] Condition : ((isnotnull(d_year#118) AND (d_year#118 = 2000)) AND isnotnull(d_date_sk#117)) -(180) ColumnarToRow [codegen id : 1] +(180) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#117, d_year#118] (181) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt index f38a6afca6..f4df8e5826 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] CometColumnarExchange [product_name,store_name,cnt,s1,s1] #1 @@ -51,7 +51,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -158,7 +158,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #22 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/explain.txt index c866b93d2a..d88a58e161 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/explain.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject (71) +- * Sort (67) +- Exchange (66) +- WindowGroupLimit (65) - +- * ColumnarToRow (64) + +- * CometColumnarToRow (64) +- CometSort (63) +- CometUnion (62) :- CometHashAggregate (21) @@ -365,7 +365,7 @@ Child 8 Input [9]: [i_category#177, i_class#178, i_brand#179, i_product_name#180 Input [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sumsales#141] Arguments: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sumsales#141], [i_category#17 ASC NULLS FIRST, sumsales#141 DESC NULLS LAST] -(64) ColumnarToRow [codegen id : 1] +(64) CometColumnarToRow [codegen id : 1] Input [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sumsales#141] (65) WindowGroupLimit @@ -400,7 +400,7 @@ Arguments: 100, [i_category#17 ASC NULLS FIRST, i_class#16 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (76) -+- * ColumnarToRow (75) ++- * CometColumnarToRow (75) +- CometProject (74) +- CometFilter (73) +- CometScan parquet spark_catalog.default.date_dim (72) @@ -421,7 +421,7 @@ Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_mont Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(75) ColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] (76) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt index 4ff3d4215c..456ecfa82d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ Exchange [i_category] #1 WindowGroupLimit [i_category,sumsales] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] CometUnion [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] @@ -28,7 +28,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/explain.txt index d9cda55ca6..0005dd3c62 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/explain.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject (58) : +- * HashAggregate (37) : +- * Project (36) : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * ColumnarToRow (9) + : :- * CometColumnarToRow (9) : : +- CometProject (8) : : +- CometBroadcastHashJoin (7) : : :- CometFilter (2) @@ -24,7 +24,7 @@ TakeOrderedAndProject (58) : : +- CometScan parquet spark_catalog.default.date_dim (3) : +- BroadcastExchange (34) : +- * BroadcastHashJoin LeftSemi BuildRight (33) - : :- * ColumnarToRow (12) + : :- * CometColumnarToRow (12) : : +- CometFilter (11) : : +- CometScan parquet spark_catalog.default.store (10) : +- BroadcastExchange (32) @@ -32,7 +32,7 @@ TakeOrderedAndProject (58) : +- * Filter (30) : +- Window (29) : +- WindowGroupLimit (28) - : +- * ColumnarToRow (27) + : +- * CometColumnarToRow (27) : +- CometSort (26) : +- CometHashAggregate (25) : +- CometExchange (24) @@ -99,7 +99,7 @@ Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] -(9) ColumnarToRow [codegen id : 4] +(9) CometColumnarToRow [codegen id : 4] Input [2]: [ss_store_sk#1, ss_net_profit#2] (10) CometScan parquet spark_catalog.default.store @@ -113,7 +113,7 @@ ReadSchema: struct Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) ColumnarToRow [codegen id : 3] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) CometScan parquet spark_catalog.default.store_sales @@ -182,7 +182,7 @@ Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] Input [3]: [s_state#15, _w0#18, s_state#15] Arguments: [s_state#15, _w0#18, s_state#15], [s_state#15 ASC NULLS FIRST, _w0#18 DESC NULLS LAST] -(27) ColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [3]: [s_state#15, _w0#18, s_state#15] (28) WindowGroupLimit @@ -343,7 +343,7 @@ Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (63) -+- * ColumnarToRow (62) ++- * CometColumnarToRow (62) +- CometProject (61) +- CometFilter (60) +- CometScan parquet spark_catalog.default.date_dim (59) @@ -364,7 +364,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(62) ColumnarToRow [codegen id : 1] +(62) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (63) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt index cb772b8002..6f92e978a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt @@ -23,7 +23,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] Project [ss_net_profit,s_county,s_state] BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] @@ -32,7 +32,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -45,7 +45,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #6 WholeStageCodegen (3) BroadcastHashJoin [s_state,s_state] - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] @@ -58,7 +58,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count Window [_w0,s_state] WindowGroupLimit [s_state,_w0] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [s_state,_w0] CometHashAggregate [s_state,_w0,sum,sum(UnscaledValue(ss_net_profit))] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/explain.txt index 45f7e2e66d..4b97b311be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (64) +* CometColumnarToRow (64) +- CometTakeOrderedAndProject (63) +- CometHashAggregate (62) +- CometExchange (61) @@ -364,14 +364,14 @@ Functions [1]: [count(1)] Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, [total_cnt#37 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] -(64) ColumnarToRow [codegen id : 1] +(64) CometColumnarToRow [codegen id : 1] Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (69) -+- * ColumnarToRow (68) ++- * CometColumnarToRow (68) +- CometProject (67) +- CometFilter (66) +- CometScan parquet spark_catalog.default.date_dim (65) @@ -392,7 +392,7 @@ Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_dat Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(68) ColumnarToRow [codegen id : 1] +(68) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (69) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt index ccd1b66fa2..fe26a3f5cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count,count(1)] @@ -32,7 +32,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] CometFilter [d_date_sk,d_date,d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/explain.txt index 12d5134a06..b1b1e31edb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (68) +* CometColumnarToRow (68) +- CometTakeOrderedAndProject (67) +- CometProject (66) +- CometBroadcastHashJoin (65) @@ -388,14 +388,14 @@ Arguments: [customer_id#25, customer_first_name#26, customer_last_name#27], [cus Input [3]: [customer_id#25, customer_first_name#26, customer_last_name#27] Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_first_name#26 ASC NULLS FIRST,customer_id#25 ASC NULLS FIRST,customer_last_name#27 ASC NULLS FIRST], output=[customer_id#25,customer_first_name#26,customer_last_name#27]), [customer_id#25, customer_first_name#26, customer_last_name#27], 100, [customer_first_name#26 ASC NULLS FIRST, customer_id#25 ASC NULLS FIRST, customer_last_name#27 ASC NULLS FIRST], [customer_id#25, customer_first_name#26, customer_last_name#27] -(68) ColumnarToRow [codegen id : 1] +(68) CometColumnarToRow [codegen id : 1] Input [3]: [customer_id#25, customer_first_name#26, customer_last_name#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (72) -+- * ColumnarToRow (71) ++- * CometColumnarToRow (71) +- CometFilter (70) +- CometScan parquet spark_catalog.default.date_dim (69) @@ -411,7 +411,7 @@ ReadSchema: struct Input [2]: [d_date_sk#9, d_year#10] Condition : (((isnotnull(d_year#10) AND (d_year#10 = 2001)) AND d_year#10 IN (2001,2002)) AND isnotnull(d_date_sk#9)) -(71) ColumnarToRow [codegen id : 1] +(71) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] (72) BroadcastExchange @@ -420,7 +420,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 BroadcastExchange (76) -+- * ColumnarToRow (75) ++- * CometColumnarToRow (75) +- CometFilter (74) +- CometScan parquet spark_catalog.default.date_dim (73) @@ -436,7 +436,7 @@ ReadSchema: struct Input [2]: [d_date_sk#22, d_year#23] Condition : (((isnotnull(d_year#23) AND (d_year#23 = 2002)) AND d_year#23 IN (2001,2002)) AND isnotnull(d_date_sk#22)) -(75) ColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#22, d_year#23] (76) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt index 631a82f1e3..9bde9ac863 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] CometProject [customer_id,customer_first_name,customer_last_name] @@ -23,7 +23,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -46,7 +46,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/explain.txt index 6789a22468..b8bba79fa5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (124) +* CometColumnarToRow (124) +- CometTakeOrderedAndProject (123) +- CometProject (122) +- CometSortMergeJoin (121) @@ -676,14 +676,14 @@ Arguments: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#1 Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#124 ASC NULLS FIRST,sales_amt_diff#125 ASC NULLS FIRST], output=[prev_year#120,year#121,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#122,curr_yr_cnt#123,sales_cnt_diff#124,sales_amt_diff#125]), [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], 100, [sales_cnt_diff#124 ASC NULLS FIRST, sales_amt_diff#125 ASC NULLS FIRST], [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] -(124) ColumnarToRow [codegen id : 1] +(124) CometColumnarToRow [codegen id : 1] Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (128) -+- * ColumnarToRow (127) ++- * CometColumnarToRow (127) +- CometFilter (126) +- CometScan parquet spark_catalog.default.date_dim (125) @@ -699,7 +699,7 @@ ReadSchema: struct Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(127) ColumnarToRow [codegen id : 1] +(127) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] (128) BroadcastExchange @@ -712,7 +712,7 @@ Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN d Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 BroadcastExchange (132) -+- * ColumnarToRow (131) ++- * CometColumnarToRow (131) +- CometFilter (130) +- CometScan parquet spark_catalog.default.date_dim (129) @@ -728,7 +728,7 @@ ReadSchema: struct Input [2]: [d_date_sk#77, d_year#78] Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(131) ColumnarToRow [codegen id : 1] +(131) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#77, d_year#78] (132) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt index 9e340130a5..5a8af58abc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] @@ -27,7 +27,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -102,7 +102,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/explain.txt index 879652ae12..97ea07aec3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/explain.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject (97) : +- Exchange (81) : +- * HashAggregate (80) : +- Union (79) - : :- * ColumnarToRow (31) + : :- * CometColumnarToRow (31) : : +- CometProject (30) : : +- CometBroadcastHashJoin (29) : : :- CometHashAggregate (16) @@ -42,7 +42,7 @@ TakeOrderedAndProject (97) : :- * Project (50) : : +- * BroadcastNestedLoopJoin Inner BuildLeft (49) : : :- BroadcastExchange (40) - : : : +- * ColumnarToRow (39) + : : : +- * CometColumnarToRow (39) : : : +- CometHashAggregate (38) : : : +- CometExchange (37) : : : +- CometHashAggregate (36) @@ -50,7 +50,7 @@ TakeOrderedAndProject (97) : : : +- CometBroadcastHashJoin (34) : : : :- CometScan parquet spark_catalog.default.catalog_sales (32) : : : +- ReusedExchange (33) - : : +- * ColumnarToRow (48) + : : +- * CometColumnarToRow (48) : : +- CometHashAggregate (47) : : +- CometExchange (46) : : +- CometHashAggregate (45) @@ -58,7 +58,7 @@ TakeOrderedAndProject (97) : : +- CometBroadcastHashJoin (43) : : :- CometScan parquet spark_catalog.default.catalog_returns (41) : : +- ReusedExchange (42) - : +- * ColumnarToRow (78) + : +- * CometColumnarToRow (78) : +- CometProject (77) : +- CometBroadcastHashJoin (76) : :- CometHashAggregate (63) @@ -239,7 +239,7 @@ Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] Arguments: [channel#24, id#25, sales#22, returns#26, profit#27], [store channel AS channel#24, s_store_sk#8 AS id#25, sales#22, coalesce(returns#20, 0.00) AS returns#26, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#27] -(31) ColumnarToRow [codegen id : 1] +(31) CometColumnarToRow [codegen id : 1] Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] (32) CometScan parquet spark_catalog.default.catalog_sales @@ -275,7 +275,7 @@ Input [3]: [cs_call_center_sk#28, sum#34, sum#35] Keys [1]: [cs_call_center_sk#28] Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#29)), sum(UnscaledValue(cs_net_profit#30))] -(39) ColumnarToRow [codegen id : 2] +(39) CometColumnarToRow [codegen id : 2] Input [3]: [cs_call_center_sk#28, sales#36, profit#37] (40) BroadcastExchange @@ -315,7 +315,7 @@ Input [2]: [sum#43, sum#44] Keys: [] Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] -(48) ColumnarToRow +(48) CometColumnarToRow Input [2]: [returns#45, profit_loss#46] (49) BroadcastNestedLoopJoin [codegen id : 3] @@ -451,7 +451,7 @@ Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] Arguments: [channel#72, id#73, sales#70, returns#74, profit#75], [web channel AS channel#72, wp_web_page_sk#56 AS id#73, sales#70, coalesce(returns#68, 0.00) AS returns#74, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#75] -(78) ColumnarToRow [codegen id : 4] +(78) CometColumnarToRow [codegen id : 4] Input [5]: [channel#72, id#73, sales#70, returns#74, profit#75] (79) Union @@ -558,7 +558,7 @@ Arguments: 100, [channel#24 ASC NULLS FIRST, id#25 ASC NULLS FIRST], [channel#24 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (102) -+- * ColumnarToRow (101) ++- * CometColumnarToRow (101) +- CometProject (100) +- CometFilter (99) +- CometScan parquet spark_catalog.default.date_dim (98) @@ -579,7 +579,7 @@ Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 < Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(101) ColumnarToRow [codegen id : 1] +(101) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (102) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt index 01cd2f085f..34656493f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter Union WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [s_store_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] CometBroadcastHashJoin [s_store_sk,sales,profit,s_store_sk,returns,profit_loss] @@ -32,7 +32,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] @@ -63,7 +63,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter BroadcastExchange #9 WholeStageCodegen (2) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [cs_call_center_sk,sales,profit,sum,sum,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] CometExchange [cs_call_center_sk] #10 @@ -73,7 +73,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [returns,profit_loss,sum,sum,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] CometExchange #11 @@ -84,7 +84,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 WholeStageCodegen (4) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [wp_web_page_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] CometBroadcastHashJoin [wp_web_page_sk,sales,profit,wp_web_page_sk,returns,profit_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/explain.txt index 1637692970..9a697cc844 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (67) +- * Project (66) - +- * ColumnarToRow (65) + +- * CometColumnarToRow (65) +- CometSortMergeJoin (64) :- CometProject (43) : +- CometSortMergeJoin (42) @@ -360,7 +360,7 @@ Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss Right output [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59], Inner -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] (66) Project [codegen id : 1] @@ -375,7 +375,7 @@ Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (71) -+- * ColumnarToRow (70) ++- * CometColumnarToRow (70) +- CometFilter (69) +- CometScan parquet spark_catalog.default.date_dim (68) @@ -391,7 +391,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(70) ColumnarToRow [codegen id : 1] +(70) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] (71) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt index 78d5fe1d7b..58ca9f30b8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] WholeStageCodegen (1) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] - ColumnarToRow + CometColumnarToRow InputAdapter CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] @@ -22,7 +22,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt index bd1ede55fb..8a06c4698c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject (115) : +- * HashAggregate (98) : +- Union (97) : :- * HashAggregate (38) - : : +- * ColumnarToRow (37) + : : +- * CometColumnarToRow (37) : : +- CometExchange (36) : : +- CometHashAggregate (35) : : +- CometProject (34) @@ -47,7 +47,7 @@ TakeOrderedAndProject (115) : : +- CometFilter (30) : : +- CometScan parquet spark_catalog.default.promotion (29) : :- * HashAggregate (67) - : : +- * ColumnarToRow (66) + : : +- * CometColumnarToRow (66) : : +- CometExchange (65) : : +- CometHashAggregate (64) : : +- CometProject (63) @@ -76,7 +76,7 @@ TakeOrderedAndProject (115) : : : +- ReusedExchange (58) : : +- ReusedExchange (61) : +- * HashAggregate (96) - : +- * ColumnarToRow (95) + : +- * CometColumnarToRow (95) : +- CometExchange (94) : +- CometHashAggregate (93) : +- CometProject (92) @@ -285,7 +285,7 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(co Input [6]: [s_store_id#17, sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] Arguments: hashpartitioning(s_store_id#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(37) ColumnarToRow [codegen id : 1] +(37) CometColumnarToRow [codegen id : 1] Input [6]: [s_store_id#17, sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] (38) HashAggregate [codegen id : 1] @@ -416,7 +416,7 @@ Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#39)), partial_sum(c Input [6]: [cp_catalog_page_id#50, sum#53, sum#54, isEmpty#55, sum#56, isEmpty#57] Arguments: hashpartitioning(cp_catalog_page_id#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(66) ColumnarToRow [codegen id : 2] +(66) CometColumnarToRow [codegen id : 2] Input [6]: [cp_catalog_page_id#50, sum#53, sum#54, isEmpty#55, sum#56, isEmpty#57] (67) HashAggregate [codegen id : 2] @@ -547,7 +547,7 @@ Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(c Input [6]: [web_site_id#81, sum#84, sum#85, isEmpty#86, sum#87, isEmpty#88] Arguments: hashpartitioning(web_site_id#81, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] -(95) ColumnarToRow [codegen id : 3] +(95) CometColumnarToRow [codegen id : 3] Input [6]: [web_site_id#81, sum#84, sum#85, isEmpty#86, sum#87, isEmpty#88] (96) HashAggregate [codegen id : 3] @@ -661,7 +661,7 @@ Arguments: 100, [channel#30 ASC NULLS FIRST, id#31 ASC NULLS FIRST], [channel#30 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (120) -+- * ColumnarToRow (119) ++- * CometColumnarToRow (119) +- CometProject (118) +- CometFilter (117) +- CometScan parquet spark_catalog.default.date_dim (116) @@ -682,7 +682,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(119) ColumnarToRow [codegen id : 1] +(119) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] (120) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt index aa61bbd579..e04a9ec1ad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Union WholeStageCodegen (1) HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [s_store_id] #3 CometHashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] @@ -38,7 +38,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] @@ -65,7 +65,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (2) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [cp_catalog_page_id] #11 CometHashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] @@ -97,7 +97,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [p_promo_sk] #10 WholeStageCodegen (3) HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [web_site_id] #15 CometHashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/explain.txt index 7354c23de0..6a35a64e6a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (36) +- * Project (35) +- Window (34) - +- * ColumnarToRow (33) + +- * CometColumnarToRow (33) +- CometSort (32) +- CometExchange (31) +- CometHashAggregate (30) @@ -186,7 +186,7 @@ Arguments: hashpartitioning(lochierarchy#28, _w0#40, 5), ENSURE_REQUIREMENTS, Co Input [5]: [total_sum#25, i_category#9, i_class#8, lochierarchy#28, _w0#40] Arguments: [total_sum#25, i_category#9, i_class#8, lochierarchy#28, _w0#40], [lochierarchy#28 ASC NULLS FIRST, _w0#40 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST] -(33) ColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [codegen id : 1] Input [5]: [total_sum#25, i_category#9, i_class#8, lochierarchy#28, _w0#40] (34) Window @@ -205,7 +205,7 @@ Arguments: 100, [lochierarchy#28 DESC NULLS LAST, CASE WHEN (lochierarchy#28 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (41) -+- * ColumnarToRow (40) ++- * CometColumnarToRow (40) +- CometProject (39) +- CometFilter (38) +- CometScan parquet spark_catalog.default.date_dim (37) @@ -226,7 +226,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(40) ColumnarToRow [codegen id : 1] +(40) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (41) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt index ae161e5ae2..d2ed714b7e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl InputAdapter Window [total_sum,lochierarchy,_w0] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [total_sum,i_category,i_class,lochierarchy,_w0] CometExchange [lochierarchy,_w0] #1 @@ -24,7 +24,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/explain.txt index 4d870a8b80..7bcf62d617 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -* ColumnarToRow (24) +* CometColumnarToRow (24) +- CometSort (23) +- CometColumnarExchange (22) +- * Project (21) +- Window (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometSort (18) +- CometExchange (17) +- CometHashAggregate (16) @@ -111,7 +111,7 @@ Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometNativeShuff Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15], [i_class#9 ASC NULLS FIRST] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] (20) Window @@ -130,14 +130,14 @@ Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17], [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#17 ASC NULLS FIRST] -(24) ColumnarToRow [codegen id : 3] +(24) CometColumnarToRow [codegen id : 3] Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * ColumnarToRow (28) ++- * CometColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan parquet spark_catalog.default.date_dim (25) @@ -158,7 +158,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt index 2b7813a707..6509498640 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 @@ -8,7 +8,7 @@ WholeStageCodegen (3) InputAdapter Window [_w0,i_class] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] CometExchange [i_class] #2 @@ -24,7 +24,7 @@ WholeStageCodegen (3) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/explain.txt index 868b2f4822..c2b7f49205 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (39) +* CometColumnarToRow (39) +- CometTakeOrderedAndProject (38) +- CometHashAggregate (37) +- CometExchange (36) @@ -220,14 +220,14 @@ Functions [1]: [count(1)] Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34, cd_dep_count#28, cnt4#35, cd_dep_employed_count#29, cnt5#36, cd_dep_college_count#30, cnt6#37] Arguments: TakeOrderedAndProject(limit=100, orderBy=[cd_gender#23 ASC NULLS FIRST,cd_marital_status#24 ASC NULLS FIRST,cd_education_status#25 ASC NULLS FIRST,cd_purchase_estimate#26 ASC NULLS FIRST,cd_credit_rating#27 ASC NULLS FIRST,cd_dep_count#28 ASC NULLS FIRST,cd_dep_employed_count#29 ASC NULLS FIRST,cd_dep_college_count#30 ASC NULLS FIRST], output=[cd_gender#23,cd_marital_status#24,cd_education_status#25,cnt1#32,cd_purchase_estimate#26,cnt2#33,cd_credit_rating#27,cnt3#34,cd_dep_count#28,cnt4#35,cd_dep_employed_count#29,cnt5#36,cd_dep_college_count#30,cnt6#37]), [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34, cd_dep_count#28, cnt4#35, cd_dep_employed_count#29, cnt5#36, cd_dep_college_count#30, cnt6#37], 100, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34, cd_dep_count#28, cnt4#35, cd_dep_employed_count#29, cnt5#36, cd_dep_college_count#30, cnt6#37] -(39) ColumnarToRow [codegen id : 1] +(39) CometColumnarToRow [codegen id : 1] Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34, cd_dep_count#28, cnt4#35, cd_dep_employed_count#29, cnt5#36, cd_dep_college_count#30, cnt6#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (44) -+- * ColumnarToRow (43) ++- * CometColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan parquet spark_catalog.default.date_dim (40) @@ -248,7 +248,7 @@ Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 200 Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(43) ColumnarToRow [codegen id : 1] +(43) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt index 2e8f2022cf..b570b700ac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6] CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6,count,count(1)] @@ -21,7 +21,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/explain.txt index 6f50d7fbd9..d8e722989b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/explain.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject (71) : : : +- * HashAggregate (16) : : : +- Exchange (15) : : : +- * HashAggregate (14) - : : : +- * ColumnarToRow (13) + : : : +- * CometColumnarToRow (13) : : : +- CometProject (12) : : : +- CometBroadcastHashJoin (11) : : : :- CometProject (7) @@ -26,7 +26,7 @@ TakeOrderedAndProject (71) : : +- * HashAggregate (33) : : +- Exchange (32) : : +- * HashAggregate (31) - : : +- * ColumnarToRow (30) + : : +- * CometColumnarToRow (30) : : +- CometProject (29) : : +- CometBroadcastHashJoin (28) : : :- CometProject (24) @@ -44,7 +44,7 @@ TakeOrderedAndProject (71) : +- * HashAggregate (49) : +- Exchange (48) : +- * HashAggregate (47) - : +- * ColumnarToRow (46) + : +- * CometColumnarToRow (46) : +- CometProject (45) : +- CometBroadcastHashJoin (44) : :- CometProject (42) @@ -59,7 +59,7 @@ TakeOrderedAndProject (71) +- * HashAggregate (67) +- Exchange (66) +- * HashAggregate (65) - +- * ColumnarToRow (64) + +- * CometColumnarToRow (64) +- CometProject (63) +- CometBroadcastHashJoin (62) :- CometProject (60) @@ -132,7 +132,7 @@ Arguments: [ss_sold_date_sk#12], [d_date_sk#14], Inner, BuildRight Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12, d_date_sk#14, d_year#15] Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15], [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15] -(13) ColumnarToRow [codegen id : 1] +(13) CometColumnarToRow [codegen id : 1] Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15] (14) HashAggregate [codegen id : 1] @@ -217,7 +217,7 @@ Arguments: [ss_sold_date_sk#32], [d_date_sk#34], Inner, BuildRight Input [12]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32, d_date_sk#34, d_year#35] Arguments: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35], [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] -(30) ColumnarToRow [codegen id : 2] +(30) CometColumnarToRow [codegen id : 2] Input [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] (31) HashAggregate [codegen id : 2] @@ -296,7 +296,7 @@ Arguments: [ws_sold_date_sk#54], [d_date_sk#56], Inner, BuildRight Input [12]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54, d_date_sk#56, d_year#57] Arguments: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, d_year#57], [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, d_year#57] -(46) ColumnarToRow [codegen id : 4] +(46) CometColumnarToRow [codegen id : 4] Input [10]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, d_year#57] (47) HashAggregate [codegen id : 4] @@ -383,7 +383,7 @@ Arguments: [ws_sold_date_sk#74], [d_date_sk#76], Inner, BuildRight Input [12]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74, d_date_sk#76, d_year#77] Arguments: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_ext_discount_amt#72, ws_ext_list_price#73, d_year#77], [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_ext_discount_amt#72, ws_ext_list_price#73, d_year#77] -(64) ColumnarToRow [codegen id : 6] +(64) CometColumnarToRow [codegen id : 6] Input [10]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_ext_discount_amt#72, ws_ext_list_price#73, d_year#77] (65) HashAggregate [codegen id : 6] @@ -426,7 +426,7 @@ Arguments: 100, [customer_id#38 ASC NULLS FIRST, customer_first_name#39 ASC NULL Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (75) -+- * ColumnarToRow (74) ++- * CometColumnarToRow (74) +- CometFilter (73) +- CometScan parquet spark_catalog.default.date_dim (72) @@ -442,7 +442,7 @@ ReadSchema: struct Input [2]: [d_date_sk#14, d_year#15] Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#14)) -(74) ColumnarToRow [codegen id : 1] +(74) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] (75) BroadcastExchange @@ -451,7 +451,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#33 BroadcastExchange (79) -+- * ColumnarToRow (78) ++- * CometColumnarToRow (78) +- CometFilter (77) +- CometScan parquet spark_catalog.default.date_dim (76) @@ -467,7 +467,7 @@ ReadSchema: struct Input [2]: [d_date_sk#34, d_year#35] Condition : ((isnotnull(d_year#35) AND (d_year#35 = 2002)) AND isnotnull(d_date_sk#34)) -(78) ColumnarToRow [codegen id : 1] +(78) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#34, d_year#35] (79) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt index d7c6ef3b25..3d79e5c73d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt @@ -11,7 +11,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 WholeStageCodegen (1) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] @@ -25,7 +25,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -40,7 +40,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 WholeStageCodegen (2) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] @@ -54,7 +54,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -70,7 +70,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 WholeStageCodegen (4) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] @@ -91,7 +91,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 WholeStageCodegen (6) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt index 96b5da2358..9bc0ec6b69 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt @@ -7,7 +7,7 @@ TakeOrderedAndProject (22) +- * HashAggregate (17) +- Exchange (16) +- * HashAggregate (15) - +- * ColumnarToRow (14) + +- * CometColumnarToRow (14) +- CometProject (13) +- CometBroadcastHashJoin (12) :- CometProject (7) @@ -87,7 +87,7 @@ Arguments: [ws_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] Arguments: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [6]: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] (15) HashAggregate [codegen id : 1] @@ -132,7 +132,7 @@ Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_ite Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (27) -+- * ColumnarToRow (26) ++- * CometColumnarToRow (26) +- CometProject (25) +- CometFilter (24) +- CometScan parquet spark_catalog.default.date_dim (23) @@ -153,7 +153,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (27) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt index 07c91b94dc..434501c513 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 WholeStageCodegen (1) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] @@ -24,7 +24,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/explain.txt index 143f2c1b57..f69741e22e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/explain.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject (86) : +- * HashAggregate (64) : +- Exchange (63) : +- * HashAggregate (62) - : +- * ColumnarToRow (61) + : +- * CometColumnarToRow (61) : +- CometProject (60) : +- CometBroadcastHashJoin (59) : :- CometProject (54) @@ -71,7 +71,7 @@ TakeOrderedAndProject (86) +- * HashAggregate (82) +- Exchange (81) +- * HashAggregate (80) - +- * ColumnarToRow (79) + +- * CometColumnarToRow (79) +- CometProject (78) +- CometBroadcastHashJoin (77) :- CometProject (72) @@ -372,7 +372,7 @@ Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -(61) ColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [codegen id : 1] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] (62) HashAggregate [codegen id : 1] @@ -457,7 +457,7 @@ Arguments: [ss_sold_date_sk#63], [d_date_sk#70], Inner, BuildRight Input [7]: [ss_quantity#61, ss_list_price#62, ss_sold_date_sk#63, i_brand_id#67, i_class_id#68, i_category_id#69, d_date_sk#70] Arguments: [ss_quantity#61, ss_list_price#62, i_brand_id#67, i_class_id#68, i_category_id#69], [ss_quantity#61, ss_list_price#62, i_brand_id#67, i_class_id#68, i_category_id#69] -(79) ColumnarToRow [codegen id : 2] +(79) CometColumnarToRow [codegen id : 2] Input [5]: [ss_quantity#61, ss_list_price#62, i_brand_id#67, i_class_id#68, i_category_id#69] (80) HashAggregate [codegen id : 2] @@ -502,7 +502,7 @@ Subquery:1 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquer * HashAggregate (103) +- Exchange (102) +- * HashAggregate (101) - +- * ColumnarToRow (100) + +- * CometColumnarToRow (100) +- CometUnion (99) :- CometProject (90) : +- CometBroadcastHashJoin (89) @@ -580,7 +580,7 @@ Child 0 Input [2]: [quantity#90, list_price#91] Child 1 Input [2]: [quantity#97, list_price#98] Child 2 Input [2]: [quantity#104, list_price#105] -(100) ColumnarToRow [codegen id : 1] +(100) CometColumnarToRow [codegen id : 1] Input [2]: [quantity#90, list_price#91] (101) HashAggregate [codegen id : 1] @@ -609,7 +609,7 @@ Subquery:4 Hosting operator id = 95 Hosting Expression = ws_sold_date_sk#101 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (108) -+- * ColumnarToRow (107) ++- * CometColumnarToRow (107) +- CometProject (106) +- CometFilter (105) +- CometScan parquet spark_catalog.default.date_dim (104) @@ -630,7 +630,7 @@ Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subqu Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(107) ColumnarToRow [codegen id : 1] +(107) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] (108) BroadcastExchange @@ -640,7 +640,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] Subquery:7 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* ColumnarToRow (112) +* CometColumnarToRow (112) +- CometProject (111) +- CometFilter (110) +- CometScan parquet spark_catalog.default.date_dim (109) @@ -661,12 +661,12 @@ Condition : (((((isnotnull(d_year#113) AND isnotnull(d_moy#114)) AND isnotnull(d Input [4]: [d_week_seq#112, d_year#113, d_moy#114, d_dom#115] Arguments: [d_week_seq#112], [d_week_seq#112] -(112) ColumnarToRow [codegen id : 1] +(112) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#112] Subquery:8 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (117) -+- * ColumnarToRow (116) ++- * CometColumnarToRow (116) +- CometProject (115) +- CometFilter (114) +- CometScan parquet spark_catalog.default.date_dim (113) @@ -687,7 +687,7 @@ Condition : (((isnotnull(d_year#116) AND (d_year#116 >= 1998)) AND (d_year#116 < Input [2]: [d_date_sk#26, d_year#116] Arguments: [d_date_sk#26], [d_date_sk#26] -(116) ColumnarToRow [codegen id : 1] +(116) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] (117) BroadcastExchange @@ -706,7 +706,7 @@ Subquery:13 Hosting operator id = 83 Hosting Expression = ReusedSubquery Subquer Subquery:14 Hosting operator id = 66 Hosting Expression = ss_sold_date_sk#63 IN dynamicpruning#64 BroadcastExchange (122) -+- * ColumnarToRow (121) ++- * CometColumnarToRow (121) +- CometProject (120) +- CometFilter (119) +- CometScan parquet spark_catalog.default.date_dim (118) @@ -727,7 +727,7 @@ Condition : ((isnotnull(d_week_seq#71) AND (d_week_seq#71 = ReusedSubquery Subqu Input [2]: [d_date_sk#70, d_week_seq#71] Arguments: [d_date_sk#70], [d_date_sk#70] -(121) ColumnarToRow [codegen id : 1] +(121) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#70] (122) BroadcastExchange @@ -737,7 +737,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:15 Hosting operator id = 119 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] Subquery:16 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#72, [id=#73] -* ColumnarToRow (126) +* CometColumnarToRow (126) +- CometProject (125) +- CometFilter (124) +- CometScan parquet spark_catalog.default.date_dim (123) @@ -758,7 +758,7 @@ Condition : (((((isnotnull(d_year#118) AND isnotnull(d_moy#119)) AND isnotnull(d Input [4]: [d_week_seq#117, d_year#118, d_moy#119, d_dom#120] Arguments: [d_week_seq#117], [d_week_seq#117] -(126) ColumnarToRow [codegen id : 1] +(126) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#117] Subquery:17 Hosting operator id = 74 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt index 1fecc0fca4..62cb19cd3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Exchange #14 WholeStageCodegen (1) HashAggregate [quantity,list_price] [sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] @@ -32,7 +32,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Exchange [i_brand_id,i_class_id,i_category_id] #1 WholeStageCodegen (1) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] @@ -44,7 +44,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_week_seq] @@ -52,7 +52,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_week_seq,d_year,d_moy,d_dom] @@ -76,7 +76,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -132,7 +132,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Exchange [i_brand_id,i_class_id,i_category_id] #16 WholeStageCodegen (2) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] @@ -144,7 +144,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_week_seq] @@ -152,7 +152,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] Subquery #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_week_seq,d_year,d_moy,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/explain.txt index 14e2e96b2e..9bae4a28f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/explain.txt @@ -12,7 +12,7 @@ TakeOrderedAndProject (124) : : +- * HashAggregate (64) : : +- Exchange (63) : : +- * HashAggregate (62) - : : +- * ColumnarToRow (61) + : : +- * CometColumnarToRow (61) : : +- CometProject (60) : : +- CometBroadcastHashJoin (59) : : :- CometProject (54) @@ -77,7 +77,7 @@ TakeOrderedAndProject (124) : : +- * HashAggregate (79) : : +- Exchange (78) : : +- * HashAggregate (77) - : : +- * ColumnarToRow (76) + : : +- * CometColumnarToRow (76) : : +- CometProject (75) : : +- CometBroadcastHashJoin (74) : : :- CometProject (72) @@ -92,7 +92,7 @@ TakeOrderedAndProject (124) : +- * HashAggregate (94) : +- Exchange (93) : +- * HashAggregate (92) - : +- * ColumnarToRow (91) + : +- * CometColumnarToRow (91) : +- CometProject (90) : +- CometBroadcastHashJoin (89) : :- CometProject (87) @@ -410,7 +410,7 @@ Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -(61) ColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [codegen id : 1] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] (62) HashAggregate [codegen id : 1] @@ -479,7 +479,7 @@ Arguments: [cs_sold_date_sk#62], [d_date_sk#69], Inner, BuildRight Input [7]: [cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62, i_brand_id#66, i_class_id#67, i_category_id#68, d_date_sk#69] Arguments: [cs_quantity#60, cs_list_price#61, i_brand_id#66, i_class_id#67, i_category_id#68], [cs_quantity#60, cs_list_price#61, i_brand_id#66, i_class_id#67, i_category_id#68] -(76) ColumnarToRow [codegen id : 3] +(76) CometColumnarToRow [codegen id : 3] Input [5]: [cs_quantity#60, cs_list_price#61, i_brand_id#66, i_class_id#67, i_category_id#68] (77) HashAggregate [codegen id : 3] @@ -548,7 +548,7 @@ Arguments: [ws_sold_date_sk#84], [d_date_sk#91], Inner, BuildRight Input [7]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84, i_brand_id#88, i_class_id#89, i_category_id#90, d_date_sk#91] Arguments: [ws_quantity#82, ws_list_price#83, i_brand_id#88, i_class_id#89, i_category_id#90], [ws_quantity#82, ws_list_price#83, i_brand_id#88, i_class_id#89, i_category_id#90] -(91) ColumnarToRow [codegen id : 5] +(91) CometColumnarToRow [codegen id : 5] Input [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#88, i_class_id#89, i_category_id#90] (92) HashAggregate [codegen id : 5] @@ -735,7 +735,7 @@ Subquery:1 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquer * HashAggregate (144) +- Exchange (143) +- * HashAggregate (142) - +- * ColumnarToRow (141) + +- * CometColumnarToRow (141) +- CometUnion (140) :- CometProject (128) : +- CometBroadcastHashJoin (127) @@ -832,7 +832,7 @@ Child 0 Input [2]: [quantity#212, list_price#213] Child 1 Input [2]: [quantity#220, list_price#221] Child 2 Input [2]: [quantity#227, list_price#228] -(141) ColumnarToRow [codegen id : 1] +(141) CometColumnarToRow [codegen id : 1] Input [2]: [quantity#212, list_price#213] (142) HashAggregate [codegen id : 1] @@ -857,7 +857,7 @@ Subquery:2 Hosting operator id = 125 Hosting Expression = ss_sold_date_sk#209 IN Subquery:3 Hosting operator id = 129 Hosting Expression = cs_sold_date_sk#216 IN dynamicpruning#217 BroadcastExchange (149) -+- * ColumnarToRow (148) ++- * CometColumnarToRow (148) +- CometProject (147) +- CometFilter (146) +- CometScan parquet spark_catalog.default.date_dim (145) @@ -878,7 +878,7 @@ Condition : (((isnotnull(d_year#219) AND (d_year#219 >= 1998)) AND (d_year#219 < Input [2]: [d_date_sk#218, d_year#219] Arguments: [d_date_sk#218], [d_date_sk#218] -(148) ColumnarToRow [codegen id : 1] +(148) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#218] (149) BroadcastExchange @@ -889,7 +889,7 @@ Subquery:4 Hosting operator id = 136 Hosting Expression = ws_sold_date_sk#224 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (154) -+- * ColumnarToRow (153) ++- * CometColumnarToRow (153) +- CometProject (152) +- CometFilter (151) +- CometScan parquet spark_catalog.default.date_dim (150) @@ -910,7 +910,7 @@ Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2 Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(153) ColumnarToRow [codegen id : 1] +(153) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] (154) BroadcastExchange @@ -919,7 +919,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (159) -+- * ColumnarToRow (158) ++- * CometColumnarToRow (158) +- CometProject (157) +- CometFilter (156) +- CometScan parquet spark_catalog.default.date_dim (155) @@ -940,7 +940,7 @@ Condition : (((isnotnull(d_year#235) AND (d_year#235 >= 1999)) AND (d_year#235 < Input [2]: [d_date_sk#26, d_year#235] Arguments: [d_date_sk#26], [d_date_sk#26] -(158) ColumnarToRow [codegen id : 1] +(158) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] (159) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt index 1610cd8884..a71531718d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt @@ -24,7 +24,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Exchange #16 WholeStageCodegen (1) HashAggregate [quantity,list_price] [sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] @@ -38,7 +38,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -57,7 +57,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Exchange [i_brand_id,i_class_id,i_category_id] #3 WholeStageCodegen (1) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] @@ -69,7 +69,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -93,7 +93,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -145,7 +145,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Exchange [i_brand_id,i_class_id,i_category_id] #19 WholeStageCodegen (3) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] @@ -166,7 +166,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Exchange [i_brand_id,i_class_id,i_category_id] #20 WholeStageCodegen (5) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/explain.txt index 9aea34f5ef..2b4fce20f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/explain.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject (147) :- * HashAggregate (39) : +- Exchange (38) : +- * HashAggregate (37) - : +- * ColumnarToRow (36) + : +- * CometColumnarToRow (36) : +- CometProject (35) : +- CometBroadcastHashJoin (34) : :- CometProject (30) @@ -43,7 +43,7 @@ TakeOrderedAndProject (147) :- * HashAggregate (65) : +- Exchange (64) : +- * HashAggregate (63) - : +- * ColumnarToRow (62) + : +- * CometColumnarToRow (62) : +- CometProject (61) : +- CometBroadcastHashJoin (60) : :- CometProject (58) @@ -69,7 +69,7 @@ TakeOrderedAndProject (147) :- * HashAggregate (92) : +- Exchange (91) : +- * HashAggregate (90) - : +- * ColumnarToRow (89) + : +- * CometColumnarToRow (89) : +- CometProject (88) : +- CometBroadcastHashJoin (87) : :- CometProject (85) @@ -96,7 +96,7 @@ TakeOrderedAndProject (147) :- * HashAggregate (119) : +- Exchange (118) : +- * HashAggregate (117) - : +- * ColumnarToRow (116) + : +- * CometColumnarToRow (116) : +- CometProject (115) : +- CometBroadcastHashJoin (114) : :- CometProject (112) @@ -123,7 +123,7 @@ TakeOrderedAndProject (147) +- * HashAggregate (145) +- Exchange (144) +- * HashAggregate (143) - +- * ColumnarToRow (142) + +- * CometColumnarToRow (142) +- CometProject (141) +- CometBroadcastHashJoin (140) :- CometProject (136) @@ -316,7 +316,7 @@ Arguments: [cs_item_sk#3], [i_item_sk#27], Inner, BuildRight Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24, i_item_sk#27, i_item_id#28] Arguments: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#29, agg2#30, agg3#31, agg4#32, agg5#33, agg6#34, agg7#35], [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, cast(cs_quantity#4 as decimal(12,2)) AS agg1#29, cast(cs_list_price#5 as decimal(12,2)) AS agg2#30, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#31, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#32, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#33, cast(c_birth_year#19 as decimal(12,2)) AS agg6#34, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#35] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [11]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#29, agg2#30, agg3#31, agg4#32, agg5#33, agg6#34, agg7#35] (37) HashAggregate [codegen id : 1] @@ -433,7 +433,7 @@ Arguments: [cs_item_sk#80], [i_item_sk#99], Inner, BuildRight Input [12]: [cs_item_sk#80, cs_quantity#81, cs_list_price#82, cs_sales_price#83, cs_coupon_amt#84, cs_net_profit#85, cd_dep_count#89, c_birth_year#93, ca_state#96, ca_country#97, i_item_sk#99, i_item_id#100] Arguments: [i_item_id#100, ca_country#97, ca_state#96, agg1#101, agg2#102, agg3#103, agg4#104, agg5#105, agg6#106, agg7#107], [i_item_id#100, ca_country#97, ca_state#96, cast(cs_quantity#81 as decimal(12,2)) AS agg1#101, cast(cs_list_price#82 as decimal(12,2)) AS agg2#102, cast(cs_coupon_amt#84 as decimal(12,2)) AS agg3#103, cast(cs_sales_price#83 as decimal(12,2)) AS agg4#104, cast(cs_net_profit#85 as decimal(12,2)) AS agg5#105, cast(c_birth_year#93 as decimal(12,2)) AS agg6#106, cast(cd_dep_count#89 as decimal(12,2)) AS agg7#107] -(62) ColumnarToRow [codegen id : 3] +(62) CometColumnarToRow [codegen id : 3] Input [10]: [i_item_id#100, ca_country#97, ca_state#96, agg1#101, agg2#102, agg3#103, agg4#104, agg5#105, agg6#106, agg7#107] (63) HashAggregate [codegen id : 3] @@ -554,7 +554,7 @@ Arguments: [cs_item_sk#153], [i_item_sk#172], Inner, BuildRight Input [11]: [cs_item_sk#153, cs_quantity#154, cs_list_price#155, cs_sales_price#156, cs_coupon_amt#157, cs_net_profit#158, cd_dep_count#162, c_birth_year#166, ca_country#170, i_item_sk#172, i_item_id#173] Arguments: [i_item_id#173, ca_country#170, agg1#174, agg2#175, agg3#176, agg4#177, agg5#178, agg6#179, agg7#180], [i_item_id#173, ca_country#170, cast(cs_quantity#154 as decimal(12,2)) AS agg1#174, cast(cs_list_price#155 as decimal(12,2)) AS agg2#175, cast(cs_coupon_amt#157 as decimal(12,2)) AS agg3#176, cast(cs_sales_price#156 as decimal(12,2)) AS agg4#177, cast(cs_net_profit#158 as decimal(12,2)) AS agg5#178, cast(c_birth_year#166 as decimal(12,2)) AS agg6#179, cast(cd_dep_count#162 as decimal(12,2)) AS agg7#180] -(89) ColumnarToRow [codegen id : 5] +(89) CometColumnarToRow [codegen id : 5] Input [9]: [i_item_id#173, ca_country#170, agg1#174, agg2#175, agg3#176, agg4#177, agg5#178, agg6#179, agg7#180] (90) HashAggregate [codegen id : 5] @@ -675,7 +675,7 @@ Arguments: [cs_item_sk#227], [i_item_sk#245], Inner, BuildRight Input [10]: [cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, cd_dep_count#236, c_birth_year#240, i_item_sk#245, i_item_id#246] Arguments: [i_item_id#246, agg1#247, agg2#248, agg3#249, agg4#250, agg5#251, agg6#252, agg7#253], [i_item_id#246, cast(cs_quantity#228 as decimal(12,2)) AS agg1#247, cast(cs_list_price#229 as decimal(12,2)) AS agg2#248, cast(cs_coupon_amt#231 as decimal(12,2)) AS agg3#249, cast(cs_sales_price#230 as decimal(12,2)) AS agg4#250, cast(cs_net_profit#232 as decimal(12,2)) AS agg5#251, cast(c_birth_year#240 as decimal(12,2)) AS agg6#252, cast(cd_dep_count#236 as decimal(12,2)) AS agg7#253] -(116) ColumnarToRow [codegen id : 7] +(116) CometColumnarToRow [codegen id : 7] Input [8]: [i_item_id#246, agg1#247, agg2#248, agg3#249, agg4#250, agg5#251, agg6#252, agg7#253] (117) HashAggregate [codegen id : 7] @@ -792,7 +792,7 @@ Arguments: [cs_item_sk#301], [i_item_sk#318], Inner, BuildRight Input [9]: [cs_item_sk#301, cs_quantity#302, cs_list_price#303, cs_sales_price#304, cs_coupon_amt#305, cs_net_profit#306, cd_dep_count#310, c_birth_year#314, i_item_sk#318] Arguments: [agg1#319, agg2#320, agg3#321, agg4#322, agg5#323, agg6#324, agg7#325], [cast(cs_quantity#302 as decimal(12,2)) AS agg1#319, cast(cs_list_price#303 as decimal(12,2)) AS agg2#320, cast(cs_coupon_amt#305 as decimal(12,2)) AS agg3#321, cast(cs_sales_price#304 as decimal(12,2)) AS agg4#322, cast(cs_net_profit#306 as decimal(12,2)) AS agg5#323, cast(c_birth_year#314 as decimal(12,2)) AS agg6#324, cast(cd_dep_count#310 as decimal(12,2)) AS agg7#325] -(142) ColumnarToRow [codegen id : 9] +(142) CometColumnarToRow [codegen id : 9] Input [7]: [agg1#319, agg2#320, agg3#321, agg4#322, agg5#323, agg6#324, agg7#325] (143) HashAggregate [codegen id : 9] @@ -823,7 +823,7 @@ Arguments: 100, [ca_country#24 ASC NULLS FIRST, ca_state#23 ASC NULLS FIRST, ca_ Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (152) -+- * ColumnarToRow (151) ++- * CometColumnarToRow (151) +- CometProject (150) +- CometFilter (149) +- CometScan parquet spark_catalog.default.date_dim (148) @@ -844,7 +844,7 @@ Condition : ((isnotnull(d_year#26) AND (d_year#26 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(151) ColumnarToRow [codegen id : 1] +(151) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] (152) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt index 509d4be06c..2834fbd22b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag Exchange [i_item_id,ca_country,ca_state,ca_county] #1 WholeStageCodegen (1) HashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] @@ -25,7 +25,7 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -57,7 +57,7 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag Exchange [i_item_id,ca_country,ca_state] #9 WholeStageCodegen (3) HashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_sk,i_item_id] @@ -88,7 +88,7 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag Exchange [i_item_id,ca_country] #11 WholeStageCodegen (5) HashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_sk,i_item_id] @@ -120,7 +120,7 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag Exchange [i_item_id] #13 WholeStageCodegen (7) HashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk,i_item_id] @@ -152,7 +152,7 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag Exchange #15 WholeStageCodegen (9) HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt index 6a93509c8e..3f92f9998f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt @@ -7,7 +7,7 @@ TakeOrderedAndProject (22) +- * HashAggregate (17) +- Exchange (16) +- * HashAggregate (15) - +- * ColumnarToRow (14) + +- * CometColumnarToRow (14) +- CometProject (13) +- CometBroadcastHashJoin (12) :- CometProject (7) @@ -87,7 +87,7 @@ Arguments: [cs_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] Arguments: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [6]: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] (15) HashAggregate [codegen id : 1] @@ -132,7 +132,7 @@ Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_ite Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (27) -+- * ColumnarToRow (26) ++- * CometColumnarToRow (26) +- CometProject (25) +- CometFilter (24) +- CometScan parquet spark_catalog.default.date_dim (23) @@ -153,7 +153,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (27) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt index cd6bcfd611..f9770664d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 WholeStageCodegen (1) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] @@ -24,7 +24,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/explain.txt index 34d510fc71..5be661c351 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/explain.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject (24) +- * Expand (20) +- * Project (19) +- * BroadcastNestedLoopJoin Inner BuildRight (18) - :- * ColumnarToRow (14) + :- * CometColumnarToRow (14) : +- CometProject (13) : +- CometBroadcastHashJoin (12) : :- CometProject (8) @@ -21,7 +21,7 @@ TakeOrderedAndProject (24) : +- CometFilter (10) : +- CometScan parquet spark_catalog.default.item (9) +- BroadcastExchange (17) - +- * ColumnarToRow (16) + +- * CometColumnarToRow (16) +- CometScan parquet spark_catalog.default.warehouse (15) @@ -89,7 +89,7 @@ Arguments: [inv_item_sk#1], [i_item_sk#7], Inner, BuildRight Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] Arguments: [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11], [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11] -(14) ColumnarToRow [codegen id : 2] +(14) CometColumnarToRow [codegen id : 2] Input [5]: [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11] (15) CometScan parquet spark_catalog.default.warehouse @@ -98,7 +98,7 @@ Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] ReadSchema: struct<> -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarToRow [codegen id : 1] Input: [] (17) BroadcastExchange @@ -143,7 +143,7 @@ Arguments: 100, [qoh#22 ASC NULLS FIRST, i_product_name#12 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * ColumnarToRow (28) ++- * CometColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan parquet spark_catalog.default.date_dim (25) @@ -164,7 +164,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt index 0e864ab701..027836b68c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] BroadcastNestedLoopJoin - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] @@ -19,7 +19,7 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -34,6 +34,6 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometScan parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/explain.txt index 4cabf98776..0728d74413 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/explain.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject (46) : +- * HashAggregate (22) : +- Exchange (21) : +- * HashAggregate (20) - : +- * ColumnarToRow (19) + : +- * CometColumnarToRow (19) : +- CometProject (18) : +- CometBroadcastHashJoin (17) : :- CometProject (13) @@ -135,7 +135,7 @@ Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#13], Inner, BuildRight Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12, w_warehouse_sk#13] Arguments: [inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12], [inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [5]: [inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12] (20) HashAggregate [codegen id : 1] @@ -292,7 +292,7 @@ Arguments: 100, [qoh#29 ASC NULLS FIRST, i_product_name#25 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (51) -+- * ColumnarToRow (50) ++- * CometColumnarToRow (50) +- CometProject (49) +- CometFilter (48) +- CometScan parquet spark_catalog.default.date_dim (47) @@ -313,7 +313,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_mont Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (51) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt index fc4778a226..ea4c7ae785 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Exchange [i_product_name,i_brand,i_class,i_category] #1 WholeStageCodegen (1) HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] @@ -21,7 +21,7 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/explain.txt index 522e028b5a..165a4b6099 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (45) +* CometColumnarToRow (45) +- CometSort (44) +- CometColumnarExchange (43) +- * Filter (42) @@ -11,7 +11,7 @@ +- * HashAggregate (36) +- * Project (35) +- * BroadcastHashJoin Inner BuildRight (34) - :- * ColumnarToRow (29) + :- * CometColumnarToRow (29) : +- CometProject (28) : +- CometBroadcastHashJoin (27) : :- CometProject (23) @@ -41,7 +41,7 @@ : +- CometFilter (25) : +- CometScan parquet spark_catalog.default.customer (24) +- BroadcastExchange (33) - +- * ColumnarToRow (32) + +- * CometColumnarToRow (32) +- CometFilter (31) +- CometScan parquet spark_catalog.default.customer_address (30) @@ -177,7 +177,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] Arguments: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25], [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -(29) ColumnarToRow [codegen id : 2] +(29) CometColumnarToRow [codegen id : 2] Input [13]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] (30) CometScan parquet spark_catalog.default.customer_address @@ -191,7 +191,7 @@ ReadSchema: struct= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_do Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt index 7da6076f5b..e1fc6bf43f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/explain.txt index 5933ea5fad..fbf3086106 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/explain.txt @@ -11,7 +11,7 @@ TakeOrderedAndProject (44) : : +- * Filter (27) : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * ColumnarToRow (12) + : : : :- * CometColumnarToRow (12) : : : : +- CometBroadcastHashJoin (11) : : : : :- CometFilter (2) : : : : : +- CometScan parquet spark_catalog.default.customer (1) @@ -24,23 +24,23 @@ TakeOrderedAndProject (44) : : : : +- CometFilter (5) : : : : +- CometScan parquet spark_catalog.default.date_dim (4) : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) + : : : +- * CometColumnarToRow (17) : : : +- CometProject (16) : : : +- CometBroadcastHashJoin (15) : : : :- CometScan parquet spark_catalog.default.web_sales (13) : : : +- ReusedExchange (14) : : +- BroadcastExchange (25) - : : +- * ColumnarToRow (24) + : : +- * CometColumnarToRow (24) : : +- CometProject (23) : : +- CometBroadcastHashJoin (22) : : :- CometScan parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (32) - : +- * ColumnarToRow (31) + : +- * CometColumnarToRow (31) : +- CometFilter (30) : +- CometScan parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (38) - +- * ColumnarToRow (37) + +- * CometColumnarToRow (37) +- CometFilter (36) +- CometScan parquet spark_catalog.default.customer_demographics (35) @@ -100,7 +100,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) ColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 5] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan parquet spark_catalog.default.web_sales @@ -122,7 +122,7 @@ Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#12] (18) BroadcastExchange @@ -154,7 +154,7 @@ Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [1]: [cs_ship_customer_sk#16] (25) BroadcastExchange @@ -186,7 +186,7 @@ ReadSchema: struct Input [2]: [ca_address_sk#20, ca_state#21] Condition : isnotnull(ca_address_sk#20) -(31) ColumnarToRow [codegen id : 3] +(31) CometColumnarToRow [codegen id : 3] Input [2]: [ca_address_sk#20, ca_state#21] (32) BroadcastExchange @@ -214,7 +214,7 @@ ReadSchema: struct Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(48) ColumnarToRow [codegen id : 1] +(48) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (49) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt index 61f4976b4a..84730bc70f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 WholeStageCodegen (1) HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] @@ -38,7 +38,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/explain.txt index 138876bfcf..97c37bc6e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/explain.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject (77) : +- * HashAggregate (19) : +- Exchange (18) : +- * HashAggregate (17) - : +- * ColumnarToRow (16) + : +- * CometColumnarToRow (16) : +- CometProject (15) : +- CometBroadcastHashJoin (14) : :- CometProject (9) @@ -40,7 +40,7 @@ TakeOrderedAndProject (77) : +- * HashAggregate (42) : +- Exchange (41) : +- * HashAggregate (40) - : +- * ColumnarToRow (39) + : +- * CometColumnarToRow (39) : +- CometProject (38) : +- CometBroadcastHashJoin (37) : :- CometProject (35) @@ -63,7 +63,7 @@ TakeOrderedAndProject (77) +- * HashAggregate (65) +- Exchange (64) +- * HashAggregate (63) - +- * ColumnarToRow (62) + +- * CometColumnarToRow (62) +- CometProject (61) +- CometBroadcastHashJoin (60) :- CometProject (58) @@ -150,7 +150,7 @@ Arguments: [ws_sold_date_sk#6], [d_date_sk#13], Inner, BuildRight Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarToRow [codegen id : 1] Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] (17) HashAggregate [codegen id : 1] @@ -255,7 +255,7 @@ Arguments: [cs_sold_date_sk#43], [d_date_sk#50], Inner, BuildRight Input [7]: [cs_item_sk#38, cs_quantity#40, cs_net_paid#41, cs_sold_date_sk#43, cr_return_quantity#47, cr_return_amount#48, d_date_sk#50] Arguments: [cs_item_sk#38, cs_quantity#40, cs_net_paid#41, cr_return_quantity#47, cr_return_amount#48], [cs_item_sk#38, cs_quantity#40, cs_net_paid#41, cr_return_quantity#47, cr_return_amount#48] -(39) ColumnarToRow [codegen id : 6] +(39) CometColumnarToRow [codegen id : 6] Input [5]: [cs_item_sk#38, cs_quantity#40, cs_net_paid#41, cr_return_quantity#47, cr_return_amount#48] (40) HashAggregate [codegen id : 6] @@ -360,7 +360,7 @@ Arguments: [ss_sold_date_sk#78], [d_date_sk#85], Inner, BuildRight Input [7]: [ss_item_sk#73, ss_quantity#75, ss_net_paid#76, ss_sold_date_sk#78, sr_return_quantity#82, sr_return_amt#83, d_date_sk#85] Arguments: [ss_item_sk#73, ss_quantity#75, ss_net_paid#76, sr_return_quantity#82, sr_return_amt#83], [ss_item_sk#73, ss_quantity#75, ss_net_paid#76, sr_return_quantity#82, sr_return_amt#83] -(62) ColumnarToRow [codegen id : 11] +(62) CometColumnarToRow [codegen id : 11] Input [5]: [ss_item_sk#73, ss_quantity#75, ss_net_paid#76, sr_return_quantity#82, sr_return_amt#83] (63) HashAggregate [codegen id : 11] @@ -437,7 +437,7 @@ Arguments: 100, [channel#37 ASC NULLS FIRST, return_rank#35 ASC NULLS FIRST, cur Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (82) -+- * ColumnarToRow (81) ++- * CometColumnarToRow (81) +- CometProject (80) +- CometFilter (79) +- CometScan parquet spark_catalog.default.date_dim (78) @@ -458,7 +458,7 @@ Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2 Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(81) ColumnarToRow [codegen id : 1] +(81) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] (82) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt index bd3b21cdbd..d47d447ebc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt @@ -26,7 +26,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] Exchange [ws_item_sk] #3 WholeStageCodegen (1) HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] @@ -39,7 +39,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -70,7 +70,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] Exchange [cs_item_sk] #8 WholeStageCodegen (6) HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] @@ -104,7 +104,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] Exchange [ss_item_sk] #11 WholeStageCodegen (11) HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt index af53d08ccc..b4f393f048 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt @@ -25,7 +25,7 @@ TakeOrderedAndProject (70) : : : +- * HashAggregate (12) : : : +- Exchange (11) : : : +- * HashAggregate (10) - : : : +- * ColumnarToRow (9) + : : : +- * CometColumnarToRow (9) : : : +- CometProject (8) : : : +- CometBroadcastHashJoin (7) : : : :- CometFilter (2) @@ -53,7 +53,7 @@ TakeOrderedAndProject (70) : : +- * HashAggregate (37) : : +- Exchange (36) : : +- * HashAggregate (35) - : : +- * ColumnarToRow (34) + : : +- * CometColumnarToRow (34) : : +- CometProject (33) : : +- CometBroadcastHashJoin (32) : : :- CometFilter (30) @@ -111,7 +111,7 @@ Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] Arguments: [ws_item_sk#1, ws_sales_price#2, d_date#6], [ws_item_sk#1, ws_sales_price#2, d_date#6] -(9) ColumnarToRow [codegen id : 1] +(9) CometColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] (10) HashAggregate [codegen id : 1] @@ -227,7 +227,7 @@ Arguments: [ss_sold_date_sk#27], [d_date_sk#29], Inner, BuildRight Input [5]: [ss_item_sk#25, ss_sales_price#26, ss_sold_date_sk#27, d_date_sk#29, d_date#30] Arguments: [ss_item_sk#25, ss_sales_price#26, d_date#30], [ss_item_sk#25, ss_sales_price#26, d_date#30] -(34) ColumnarToRow [codegen id : 11] +(34) CometColumnarToRow [codegen id : 11] Input [3]: [ss_item_sk#25, ss_sales_price#26, d_date#30] (35) HashAggregate [codegen id : 11] @@ -400,7 +400,7 @@ Arguments: 100, [item_sk#48 ASC NULLS FIRST, d_date#49 ASC NULLS FIRST], [item_s Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (75) -+- * ColumnarToRow (74) ++- * CometColumnarToRow (74) +- CometProject (73) +- CometFilter (72) +- CometScan parquet spark_catalog.default.date_dim (71) @@ -421,7 +421,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_mont Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(74) ColumnarToRow [codegen id : 1] +(74) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] (75) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt index 47ad5e34ff..bd2049a31b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt @@ -41,7 +41,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store Exchange [ws_item_sk,d_date] #5 WholeStageCodegen (1) HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_item_sk,ws_sales_price,d_date] CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] @@ -50,7 +50,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] @@ -95,7 +95,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store Exchange [ss_item_sk,d_date] #12 WholeStageCodegen (11) HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_item_sk,ss_sales_price,d_date] CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt index ad7a09b49d..6953beb801 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject (45) : : +- * HashAggregate (21) : : +- Exchange (20) : : +- * HashAggregate (19) - : : +- * ColumnarToRow (18) + : : +- * CometColumnarToRow (18) : : +- CometProject (17) : : +- CometBroadcastHashJoin (16) : : :- CometProject (12) @@ -130,7 +130,7 @@ Arguments: [cs_call_center_sk#4], [cc_call_center_sk#12], Inner, BuildRight Input [8]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11, cc_call_center_sk#12, cc_name#13] Arguments: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13], [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] -(18) ColumnarToRow [codegen id : 1] +(18) CometColumnarToRow [codegen id : 1] Input [6]: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] (19) HashAggregate [codegen id : 1] @@ -256,7 +256,7 @@ Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#1 Subquery:1 Hosting operator id = 3 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (49) -+- * ColumnarToRow (48) ++- * CometColumnarToRow (48) +- CometFilter (47) +- CometScan parquet spark_catalog.default.date_dim (46) @@ -272,7 +272,7 @@ ReadSchema: struct Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(48) ColumnarToRow [codegen id : 1] +(48) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (49) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt index daeafbf929..0be8059a8b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m Exchange [i_category,i_brand,cc_name,d_year,d_moy] #2 WholeStageCodegen (1) HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] @@ -38,7 +38,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/explain.txt index 4c45b9d4dd..31316b5eee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/explain.txt @@ -11,7 +11,7 @@ TakeOrderedAndProject (83) : :- * HashAggregate (22) : : +- Exchange (21) : : +- * HashAggregate (20) - : : +- * ColumnarToRow (19) + : : +- * CometColumnarToRow (19) : : +- CometProject (18) : : +- CometBroadcastHashJoin (17) : : :- CometProject (13) @@ -33,7 +33,7 @@ TakeOrderedAndProject (83) : :- * HashAggregate (41) : : +- Exchange (40) : : +- * HashAggregate (39) - : : +- * ColumnarToRow (38) + : : +- * CometColumnarToRow (38) : : +- CometProject (37) : : +- CometBroadcastHashJoin (36) : : :- CometProject (32) @@ -52,7 +52,7 @@ TakeOrderedAndProject (83) : +- * HashAggregate (64) : +- Exchange (63) : +- * HashAggregate (62) - : +- * ColumnarToRow (61) + : +- * CometColumnarToRow (61) : +- CometProject (60) : +- CometBroadcastHashJoin (59) : :- CometProject (55) @@ -172,7 +172,7 @@ Arguments: [store_sk#6], [s_store_sk#24], Inner, BuildRight Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#25] Arguments: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25], [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] (20) HashAggregate [codegen id : 1] @@ -265,7 +265,7 @@ Arguments: [page_sk#48], [cp_catalog_page_sk#65], Inner, BuildRight Input [7]: [page_sk#48, sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_sk#65, cp_catalog_page_id#66] Arguments: [sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_id#66], [sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_id#66] -(38) ColumnarToRow [codegen id : 3] +(38) CometColumnarToRow [codegen id : 3] Input [5]: [sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_id#66] (39) HashAggregate [codegen id : 3] @@ -377,7 +377,7 @@ Arguments: [wsr_web_site_sk#89], [web_site_sk#111], Inner, BuildRight Input [7]: [wsr_web_site_sk#89, sales_price#91, profit#92, return_amt#93, net_loss#94, web_site_sk#111, web_site_id#112] Arguments: [sales_price#91, profit#92, return_amt#93, net_loss#94, web_site_id#112], [sales_price#91, profit#92, return_amt#93, net_loss#94, web_site_id#112] -(61) ColumnarToRow [codegen id : 5] +(61) CometColumnarToRow [codegen id : 5] Input [5]: [sales_price#91, profit#92, return_amt#93, net_loss#94, web_site_id#112] (62) HashAggregate [codegen id : 5] @@ -502,7 +502,7 @@ Arguments: 100, [channel#38 ASC NULLS FIRST, id#39 ASC NULLS FIRST], [channel#38 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (88) -+- * ColumnarToRow (87) ++- * CometColumnarToRow (87) +- CometProject (86) +- CometFilter (85) +- CometScan parquet spark_catalog.default.date_dim (84) @@ -523,7 +523,7 @@ Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#2 Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(87) ColumnarToRow [codegen id : 1] +(87) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] (88) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt index c3b00538b8..fa3410ac3f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Exchange [s_store_id] #3 WholeStageCodegen (1) HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [sales_price,profit,return_amt,net_loss,s_store_id] CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] @@ -34,7 +34,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] @@ -56,7 +56,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Exchange [cp_catalog_page_id] #7 WholeStageCodegen (3) HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] @@ -81,7 +81,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Exchange [web_site_id] #9 WholeStageCodegen (5) HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [sales_price,profit,return_amt,net_loss,web_site_id] CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/explain.txt index 8227cd53db..4da968046c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/explain.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject (40) +- * HashAggregate (36) +- * Project (35) +- * BroadcastHashJoin Inner BuildRight (34) - :- * ColumnarToRow (19) + :- * CometColumnarToRow (19) : +- CometProject (18) : +- CometBroadcastHashJoin (17) : :- CometProject (12) @@ -28,7 +28,7 @@ TakeOrderedAndProject (40) +- BroadcastExchange (33) +- * Project (32) +- * BroadcastHashJoin Inner BuildRight (31) - :- * ColumnarToRow (22) + :- * CometColumnarToRow (22) : +- CometFilter (21) : +- CometScan parquet spark_catalog.default.item (20) +- BroadcastExchange (30) @@ -36,7 +36,7 @@ TakeOrderedAndProject (40) +- * HashAggregate (28) +- Exchange (27) +- * HashAggregate (26) - +- * ColumnarToRow (25) + +- * CometColumnarToRow (25) +- CometFilter (24) +- CometScan parquet spark_catalog.default.item (23) @@ -129,7 +129,7 @@ Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight Input [4]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7, d_date_sk#9] Arguments: [ca_state#2, ss_item_sk#5], [ca_state#2, ss_item_sk#5] -(19) ColumnarToRow [codegen id : 4] +(19) CometColumnarToRow [codegen id : 4] Input [2]: [ca_state#2, ss_item_sk#5] (20) CometScan parquet spark_catalog.default.item @@ -143,7 +143,7 @@ ReadSchema: struct Input [3]: [i_item_sk#13, i_current_price#14, i_category#15] Condition : ((isnotnull(i_current_price#14) AND isnotnull(i_category#15)) AND isnotnull(i_item_sk#13)) -(22) ColumnarToRow [codegen id : 3] +(22) CometColumnarToRow [codegen id : 3] Input [3]: [i_item_sk#13, i_current_price#14, i_category#15] (23) CometScan parquet spark_catalog.default.item @@ -157,7 +157,7 @@ ReadSchema: struct Input [2]: [i_current_price#16, i_category#17] Condition : isnotnull(i_category#17) -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [2]: [i_current_price#16, i_category#17] (26) HashAggregate [codegen id : 1] @@ -240,7 +240,7 @@ Arguments: 100, [cnt#28 ASC NULLS FIRST, state#27 ASC NULLS FIRST], [state#27, c Subquery:1 Hosting operator id = 8 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (45) -+- * ColumnarToRow (44) ++- * CometColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan parquet spark_catalog.default.date_dim (41) @@ -261,7 +261,7 @@ Condition : ((isnotnull(d_month_seq#10) AND (d_month_seq#10 = ReusedSubquery Sub Input [2]: [d_date_sk#9, d_month_seq#10] Arguments: [d_date_sk#9], [d_date_sk#9] -(44) ColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] (45) BroadcastExchange @@ -271,7 +271,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 42 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] Subquery:3 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* ColumnarToRow (52) +* CometColumnarToRow (52) +- CometHashAggregate (51) +- CometExchange (50) +- CometHashAggregate (49) @@ -309,7 +309,7 @@ Input [1]: [d_month_seq#29] Keys [1]: [d_month_seq#29] Functions: [] -(52) ColumnarToRow [codegen id : 1] +(52) CometColumnarToRow [codegen id : 1] Input [1]: [d_month_seq#29] Subquery:4 Hosting operator id = 14 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt index c5bbfc7d58..93e0567ac4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject [cnt,state] HashAggregate [ca_state] [count,count] Project [ca_state] BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ca_state,ss_item_sk] CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] @@ -27,7 +27,7 @@ TakeOrderedAndProject [cnt,state] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -35,7 +35,7 @@ TakeOrderedAndProject [cnt,state] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [d_month_seq] CometExchange [d_month_seq] #5 @@ -54,7 +54,7 @@ TakeOrderedAndProject [cnt,state] WholeStageCodegen (3) Project [i_item_sk] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [i_item_sk,i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] @@ -67,7 +67,7 @@ TakeOrderedAndProject [cnt,state] Exchange [i_category] #9 WholeStageCodegen (1) HashAggregate [i_category,i_current_price] [sum,count,sum,count] - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt index 37e2319107..4ca8fa2912 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (181) +* CometColumnarToRow (181) +- CometSort (180) +- CometColumnarExchange (179) +- * Project (178) @@ -40,7 +40,7 @@ : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (34) : : : : : : : : : : : : : : : :- * Project (32) : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (31) - : : : : : : : : : : : : : : : : :- * ColumnarToRow (11) + : : : : : : : : : : : : : : : : :- * CometColumnarToRow (11) : : : : : : : : : : : : : : : : : +- CometSort (10) : : : : : : : : : : : : : : : : : +- CometExchange (9) : : : : : : : : : : : : : : : : : +- CometProject (8) @@ -57,7 +57,7 @@ : : : : : : : : : : : : : : : : +- * HashAggregate (27) : : : : : : : : : : : : : : : : +- Exchange (26) : : : : : : : : : : : : : : : : +- * HashAggregate (25) - : : : : : : : : : : : : : : : : +- * ColumnarToRow (24) + : : : : : : : : : : : : : : : : +- * CometColumnarToRow (24) : : : : : : : : : : : : : : : : +- CometProject (23) : : : : : : : : : : : : : : : : +- CometSortMergeJoin (22) : : : : : : : : : : : : : : : : :- CometSort (16) @@ -72,44 +72,44 @@ : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (17) : : : : : : : : : : : : : : : +- ReusedExchange (33) : : : : : : : : : : : : : : +- BroadcastExchange (39) - : : : : : : : : : : : : : : +- * ColumnarToRow (38) + : : : : : : : : : : : : : : +- * CometColumnarToRow (38) : : : : : : : : : : : : : : +- CometFilter (37) : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (36) : : : : : : : : : : : : : +- BroadcastExchange (45) - : : : : : : : : : : : : : +- * ColumnarToRow (44) + : : : : : : : : : : : : : +- * CometColumnarToRow (44) : : : : : : : : : : : : : +- CometFilter (43) : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (42) : : : : : : : : : : : : +- BroadcastExchange (51) - : : : : : : : : : : : : +- * ColumnarToRow (50) + : : : : : : : : : : : : +- * CometColumnarToRow (50) : : : : : : : : : : : : +- CometFilter (49) : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (48) : : : : : : : : : : : +- ReusedExchange (54) : : : : : : : : : : +- BroadcastExchange (60) - : : : : : : : : : : +- * ColumnarToRow (59) + : : : : : : : : : : +- * CometColumnarToRow (59) : : : : : : : : : : +- CometFilter (58) : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (57) : : : : : : : : : +- ReusedExchange (63) : : : : : : : : +- BroadcastExchange (69) - : : : : : : : : +- * ColumnarToRow (68) + : : : : : : : : +- * CometColumnarToRow (68) : : : : : : : : +- CometFilter (67) : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (66) : : : : : : : +- BroadcastExchange (75) - : : : : : : : +- * ColumnarToRow (74) + : : : : : : : +- * CometColumnarToRow (74) : : : : : : : +- CometFilter (73) : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (72) : : : : : : +- ReusedExchange (78) : : : : : +- BroadcastExchange (84) - : : : : : +- * ColumnarToRow (83) + : : : : : +- * CometColumnarToRow (83) : : : : : +- CometFilter (82) : : : : : +- CometScan parquet spark_catalog.default.customer_address (81) : : : : +- ReusedExchange (87) : : : +- BroadcastExchange (93) - : : : +- * ColumnarToRow (92) + : : : +- * CometColumnarToRow (92) : : : +- CometFilter (91) : : : +- CometScan parquet spark_catalog.default.income_band (90) : : +- ReusedExchange (96) : +- BroadcastExchange (103) - : +- * ColumnarToRow (102) + : +- * CometColumnarToRow (102) : +- CometProject (101) : +- CometFilter (100) : +- CometScan parquet spark_catalog.default.item (99) @@ -149,7 +149,7 @@ : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (129) : : : : : : : : : : : : : : :- * Project (127) : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (126) - : : : : : : : : : : : : : : : :- * ColumnarToRow (120) + : : : : : : : : : : : : : : : :- * CometColumnarToRow (120) : : : : : : : : : : : : : : : : +- CometSort (119) : : : : : : : : : : : : : : : : +- CometExchange (118) : : : : : : : : : : : : : : : : +- CometProject (117) @@ -230,7 +230,7 @@ Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeSh Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] -(11) ColumnarToRow [codegen id : 1] +(11) CometColumnarToRow [codegen id : 1] Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] (12) CometScan parquet spark_catalog.default.catalog_sales @@ -288,7 +288,7 @@ Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number# Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] (25) HashAggregate [codegen id : 2] @@ -355,7 +355,7 @@ ReadSchema: struct Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] Condition : ((isnotnull(s_store_sk#39) AND isnotnull(s_store_name#40)) AND isnotnull(s_zip#41)) -(38) ColumnarToRow [codegen id : 5] +(38) CometColumnarToRow [codegen id : 5] Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] (39) BroadcastExchange @@ -383,7 +383,7 @@ ReadSchema: struct Input [2]: [d_date_sk#48, d_year#49] Condition : isnotnull(d_date_sk#48) -(50) ColumnarToRow [codegen id : 7] +(50) CometColumnarToRow [codegen id : 7] Input [2]: [d_date_sk#48, d_year#49] (51) BroadcastExchange @@ -452,7 +452,7 @@ ReadSchema: struct Input [2]: [cd_demo_sk#52, cd_marital_status#53] Condition : (isnotnull(cd_demo_sk#52) AND isnotnull(cd_marital_status#53)) -(59) ColumnarToRow [codegen id : 9] +(59) CometColumnarToRow [codegen id : 9] Input [2]: [cd_demo_sk#52, cd_marital_status#53] (60) BroadcastExchange @@ -493,7 +493,7 @@ ReadSchema: struct Input [1]: [p_promo_sk#56] Condition : isnotnull(p_promo_sk#56) -(68) ColumnarToRow [codegen id : 11] +(68) CometColumnarToRow [codegen id : 11] Input [1]: [p_promo_sk#56] (69) BroadcastExchange @@ -521,7 +521,7 @@ ReadSchema: struct Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] Condition : (isnotnull(hd_demo_sk#57) AND isnotnull(hd_income_band_sk#58)) -(74) ColumnarToRow [codegen id : 12] +(74) CometColumnarToRow [codegen id : 12] Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] (75) BroadcastExchange @@ -562,7 +562,7 @@ ReadSchema: struct Input [1]: [ib_income_band_sk#71] Condition : isnotnull(ib_income_band_sk#71) -(92) ColumnarToRow [codegen id : 16] +(92) CometColumnarToRow [codegen id : 16] Input [1]: [ib_income_band_sk#71] (93) BroadcastExchange @@ -648,7 +648,7 @@ Condition : ((((((isnotnull(i_current_price#74) AND i_color#75 IN (purple Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] Arguments: [i_item_sk#73, i_product_name#76], [i_item_sk#73, i_product_name#76] -(102) ColumnarToRow [codegen id : 18] +(102) CometColumnarToRow [codegen id : 18] Input [2]: [i_item_sk#73, i_product_name#76] (103) BroadcastExchange @@ -735,7 +735,7 @@ Arguments: hashpartitioning(ss_item_sk#106, 5), ENSURE_REQUIREMENTS, CometNative Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] Arguments: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117], [ss_item_sk#106 ASC NULLS FIRST] -(120) ColumnarToRow [codegen id : 21] +(120) CometColumnarToRow [codegen id : 21] Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] (121) ReusedExchange [Reuses operator id: 26] @@ -1005,14 +1005,14 @@ Arguments: rangepartitioning(product_name#89 ASC NULLS FIRST, store_name#91 ASC Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#181, s2#182, s3#183, syear#179, cnt#180] Arguments: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#181, s2#182, s3#183, syear#179, cnt#180], [product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#180 ASC NULLS FIRST, s1#103 ASC NULLS FIRST, s1#181 ASC NULLS FIRST] -(181) ColumnarToRow [codegen id : 42] +(181) CometColumnarToRow [codegen id : 42] Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#181, s2#182, s3#183, syear#179, cnt#180] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (185) -+- * ColumnarToRow (184) ++- * CometColumnarToRow (184) +- CometFilter (183) +- CometScan parquet spark_catalog.default.date_dim (182) @@ -1028,7 +1028,7 @@ ReadSchema: struct Input [2]: [d_date_sk#37, d_year#38] Condition : ((isnotnull(d_year#38) AND (d_year#38 = 1999)) AND isnotnull(d_date_sk#37)) -(184) ColumnarToRow [codegen id : 1] +(184) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#37, d_year#38] (185) BroadcastExchange @@ -1037,7 +1037,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 110 Hosting Expression = ss_sold_date_sk#117 IN dynamicpruning#118 BroadcastExchange (189) -+- * ColumnarToRow (188) ++- * CometColumnarToRow (188) +- CometFilter (187) +- CometScan parquet spark_catalog.default.date_dim (186) @@ -1053,7 +1053,7 @@ ReadSchema: struct Input [2]: [d_date_sk#132, d_year#133] Condition : ((isnotnull(d_year#133) AND (d_year#133 = 2000)) AND isnotnull(d_date_sk#132)) -(188) ColumnarToRow [codegen id : 1] +(188) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#132, d_year#133] (189) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt index f43131af19..fe4c12a116 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (42) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] CometColumnarExchange [product_name,store_name,cnt,s1,s1] #1 @@ -48,7 +48,7 @@ WholeStageCodegen (42) SortMergeJoin [ss_item_sk,cs_item_sk] InputAdapter WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometExchange [ss_item_sk] #3 @@ -60,7 +60,7 @@ WholeStageCodegen (42) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -77,7 +77,7 @@ WholeStageCodegen (42) Exchange [cs_item_sk] #6 WholeStageCodegen (2) HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] @@ -96,21 +96,21 @@ WholeStageCodegen (42) InputAdapter BroadcastExchange #9 WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [s_store_sk,s_store_name,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] InputAdapter BroadcastExchange #10 WholeStageCodegen (6) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter BroadcastExchange #11 WholeStageCodegen (7) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -119,7 +119,7 @@ WholeStageCodegen (42) InputAdapter BroadcastExchange #12 WholeStageCodegen (9) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [cd_demo_sk,cd_marital_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] @@ -128,14 +128,14 @@ WholeStageCodegen (42) InputAdapter BroadcastExchange #13 WholeStageCodegen (11) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [p_promo_sk] CometScan parquet spark_catalog.default.promotion [p_promo_sk] InputAdapter BroadcastExchange #14 WholeStageCodegen (12) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [hd_demo_sk,hd_income_band_sk] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] @@ -144,7 +144,7 @@ WholeStageCodegen (42) InputAdapter BroadcastExchange #15 WholeStageCodegen (14) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] @@ -153,7 +153,7 @@ WholeStageCodegen (42) InputAdapter BroadcastExchange #16 WholeStageCodegen (16) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [ib_income_band_sk] CometScan parquet spark_catalog.default.income_band [ib_income_band_sk] @@ -162,7 +162,7 @@ WholeStageCodegen (42) InputAdapter BroadcastExchange #17 WholeStageCodegen (18) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] CometFilter [i_item_sk,i_current_price,i_color,i_product_name] @@ -209,7 +209,7 @@ WholeStageCodegen (42) SortMergeJoin [ss_item_sk,cs_item_sk] InputAdapter WholeStageCodegen (21) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometExchange [ss_item_sk] #19 @@ -221,7 +221,7 @@ WholeStageCodegen (42) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #21 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/explain.txt index 49a984b30c..42d820eb40 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/explain.txt @@ -11,7 +11,7 @@ TakeOrderedAndProject (71) :- * HashAggregate (22) : +- Exchange (21) : +- * HashAggregate (20) - : +- * ColumnarToRow (19) + : +- * CometColumnarToRow (19) : +- CometProject (18) : +- CometBroadcastHashJoin (17) : :- CometProject (13) @@ -160,7 +160,7 @@ Arguments: [ss_item_sk#1], [i_item_sk#14], Inner, BuildRight Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13, i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] Arguments: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13, i_brand#15, i_class#16, i_category#17, i_product_name#18], [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13, i_brand#15, i_class#16, i_category#17, i_product_name#18] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13, i_brand#15, i_class#16, i_category#17, i_product_name#18] (20) HashAggregate [codegen id : 1] @@ -443,7 +443,7 @@ Arguments: 100, [i_category#24 ASC NULLS FIRST, i_class#25 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (76) -+- * ColumnarToRow (75) ++- * CometColumnarToRow (75) +- CometProject (74) +- CometFilter (73) +- CometScan parquet spark_catalog.default.date_dim (72) @@ -464,7 +464,7 @@ Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_mont Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(75) ColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] (76) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt index cc9627e89b..8c907f510a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 WholeStageCodegen (1) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] @@ -32,7 +32,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt index ccef440881..359b764e83 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject (58) : +- * HashAggregate (37) : +- * Project (36) : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * ColumnarToRow (9) + : :- * CometColumnarToRow (9) : : +- CometProject (8) : : +- CometBroadcastHashJoin (7) : : :- CometFilter (2) @@ -24,7 +24,7 @@ TakeOrderedAndProject (58) : : +- CometScan parquet spark_catalog.default.date_dim (3) : +- BroadcastExchange (34) : +- * BroadcastHashJoin LeftSemi BuildRight (33) - : :- * ColumnarToRow (12) + : :- * CometColumnarToRow (12) : : +- CometFilter (11) : : +- CometScan parquet spark_catalog.default.store (10) : +- BroadcastExchange (32) @@ -36,7 +36,7 @@ TakeOrderedAndProject (58) : +- * HashAggregate (26) : +- Exchange (25) : +- * HashAggregate (24) - : +- * ColumnarToRow (23) + : +- * CometColumnarToRow (23) : +- CometProject (22) : +- CometBroadcastHashJoin (21) : :- CometProject (19) @@ -99,7 +99,7 @@ Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] -(9) ColumnarToRow [codegen id : 5] +(9) CometColumnarToRow [codegen id : 5] Input [2]: [ss_store_sk#1, ss_net_profit#2] (10) CometScan parquet spark_catalog.default.store @@ -113,7 +113,7 @@ ReadSchema: struct Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) ColumnarToRow [codegen id : 4] +(12) CometColumnarToRow [codegen id : 4] Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) CometScan parquet spark_catalog.default.store_sales @@ -164,7 +164,7 @@ Arguments: [ss_sold_date_sk#12], [d_date_sk#16], Inner, BuildRight Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#15, d_date_sk#16] Arguments: [ss_net_profit#11, s_state#15], [ss_net_profit#11, s_state#15] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarToRow [codegen id : 1] Input [2]: [ss_net_profit#11, s_state#15] (24) HashAggregate [codegen id : 1] @@ -347,7 +347,7 @@ Arguments: 100, [lochierarchy#28 DESC NULLS LAST, CASE WHEN (lochierarchy#28 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (63) -+- * ColumnarToRow (62) ++- * CometColumnarToRow (62) +- CometProject (61) +- CometFilter (60) +- CometScan parquet spark_catalog.default.date_dim (59) @@ -368,7 +368,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(62) ColumnarToRow [codegen id : 1] +(62) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (63) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt index 67f4841b4f..3bdde5cb86 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt @@ -23,7 +23,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] Project [ss_net_profit,s_county,s_state] BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] @@ -32,7 +32,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -45,7 +45,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #6 WholeStageCodegen (4) BroadcastHashJoin [s_state,s_state] - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] @@ -64,7 +64,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count Exchange [s_state] #8 WholeStageCodegen (1) HashAggregate [s_state,ss_net_profit] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_net_profit,s_state] CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/explain.txt index 45f7e2e66d..4b97b311be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (64) +* CometColumnarToRow (64) +- CometTakeOrderedAndProject (63) +- CometHashAggregate (62) +- CometExchange (61) @@ -364,14 +364,14 @@ Functions [1]: [count(1)] Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, [total_cnt#37 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] -(64) ColumnarToRow [codegen id : 1] +(64) CometColumnarToRow [codegen id : 1] Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (69) -+- * ColumnarToRow (68) ++- * CometColumnarToRow (68) +- CometProject (67) +- CometFilter (66) +- CometScan parquet spark_catalog.default.date_dim (65) @@ -392,7 +392,7 @@ Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_dat Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(68) ColumnarToRow [codegen id : 1] +(68) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (69) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt index ccd1b66fa2..fe26a3f5cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count,count(1)] @@ -32,7 +32,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] CometFilter [d_date_sk,d_date,d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/explain.txt index 17b7fb0b81..471e32ad3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/explain.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject (71) : : : +- * HashAggregate (16) : : : +- Exchange (15) : : : +- * HashAggregate (14) - : : : +- * ColumnarToRow (13) + : : : +- * CometColumnarToRow (13) : : : +- CometProject (12) : : : +- CometBroadcastHashJoin (11) : : : :- CometProject (7) @@ -26,7 +26,7 @@ TakeOrderedAndProject (71) : : +- * HashAggregate (33) : : +- Exchange (32) : : +- * HashAggregate (31) - : : +- * ColumnarToRow (30) + : : +- * CometColumnarToRow (30) : : +- CometProject (29) : : +- CometBroadcastHashJoin (28) : : :- CometProject (24) @@ -44,7 +44,7 @@ TakeOrderedAndProject (71) : +- * HashAggregate (49) : +- Exchange (48) : +- * HashAggregate (47) - : +- * ColumnarToRow (46) + : +- * CometColumnarToRow (46) : +- CometProject (45) : +- CometBroadcastHashJoin (44) : :- CometProject (42) @@ -59,7 +59,7 @@ TakeOrderedAndProject (71) +- * HashAggregate (67) +- Exchange (66) +- * HashAggregate (65) - +- * ColumnarToRow (64) + +- * CometColumnarToRow (64) +- CometProject (63) +- CometBroadcastHashJoin (62) :- CometProject (60) @@ -132,7 +132,7 @@ Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7, d_date_sk#9, d_year#10] Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10], [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] -(13) ColumnarToRow [codegen id : 1] +(13) CometColumnarToRow [codegen id : 1] Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] (14) HashAggregate [codegen id : 1] @@ -217,7 +217,7 @@ Arguments: [ss_sold_date_sk#22], [d_date_sk#24], Inner, BuildRight Input [7]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22, d_date_sk#24, d_year#25] Arguments: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25], [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] -(30) ColumnarToRow [codegen id : 2] +(30) CometColumnarToRow [codegen id : 2] Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] (31) HashAggregate [codegen id : 2] @@ -296,7 +296,7 @@ Arguments: [ws_sold_date_sk#38], [d_date_sk#40], Inner, BuildRight Input [7]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38, d_date_sk#40, d_year#41] Arguments: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41], [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] -(46) ColumnarToRow [codegen id : 4] +(46) CometColumnarToRow [codegen id : 4] Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] (47) HashAggregate [codegen id : 4] @@ -383,7 +383,7 @@ Arguments: [ws_sold_date_sk#53], [d_date_sk#55], Inner, BuildRight Input [7]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53, d_date_sk#55, d_year#56] Arguments: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56], [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] -(64) ColumnarToRow [codegen id : 6] +(64) CometColumnarToRow [codegen id : 6] Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] (65) HashAggregate [codegen id : 6] @@ -426,7 +426,7 @@ Arguments: 100, [customer_first_name#29 ASC NULLS FIRST, customer_id#28 ASC NULL Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (75) -+- * ColumnarToRow (74) ++- * CometColumnarToRow (74) +- CometFilter (73) +- CometScan parquet spark_catalog.default.date_dim (72) @@ -442,7 +442,7 @@ ReadSchema: struct Input [2]: [d_date_sk#9, d_year#10] Condition : (((isnotnull(d_year#10) AND (d_year#10 = 2001)) AND d_year#10 IN (2001,2002)) AND isnotnull(d_date_sk#9)) -(74) ColumnarToRow [codegen id : 1] +(74) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] (75) BroadcastExchange @@ -451,7 +451,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 BroadcastExchange (79) -+- * ColumnarToRow (78) ++- * CometColumnarToRow (78) +- CometFilter (77) +- CometScan parquet spark_catalog.default.date_dim (76) @@ -467,7 +467,7 @@ ReadSchema: struct Input [2]: [d_date_sk#24, d_year#25] Condition : (((isnotnull(d_year#25) AND (d_year#25 = 2002)) AND d_year#25 IN (2001,2002)) AND isnotnull(d_date_sk#24)) -(78) ColumnarToRow [codegen id : 1] +(78) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#24, d_year#25] (79) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt index 4360a95551..d608c355d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt @@ -11,7 +11,7 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] Exchange [c_customer_id,c_first_name,c_last_name,d_year] #1 WholeStageCodegen (1) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] @@ -25,7 +25,7 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -40,7 +40,7 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] Exchange [c_customer_id,c_first_name,c_last_name,d_year] #6 WholeStageCodegen (2) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] @@ -54,7 +54,7 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -70,7 +70,7 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] Exchange [c_customer_id,c_first_name,c_last_name,d_year] #11 WholeStageCodegen (4) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] @@ -91,7 +91,7 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] Exchange [c_customer_id,c_first_name,c_last_name,d_year] #14 WholeStageCodegen (6) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/explain.txt index 109d01b5aa..5f3c02db34 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/explain.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject (125) : +- * HashAggregate (64) : +- Exchange (63) : +- * HashAggregate (62) - : +- * ColumnarToRow (61) + : +- * CometColumnarToRow (61) : +- CometHashAggregate (60) : +- CometExchange (59) : +- CometHashAggregate (58) @@ -75,7 +75,7 @@ TakeOrderedAndProject (125) +- * HashAggregate (119) +- Exchange (118) +- * HashAggregate (117) - +- * ColumnarToRow (116) + +- * CometColumnarToRow (116) +- CometHashAggregate (115) +- CometExchange (114) +- CometHashAggregate (113) @@ -401,7 +401,7 @@ Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_ Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] -(61) ColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [codegen id : 1] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] (62) HashAggregate [codegen id : 1] @@ -645,7 +645,7 @@ Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufac Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] Functions: [] -(116) ColumnarToRow [codegen id : 4] +(116) CometColumnarToRow [codegen id : 4] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] (117) HashAggregate [codegen id : 4] @@ -696,7 +696,7 @@ Arguments: 100, [sales_cnt_diff#137 ASC NULLS FIRST, sales_amt_diff#138 ASC NULL Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (129) -+- * ColumnarToRow (128) ++- * CometColumnarToRow (128) +- CometFilter (127) +- CometScan parquet spark_catalog.default.date_dim (126) @@ -712,7 +712,7 @@ ReadSchema: struct Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(128) ColumnarToRow [codegen id : 1] +(128) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] (129) BroadcastExchange @@ -725,7 +725,7 @@ Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN d Subquery:4 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 BroadcastExchange (133) -+- * ColumnarToRow (132) ++- * CometColumnarToRow (132) +- CometFilter (131) +- CometScan parquet spark_catalog.default.date_dim (130) @@ -741,7 +741,7 @@ ReadSchema: struct Input [2]: [d_date_sk#81, d_year#82] Condition : ((isnotnull(d_year#82) AND (d_year#82 = 2001)) AND isnotnull(d_date_sk#81)) -(132) ColumnarToRow [codegen id : 1] +(132) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#81, d_year#82] (133) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt index a80e7c7ef6..0b12c05866 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 WholeStageCodegen (1) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 @@ -33,7 +33,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -97,7 +97,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 WholeStageCodegen (4) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 @@ -116,7 +116,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/explain.txt index 5780ffc264..3d22b34198 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/explain.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject (99) : : :- * HashAggregate (17) : : : +- Exchange (16) : : : +- * HashAggregate (15) - : : : +- * ColumnarToRow (14) + : : : +- * CometColumnarToRow (14) : : : +- CometProject (13) : : : +- CometBroadcastHashJoin (12) : : : :- CometProject (8) @@ -31,7 +31,7 @@ TakeOrderedAndProject (99) : : +- * HashAggregate (29) : : +- Exchange (28) : : +- * HashAggregate (27) - : : +- * ColumnarToRow (26) + : : +- * CometColumnarToRow (26) : : +- CometProject (25) : : +- CometBroadcastHashJoin (24) : : :- CometProject (22) @@ -46,7 +46,7 @@ TakeOrderedAndProject (99) : : : +- * HashAggregate (40) : : : +- Exchange (39) : : : +- * HashAggregate (38) - : : : +- * ColumnarToRow (37) + : : : +- * CometColumnarToRow (37) : : : +- CometProject (36) : : : +- CometBroadcastHashJoin (35) : : : :- CometScan parquet spark_catalog.default.catalog_sales (33) @@ -54,7 +54,7 @@ TakeOrderedAndProject (99) : : +- * HashAggregate (49) : : +- Exchange (48) : : +- * HashAggregate (47) - : : +- * ColumnarToRow (46) + : : +- * CometColumnarToRow (46) : : +- CometProject (45) : : +- CometBroadcastHashJoin (44) : : :- CometScan parquet spark_catalog.default.catalog_returns (42) @@ -64,7 +64,7 @@ TakeOrderedAndProject (99) : :- * HashAggregate (65) : : +- Exchange (64) : : +- * HashAggregate (63) - : : +- * ColumnarToRow (62) + : : +- * CometColumnarToRow (62) : : +- CometProject (61) : : +- CometBroadcastHashJoin (60) : : :- CometProject (56) @@ -79,7 +79,7 @@ TakeOrderedAndProject (99) : +- * HashAggregate (77) : +- Exchange (76) : +- * HashAggregate (75) - : +- * ColumnarToRow (74) + : +- * CometColumnarToRow (74) : +- CometProject (73) : +- CometBroadcastHashJoin (72) : :- CometProject (70) @@ -164,7 +164,7 @@ Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] Arguments: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8], [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] (15) HashAggregate [codegen id : 1] @@ -221,7 +221,7 @@ Arguments: [sr_store_sk#17], [s_store_sk#23], Inner, BuildRight Input [4]: [sr_store_sk#17, sr_return_amt#18, sr_net_loss#19, s_store_sk#23] Arguments: [sr_return_amt#18, sr_net_loss#19, s_store_sk#23], [sr_return_amt#18, sr_net_loss#19, s_store_sk#23] -(26) ColumnarToRow [codegen id : 2] +(26) CometColumnarToRow [codegen id : 2] Input [3]: [sr_return_amt#18, sr_net_loss#19, s_store_sk#23] (27) HashAggregate [codegen id : 2] @@ -275,7 +275,7 @@ Arguments: [cs_sold_date_sk#39], [d_date_sk#41], Inner, BuildRight Input [5]: [cs_call_center_sk#36, cs_ext_sales_price#37, cs_net_profit#38, cs_sold_date_sk#39, d_date_sk#41] Arguments: [cs_call_center_sk#36, cs_ext_sales_price#37, cs_net_profit#38], [cs_call_center_sk#36, cs_ext_sales_price#37, cs_net_profit#38] -(37) ColumnarToRow [codegen id : 5] +(37) CometColumnarToRow [codegen id : 5] Input [3]: [cs_call_center_sk#36, cs_ext_sales_price#37, cs_net_profit#38] (38) HashAggregate [codegen id : 5] @@ -319,7 +319,7 @@ Arguments: [cr_returned_date_sk#52], [d_date_sk#54], Inner, BuildRight Input [4]: [cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52, d_date_sk#54] Arguments: [cr_return_amount#50, cr_net_loss#51], [cr_return_amount#50, cr_net_loss#51] -(46) ColumnarToRow [codegen id : 7] +(46) CometColumnarToRow [codegen id : 7] Input [2]: [cr_return_amount#50, cr_net_loss#51] (47) HashAggregate [codegen id : 7] @@ -396,7 +396,7 @@ Arguments: [ws_web_page_sk#66], [wp_web_page_sk#72], Inner, BuildRight Input [4]: [ws_web_page_sk#66, ws_ext_sales_price#67, ws_net_profit#68, wp_web_page_sk#72] Arguments: [ws_ext_sales_price#67, ws_net_profit#68, wp_web_page_sk#72], [ws_ext_sales_price#67, ws_net_profit#68, wp_web_page_sk#72] -(62) ColumnarToRow [codegen id : 9] +(62) CometColumnarToRow [codegen id : 9] Input [3]: [ws_ext_sales_price#67, ws_net_profit#68, wp_web_page_sk#72] (63) HashAggregate [codegen id : 9] @@ -453,7 +453,7 @@ Arguments: [wr_web_page_sk#81], [wp_web_page_sk#87], Inner, BuildRight Input [4]: [wr_web_page_sk#81, wr_return_amt#82, wr_net_loss#83, wp_web_page_sk#87] Arguments: [wr_return_amt#82, wr_net_loss#83, wp_web_page_sk#87], [wr_return_amt#82, wr_net_loss#83, wp_web_page_sk#87] -(74) ColumnarToRow [codegen id : 10] +(74) CometColumnarToRow [codegen id : 10] Input [3]: [wr_return_amt#82, wr_net_loss#83, wp_web_page_sk#87] (75) HashAggregate [codegen id : 10] @@ -592,7 +592,7 @@ Arguments: 100, [channel#32 ASC NULLS FIRST, id#33 ASC NULLS FIRST], [channel#32 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (104) -+- * ColumnarToRow (103) ++- * CometColumnarToRow (103) +- CometProject (102) +- CometFilter (101) +- CometScan parquet spark_catalog.default.date_dim (100) @@ -613,7 +613,7 @@ Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 < Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(103) ColumnarToRow [codegen id : 1] +(103) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (104) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt index db4e102473..456d92df41 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt @@ -23,7 +23,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Exchange [s_store_sk] #3 WholeStageCodegen (1) HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] @@ -34,7 +34,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] @@ -54,7 +54,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Exchange [s_store_sk] #8 WholeStageCodegen (2) HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [sr_return_amt,sr_net_loss,s_store_sk] CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] @@ -76,7 +76,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Exchange [cs_call_center_sk] #10 WholeStageCodegen (5) HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] @@ -88,7 +88,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Exchange #11 WholeStageCodegen (7) HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cr_return_amount,cr_net_loss] CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] @@ -103,7 +103,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Exchange [wp_web_page_sk] #12 WholeStageCodegen (9) HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] @@ -124,7 +124,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Exchange [wp_web_page_sk] #15 WholeStageCodegen (10) HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/explain.txt index b15e683bd4..6c37484ce6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/explain.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject (69) : : +- * HashAggregate (21) : : +- Exchange (20) : : +- * HashAggregate (19) - : : +- * ColumnarToRow (18) + : : +- * CometColumnarToRow (18) : : +- CometProject (17) : : +- CometBroadcastHashJoin (16) : : :- CometProject (12) @@ -31,7 +31,7 @@ TakeOrderedAndProject (69) : +- * HashAggregate (41) : +- Exchange (40) : +- * HashAggregate (39) - : +- * ColumnarToRow (38) + : +- * CometColumnarToRow (38) : +- CometProject (37) : +- CometBroadcastHashJoin (36) : :- CometProject (34) @@ -52,7 +52,7 @@ TakeOrderedAndProject (69) +- * HashAggregate (64) +- Exchange (63) +- * HashAggregate (62) - +- * ColumnarToRow (61) + +- * CometColumnarToRow (61) +- CometProject (60) +- CometBroadcastHashJoin (59) :- CometProject (57) @@ -150,7 +150,7 @@ Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -(18) ColumnarToRow [codegen id : 1] +(18) CometColumnarToRow [codegen id : 1] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] (19) HashAggregate [codegen id : 1] @@ -243,7 +243,7 @@ Arguments: [ws_sold_date_sk#33], [d_date_sk#38], Inner, BuildRight Input [8]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, d_date_sk#38, d_year#39] Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39], [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] -(38) ColumnarToRow [codegen id : 3] +(38) CometColumnarToRow [codegen id : 3] Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] (39) HashAggregate [codegen id : 3] @@ -350,7 +350,7 @@ Arguments: [cs_sold_date_sk#60], [d_date_sk#65], Inner, BuildRight Input [8]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, d_date_sk#65, d_year#66] Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66], [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] -(61) ColumnarToRow [codegen id : 6] +(61) CometColumnarToRow [codegen id : 6] Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] (62) HashAggregate [codegen id : 6] @@ -397,7 +397,7 @@ Arguments: 100, [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (73) -+- * ColumnarToRow (72) ++- * CometColumnarToRow (72) +- CometFilter (71) +- CometScan parquet spark_catalog.default.date_dim (70) @@ -413,7 +413,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(72) ColumnarToRow [codegen id : 1] +(72) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] (73) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt index 9e94433e61..cd90f93b44 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp Exchange [d_year,ss_item_sk,ss_customer_sk] #1 WholeStageCodegen (1) HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] @@ -28,7 +28,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -49,7 +49,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp Exchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 WholeStageCodegen (3) HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] @@ -76,7 +76,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp Exchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 WholeStageCodegen (6) HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/explain.txt index 07d7277d6a..4761544891 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/explain.txt @@ -11,7 +11,7 @@ TakeOrderedAndProject (115) : :- * HashAggregate (38) : : +- Exchange (37) : : +- * HashAggregate (36) - : : +- * ColumnarToRow (35) + : : +- * CometColumnarToRow (35) : : +- CometProject (34) : : +- CometBroadcastHashJoin (33) : : :- CometProject (28) @@ -49,7 +49,7 @@ TakeOrderedAndProject (115) : :- * HashAggregate (67) : : +- Exchange (66) : : +- * HashAggregate (65) - : : +- * ColumnarToRow (64) + : : +- * CometColumnarToRow (64) : : +- CometProject (63) : : +- CometBroadcastHashJoin (62) : : :- CometProject (60) @@ -78,7 +78,7 @@ TakeOrderedAndProject (115) : +- * HashAggregate (96) : +- Exchange (95) : +- * HashAggregate (94) - : +- * ColumnarToRow (93) + : +- * CometColumnarToRow (93) : +- CometProject (92) : +- CometBroadcastHashJoin (91) : :- CometProject (89) @@ -276,7 +276,7 @@ Arguments: [ss_promo_sk#3], [p_promo_sk#20], Inner, BuildRight Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17, p_promo_sk#20] Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] -(35) ColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [codegen id : 1] Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] (36) HashAggregate [codegen id : 1] @@ -409,7 +409,7 @@ Arguments: [cs_promo_sk#42], [p_promo_sk#57], Inner, BuildRight Input [7]: [cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55, p_promo_sk#57] Arguments: [cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55], [cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55] -(64) ColumnarToRow [codegen id : 3] +(64) CometColumnarToRow [codegen id : 3] Input [5]: [cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55] (65) HashAggregate [codegen id : 3] @@ -542,7 +542,7 @@ Arguments: [ws_promo_sk#78], [p_promo_sk#93], Inner, BuildRight Input [7]: [ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91, p_promo_sk#93] Arguments: [ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91], [ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91] -(93) ColumnarToRow [codegen id : 5] +(93) CometColumnarToRow [codegen id : 5] Input [5]: [ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91] (94) HashAggregate [codegen id : 5] @@ -667,7 +667,7 @@ Arguments: 100, [channel#35 ASC NULLS FIRST, id#36 ASC NULLS FIRST], [channel#35 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (120) -+- * ColumnarToRow (119) ++- * CometColumnarToRow (119) +- CometProject (118) +- CometFilter (117) +- CometScan parquet spark_catalog.default.date_dim (116) @@ -688,7 +688,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(119) ColumnarToRow [codegen id : 1] +(119) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] (120) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt index 75dbc06d7f..beb098e869 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Exchange [s_store_id] #3 WholeStageCodegen (1) HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] @@ -40,7 +40,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] @@ -71,7 +71,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Exchange [cp_catalog_page_id] #11 WholeStageCodegen (3) HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] @@ -105,7 +105,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Exchange [web_site_id] #15 WholeStageCodegen (5) HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt index 3f5b2d6b2b..9c5e692daa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt @@ -11,7 +11,7 @@ TakeOrderedAndProject (36) :- * HashAggregate (17) : +- Exchange (16) : +- * HashAggregate (15) - : +- * ColumnarToRow (14) + : +- * CometColumnarToRow (14) : +- CometProject (13) : +- CometBroadcastHashJoin (12) : :- CometProject (8) @@ -101,7 +101,7 @@ Arguments: [ws_item_sk#1], [i_item_sk#7], Inner, BuildRight Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#7, i_class#8, i_category#9] Arguments: [ws_net_paid#2, i_class#8, i_category#9], [ws_net_paid#2, i_class#8, i_category#9] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [3]: [ws_net_paid#2, i_class#8, i_category#9] (15) HashAggregate [codegen id : 1] @@ -222,7 +222,7 @@ Arguments: 100, [lochierarchy#16 DESC NULLS LAST, CASE WHEN (lochierarchy#16 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (41) -+- * ColumnarToRow (40) ++- * CometColumnarToRow (40) +- CometProject (39) +- CometFilter (38) +- CometScan parquet spark_catalog.default.date_dim (37) @@ -243,7 +243,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(40) ColumnarToRow [codegen id : 1] +(40) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (41) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt index 8996119f2c..4bcd56748a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl Exchange [i_category,i_class] #3 WholeStageCodegen (1) HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_net_paid,i_class,i_category] CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] @@ -32,7 +32,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt index 61ef88e69c..d4726a7e03 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (24) +* CometColumnarToRow (24) +- CometSort (23) +- CometColumnarExchange (22) +- * Project (21) @@ -9,7 +9,7 @@ +- * HashAggregate (17) +- Exchange (16) +- * HashAggregate (15) - +- * ColumnarToRow (14) + +- * CometColumnarToRow (14) +- CometProject (13) +- CometBroadcastHashJoin (12) :- CometProject (7) @@ -89,7 +89,7 @@ Arguments: [ss_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] Arguments: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [6]: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] (15) HashAggregate [codegen id : 1] @@ -134,14 +134,14 @@ Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19] Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19], [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#19 ASC NULLS FIRST] -(24) ColumnarToRow [codegen id : 5] +(24) CometColumnarToRow [codegen id : 5] Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * ColumnarToRow (28) ++- * CometColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan parquet spark_catalog.default.date_dim (25) @@ -162,7 +162,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt index a08ff0a3a2..4ce976ae30 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 @@ -17,7 +17,7 @@ WholeStageCodegen (5) Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 WholeStageCodegen (1) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] @@ -28,7 +28,7 @@ WholeStageCodegen (5) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt index 868b2f4822..c2b7f49205 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (39) +* CometColumnarToRow (39) +- CometTakeOrderedAndProject (38) +- CometHashAggregate (37) +- CometExchange (36) @@ -220,14 +220,14 @@ Functions [1]: [count(1)] Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34, cd_dep_count#28, cnt4#35, cd_dep_employed_count#29, cnt5#36, cd_dep_college_count#30, cnt6#37] Arguments: TakeOrderedAndProject(limit=100, orderBy=[cd_gender#23 ASC NULLS FIRST,cd_marital_status#24 ASC NULLS FIRST,cd_education_status#25 ASC NULLS FIRST,cd_purchase_estimate#26 ASC NULLS FIRST,cd_credit_rating#27 ASC NULLS FIRST,cd_dep_count#28 ASC NULLS FIRST,cd_dep_employed_count#29 ASC NULLS FIRST,cd_dep_college_count#30 ASC NULLS FIRST], output=[cd_gender#23,cd_marital_status#24,cd_education_status#25,cnt1#32,cd_purchase_estimate#26,cnt2#33,cd_credit_rating#27,cnt3#34,cd_dep_count#28,cnt4#35,cd_dep_employed_count#29,cnt5#36,cd_dep_college_count#30,cnt6#37]), [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34, cd_dep_count#28, cnt4#35, cd_dep_employed_count#29, cnt5#36, cd_dep_college_count#30, cnt6#37], 100, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34, cd_dep_count#28, cnt4#35, cd_dep_employed_count#29, cnt5#36, cd_dep_college_count#30, cnt6#37] -(39) ColumnarToRow [codegen id : 1] +(39) CometColumnarToRow [codegen id : 1] Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34, cd_dep_count#28, cnt4#35, cd_dep_employed_count#29, cnt5#36, cd_dep_college_count#30, cnt6#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (44) -+- * ColumnarToRow (43) ++- * CometColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan parquet spark_catalog.default.date_dim (40) @@ -248,7 +248,7 @@ Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 200 Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(43) ColumnarToRow [codegen id : 1] +(43) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt index 2e8f2022cf..b570b700ac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6] CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6,count,count(1)] @@ -21,7 +21,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt index 689697306d..0bcd9798e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (68) +* CometColumnarToRow (68) +- CometTakeOrderedAndProject (67) +- CometProject (66) +- CometBroadcastHashJoin (65) @@ -388,14 +388,14 @@ Arguments: [customer_id#35, customer_first_name#36, customer_last_name#37, custo Input [4]: [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38] Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#35 ASC NULLS FIRST,customer_first_name#36 ASC NULLS FIRST,customer_last_name#37 ASC NULLS FIRST,customer_email_address#38 ASC NULLS FIRST], output=[customer_id#35,customer_first_name#36,customer_last_name#37,customer_email_address#38]), [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38], 100, [customer_id#35 ASC NULLS FIRST, customer_first_name#36 ASC NULLS FIRST, customer_last_name#37 ASC NULLS FIRST, customer_email_address#38 ASC NULLS FIRST], [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38] -(68) ColumnarToRow [codegen id : 1] +(68) CometColumnarToRow [codegen id : 1] Input [4]: [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (72) -+- * ColumnarToRow (71) ++- * CometColumnarToRow (71) +- CometFilter (70) +- CometScan parquet spark_catalog.default.date_dim (69) @@ -411,7 +411,7 @@ ReadSchema: struct Input [2]: [d_date_sk#14, d_year#15] Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#14)) -(71) ColumnarToRow [codegen id : 1] +(71) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] (72) BroadcastExchange @@ -420,7 +420,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#31 BroadcastExchange (76) -+- * ColumnarToRow (75) ++- * CometColumnarToRow (75) +- CometFilter (74) +- CometScan parquet spark_catalog.default.date_dim (73) @@ -436,7 +436,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 2002)) AND isnotnull(d_date_sk#32)) -(75) ColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] (76) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt index 8743dd42ef..987c965c03 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] CometProject [customer_id,customer_first_name,customer_last_name,customer_email_address] @@ -23,7 +23,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -46,7 +46,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt index a428886298..b8d68b0a31 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (22) +- * Project (21) +- Window (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometSort (18) +- CometExchange (17) +- CometHashAggregate (16) @@ -109,7 +109,7 @@ Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometNativeShuff Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15], [i_class#9 ASC NULLS FIRST] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] (20) Window @@ -128,7 +128,7 @@ Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_ite Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (27) -+- * ColumnarToRow (26) ++- * CometColumnarToRow (26) +- CometProject (25) +- CometFilter (24) +- CometScan parquet spark_catalog.default.date_dim (23) @@ -149,7 +149,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (27) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt index 6f51be0c5b..4a829e8612 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] CometExchange [i_class] #1 @@ -20,7 +20,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt index 31a11bba90..5c694fb53c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (85) +* CometColumnarToRow (85) +- CometTakeOrderedAndProject (84) +- CometBroadcastHashJoin (83) :- CometFilter (64) @@ -480,13 +480,13 @@ Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#61, i_c Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#50,i_brand_id#40,i_class_id#41,i_category_id#42,sales#51,number_sales#52,channel#71,i_brand_id#61,i_class_id#62,i_category_id#63,sales#72,number_sales#73]), [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73], 100, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] -(85) ColumnarToRow [codegen id : 1] +(85) CometColumnarToRow [codegen id : 1] Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] ===== Subqueries ===== Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* ColumnarToRow (102) +* CometColumnarToRow (102) +- CometHashAggregate (101) +- CometExchange (100) +- CometHashAggregate (99) @@ -581,7 +581,7 @@ Input [2]: [sum#95, count#96] Keys: [] Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] -(102) ColumnarToRow [codegen id : 1] +(102) CometColumnarToRow [codegen id : 1] Input [1]: [average_sales#97] Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#12 @@ -592,7 +592,7 @@ Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#90 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (107) -+- * ColumnarToRow (106) ++- * CometColumnarToRow (106) +- CometProject (105) +- CometFilter (104) +- CometScan parquet spark_catalog.default.date_dim (103) @@ -613,7 +613,7 @@ Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subq Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(106) ColumnarToRow [codegen id : 1] +(106) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] (107) BroadcastExchange @@ -621,7 +621,7 @@ Input [1]: [d_date_sk#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* ColumnarToRow (111) +* CometColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometScan parquet spark_catalog.default.date_dim (108) @@ -642,12 +642,12 @@ Condition : (((((isnotnull(d_year#99) AND isnotnull(d_moy#100)) AND isnotnull(d_ Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] Arguments: [d_week_seq#98], [d_week_seq#98] -(111) ColumnarToRow [codegen id : 1] +(111) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#98] Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (116) -+- * ColumnarToRow (115) ++- * CometColumnarToRow (115) +- CometProject (114) +- CometFilter (113) +- CometScan parquet spark_catalog.default.date_dim (112) @@ -668,7 +668,7 @@ Condition : (((isnotnull(d_year#102) AND (d_year#102 >= 1998)) AND (d_year#102 < Input [2]: [d_date_sk#26, d_year#102] Arguments: [d_date_sk#26], [d_date_sk#26] -(115) ColumnarToRow [codegen id : 1] +(115) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] (116) BroadcastExchange @@ -685,7 +685,7 @@ Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquer Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 BroadcastExchange (121) -+- * ColumnarToRow (120) ++- * CometColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan parquet spark_catalog.default.date_dim (117) @@ -706,7 +706,7 @@ Condition : ((isnotnull(d_week_seq#65) AND (d_week_seq#65 = Subquery scalar-subq Input [2]: [d_date_sk#64, d_week_seq#65] Arguments: [d_date_sk#64], [d_date_sk#64] -(120) ColumnarToRow [codegen id : 1] +(120) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#64] (121) BroadcastExchange @@ -714,7 +714,7 @@ Input [1]: [d_date_sk#64] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#66, [id=#67] -* ColumnarToRow (125) +* CometColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometScan parquet spark_catalog.default.date_dim (122) @@ -735,7 +735,7 @@ Condition : (((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND isnotnull(d Input [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] Arguments: [d_week_seq#103], [d_week_seq#103] -(125) ColumnarToRow [codegen id : 1] +(125) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#103] Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt index bb4be4d3ae..befc6dadbb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt @@ -1,12 +1,12 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] CometBroadcastHashJoin [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Subquery #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #14 @@ -40,13 +40,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_week_seq] Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_week_seq,d_year,d_moy,d_dom] @@ -71,7 +71,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -132,13 +132,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_week_seq] Subquery #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_week_seq,d_year,d_moy,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt index 4bc928e635..124efb8084 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (122) +* CometColumnarToRow (122) +- CometTakeOrderedAndProject (121) +- CometHashAggregate (120) +- CometExchange (119) @@ -682,13 +682,13 @@ Functions: [] Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#49 ASC NULLS FIRST,i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#49,i_brand_id#40,i_class_id#41,i_category_id#42,sum_sales#89,number_sales#90]), [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90], 100, [channel#49 ASC NULLS FIRST, i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] -(122) ColumnarToRow [codegen id : 1] +(122) CometColumnarToRow [codegen id : 1] Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] ===== Subqueries ===== Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* ColumnarToRow (142) +* CometColumnarToRow (142) +- CometHashAggregate (141) +- CometExchange (140) +- CometHashAggregate (139) @@ -802,14 +802,14 @@ Input [2]: [sum#143, count#144] Keys: [] Functions [1]: [avg((cast(quantity#126 as decimal(10,0)) * list_price#127))] -(142) ColumnarToRow [codegen id : 1] +(142) CometColumnarToRow [codegen id : 1] Input [1]: [average_sales#145] Subquery:2 Hosting operator id = 123 Hosting Expression = ss_sold_date_sk#123 IN dynamicpruning#12 Subquery:3 Hosting operator id = 127 Hosting Expression = cs_sold_date_sk#130 IN dynamicpruning#131 BroadcastExchange (147) -+- * ColumnarToRow (146) ++- * CometColumnarToRow (146) +- CometProject (145) +- CometFilter (144) +- CometScan parquet spark_catalog.default.date_dim (143) @@ -830,7 +830,7 @@ Condition : (((isnotnull(d_year#133) AND (d_year#133 >= 1998)) AND (d_year#133 < Input [2]: [d_date_sk#132, d_year#133] Arguments: [d_date_sk#132], [d_date_sk#132] -(146) ColumnarToRow [codegen id : 1] +(146) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#132] (147) BroadcastExchange @@ -841,7 +841,7 @@ Subquery:4 Hosting operator id = 134 Hosting Expression = ws_sold_date_sk#138 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (152) -+- * ColumnarToRow (151) ++- * CometColumnarToRow (151) +- CometProject (150) +- CometFilter (149) +- CometScan parquet spark_catalog.default.date_dim (148) @@ -862,7 +862,7 @@ Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2 Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(151) ColumnarToRow [codegen id : 1] +(151) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] (152) BroadcastExchange @@ -871,7 +871,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (157) -+- * ColumnarToRow (156) ++- * CometColumnarToRow (156) +- CometProject (155) +- CometFilter (154) +- CometScan parquet spark_catalog.default.date_dim (153) @@ -892,7 +892,7 @@ Condition : (((isnotnull(d_year#146) AND (d_year#146 >= 1999)) AND (d_year#146 < Input [2]: [d_date_sk#26, d_year#146] Arguments: [d_date_sk#26], [d_date_sk#26] -(156) ColumnarToRow [codegen id : 1] +(156) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] (157) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index 760d0963ed..162f4f2447 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] @@ -13,7 +13,7 @@ WholeStageCodegen (1) CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Subquery #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #16 @@ -30,7 +30,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -57,7 +57,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -81,7 +81,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt index 5c25745ba0..6275dfe15e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (143) +* CometColumnarToRow (143) +- CometTakeOrderedAndProject (142) +- CometUnion (141) :- CometHashAggregate (38) @@ -785,14 +785,14 @@ Child 4 Input [11]: [i_item_id#144, ca_country#145, ca_state#146, county#147, ag Input [11]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#110, agg2#111, agg3#112, agg4#113, agg5#114, agg6#115, agg7#116] Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#24 ASC NULLS FIRST,ca_state#23 ASC NULLS FIRST,ca_county#22 ASC NULLS FIRST,i_item_id#28 ASC NULLS FIRST], output=[i_item_id#28,ca_country#24,ca_state#23,ca_county#22,agg1#110,agg2#111,agg3#112,agg4#113,agg5#114,agg6#115,agg7#116]), [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#110, agg2#111, agg3#112, agg4#113, agg5#114, agg6#115, agg7#116], 100, [ca_country#24 ASC NULLS FIRST, ca_state#23 ASC NULLS FIRST, ca_county#22 ASC NULLS FIRST, i_item_id#28 ASC NULLS FIRST], [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#110, agg2#111, agg3#112, agg4#113, agg5#114, agg6#115, agg7#116] -(143) ColumnarToRow [codegen id : 1] +(143) CometColumnarToRow [codegen id : 1] Input [11]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#110, agg2#111, agg3#112, agg4#113, agg5#114, agg6#115, agg7#116] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (148) -+- * ColumnarToRow (147) ++- * CometColumnarToRow (147) +- CometProject (146) +- CometFilter (145) +- CometScan parquet spark_catalog.default.date_dim (144) @@ -813,7 +813,7 @@ Condition : ((isnotnull(d_year#26) AND (d_year#26 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(147) ColumnarToRow [codegen id : 1] +(147) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] (148) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt index add76afad3..b974a7afce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometUnion [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] @@ -23,7 +23,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt index a0c02e00e9..0610ff5ce3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (22) +- * Project (21) +- Window (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometSort (18) +- CometExchange (17) +- CometHashAggregate (16) @@ -109,7 +109,7 @@ Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometNativeShuff Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15], [i_class#9 ASC NULLS FIRST] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] (20) Window @@ -128,7 +128,7 @@ Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_ite Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (27) -+- * ColumnarToRow (26) ++- * CometColumnarToRow (26) +- CometProject (25) +- CometFilter (24) +- CometScan parquet spark_catalog.default.date_dim (23) @@ -149,7 +149,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (27) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt index c4d5e48ae5..8e639b6fa0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] CometExchange [i_class] #1 @@ -20,7 +20,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt index 34d510fc71..5be661c351 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject (24) +- * Expand (20) +- * Project (19) +- * BroadcastNestedLoopJoin Inner BuildRight (18) - :- * ColumnarToRow (14) + :- * CometColumnarToRow (14) : +- CometProject (13) : +- CometBroadcastHashJoin (12) : :- CometProject (8) @@ -21,7 +21,7 @@ TakeOrderedAndProject (24) : +- CometFilter (10) : +- CometScan parquet spark_catalog.default.item (9) +- BroadcastExchange (17) - +- * ColumnarToRow (16) + +- * CometColumnarToRow (16) +- CometScan parquet spark_catalog.default.warehouse (15) @@ -89,7 +89,7 @@ Arguments: [inv_item_sk#1], [i_item_sk#7], Inner, BuildRight Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] Arguments: [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11], [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11] -(14) ColumnarToRow [codegen id : 2] +(14) CometColumnarToRow [codegen id : 2] Input [5]: [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11] (15) CometScan parquet spark_catalog.default.warehouse @@ -98,7 +98,7 @@ Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] ReadSchema: struct<> -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarToRow [codegen id : 1] Input: [] (17) BroadcastExchange @@ -143,7 +143,7 @@ Arguments: 100, [qoh#22 ASC NULLS FIRST, i_product_name#12 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * ColumnarToRow (28) ++- * CometColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan parquet spark_catalog.default.date_dim (25) @@ -164,7 +164,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt index 0e864ab701..027836b68c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] BroadcastNestedLoopJoin - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] @@ -19,7 +19,7 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -34,6 +34,6 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometScan parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt index b284c9b275..ce03b5946a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (46) +* CometColumnarToRow (46) +- CometTakeOrderedAndProject (45) +- CometUnion (44) :- CometHashAggregate (23) @@ -258,14 +258,14 @@ Child 4 Input [5]: [i_product_name#37, i_brand#38, i_class#39, i_category#40, qo Input [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#27] Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#27 ASC NULLS FIRST,i_product_name#12 ASC NULLS FIRST,i_brand#9 ASC NULLS FIRST,i_class#10 ASC NULLS FIRST,i_category#11 ASC NULLS FIRST], output=[i_product_name#12,i_brand#9,i_class#10,i_category#11,qoh#27]), [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#27], 100, [qoh#27 ASC NULLS FIRST, i_product_name#12 ASC NULLS FIRST, i_brand#9 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#27] -(46) ColumnarToRow [codegen id : 1] +(46) CometColumnarToRow [codegen id : 1] Input [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (51) -+- * ColumnarToRow (50) ++- * CometColumnarToRow (50) +- CometProject (49) +- CometFilter (48) +- CometScan parquet spark_catalog.default.date_dim (47) @@ -286,7 +286,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_mont Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (51) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt index 63eda27a6b..02f615b128 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] CometUnion [i_product_name,i_brand,i_class,i_category,qoh] @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt index 15584fda11..b93cfd311f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (45) +* CometColumnarToRow (45) +- CometSort (44) +- CometColumnarExchange (43) +- * Filter (42) @@ -11,7 +11,7 @@ +- * HashAggregate (36) +- * Project (35) +- * BroadcastHashJoin Inner BuildRight (34) - :- * ColumnarToRow (29) + :- * CometColumnarToRow (29) : +- CometProject (28) : +- CometBroadcastHashJoin (27) : :- CometProject (23) @@ -41,7 +41,7 @@ : +- CometFilter (25) : +- CometScan parquet spark_catalog.default.customer (24) +- BroadcastExchange (33) - +- * ColumnarToRow (32) + +- * CometColumnarToRow (32) +- CometFilter (31) +- CometScan parquet spark_catalog.default.customer_address (30) @@ -177,7 +177,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] Arguments: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25], [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -(29) ColumnarToRow [codegen id : 2] +(29) CometColumnarToRow [codegen id : 2] Input [13]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] (30) CometScan parquet spark_catalog.default.customer_address @@ -191,7 +191,7 @@ ReadSchema: struct= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_do Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt index 7da6076f5b..e1fc6bf43f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt index 5933ea5fad..fbf3086106 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt @@ -11,7 +11,7 @@ TakeOrderedAndProject (44) : : +- * Filter (27) : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * ColumnarToRow (12) + : : : :- * CometColumnarToRow (12) : : : : +- CometBroadcastHashJoin (11) : : : : :- CometFilter (2) : : : : : +- CometScan parquet spark_catalog.default.customer (1) @@ -24,23 +24,23 @@ TakeOrderedAndProject (44) : : : : +- CometFilter (5) : : : : +- CometScan parquet spark_catalog.default.date_dim (4) : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) + : : : +- * CometColumnarToRow (17) : : : +- CometProject (16) : : : +- CometBroadcastHashJoin (15) : : : :- CometScan parquet spark_catalog.default.web_sales (13) : : : +- ReusedExchange (14) : : +- BroadcastExchange (25) - : : +- * ColumnarToRow (24) + : : +- * CometColumnarToRow (24) : : +- CometProject (23) : : +- CometBroadcastHashJoin (22) : : :- CometScan parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (32) - : +- * ColumnarToRow (31) + : +- * CometColumnarToRow (31) : +- CometFilter (30) : +- CometScan parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (38) - +- * ColumnarToRow (37) + +- * CometColumnarToRow (37) +- CometFilter (36) +- CometScan parquet spark_catalog.default.customer_demographics (35) @@ -100,7 +100,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) ColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 5] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan parquet spark_catalog.default.web_sales @@ -122,7 +122,7 @@ Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#12] (18) BroadcastExchange @@ -154,7 +154,7 @@ Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [1]: [cs_ship_customer_sk#16] (25) BroadcastExchange @@ -186,7 +186,7 @@ ReadSchema: struct Input [2]: [ca_address_sk#20, ca_state#21] Condition : isnotnull(ca_address_sk#20) -(31) ColumnarToRow [codegen id : 3] +(31) CometColumnarToRow [codegen id : 3] Input [2]: [ca_address_sk#20, ca_state#21] (32) BroadcastExchange @@ -214,7 +214,7 @@ ReadSchema: struct Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (51) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt index a18f68d837..6355321c9b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 @@ -34,7 +34,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] @@ -51,7 +51,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 @@ -64,7 +64,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt index 685f048dea..fbd4aea034 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject (77) : +- Window (24) : +- * Sort (23) : +- Window (22) - : +- * ColumnarToRow (21) + : +- * CometColumnarToRow (21) : +- CometSort (20) : +- CometExchange (19) : +- CometHashAggregate (18) @@ -35,7 +35,7 @@ TakeOrderedAndProject (77) : +- Window (47) : +- * Sort (46) : +- Window (45) - : +- * ColumnarToRow (44) + : +- * CometColumnarToRow (44) : +- CometSort (43) : +- CometExchange (42) : +- CometHashAggregate (41) @@ -58,7 +58,7 @@ TakeOrderedAndProject (77) +- Window (70) +- * Sort (69) +- Window (68) - +- * ColumnarToRow (67) + +- * CometColumnarToRow (67) +- CometSort (66) +- CometExchange (65) +- CometHashAggregate (64) @@ -172,7 +172,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [3]: [item#22, return_ratio#23, currency_ratio#24] Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) ColumnarToRow [codegen id : 1] +(21) CometColumnarToRow [codegen id : 1] Input [3]: [item#22, return_ratio#23, currency_ratio#24] (22) Window @@ -273,7 +273,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] -(44) ColumnarToRow [codegen id : 4] +(44) CometColumnarToRow [codegen id : 4] Input [3]: [item#47, return_ratio#48, currency_ratio#49] (45) Window @@ -374,7 +374,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] -(67) ColumnarToRow [codegen id : 7] +(67) CometColumnarToRow [codegen id : 7] Input [3]: [item#72, return_ratio#73, currency_ratio#74] (68) Window @@ -425,7 +425,7 @@ Arguments: 100, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, cur Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (82) -+- * ColumnarToRow (81) ++- * CometColumnarToRow (81) +- CometProject (80) +- CometFilter (79) +- CometScan parquet spark_catalog.default.date_dim (78) @@ -446,7 +446,7 @@ Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2 Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(81) ColumnarToRow [codegen id : 1] +(81) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] (82) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt index d98b2b0a66..aaf75c375c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] InputAdapter Window [return_ratio] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item,return_ratio,currency_ratio] CometExchange #2 @@ -35,7 +35,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -57,7 +57,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] InputAdapter Window [return_ratio] WholeStageCodegen (4) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item,return_ratio,currency_ratio] CometExchange #7 @@ -87,7 +87,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] InputAdapter Window [return_ratio] WholeStageCodegen (7) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item,return_ratio,currency_ratio] CometExchange #10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt index 3211e46f6d..725868a338 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject (72) : : +- * BroadcastHashJoin Inner BuildRight (23) : : :- * Project (16) : : : +- Window (15) - : : : +- * ColumnarToRow (14) + : : : +- * CometColumnarToRow (14) : : : +- CometSort (13) : : : +- CometExchange (12) : : : +- CometHashAggregate (11) @@ -37,7 +37,7 @@ TakeOrderedAndProject (72) : : +- BroadcastExchange (22) : : +- * Project (21) : : +- Window (20) - : : +- * ColumnarToRow (19) + : : +- * CometColumnarToRow (19) : : +- CometSort (18) : : +- ReusedExchange (17) : +- * Sort (55) @@ -49,7 +49,7 @@ TakeOrderedAndProject (72) : +- * BroadcastHashJoin Inner BuildRight (49) : :- * Project (42) : : +- Window (41) - : : +- * ColumnarToRow (40) + : : +- * CometColumnarToRow (40) : : +- CometSort (39) : : +- CometExchange (38) : : +- CometHashAggregate (37) @@ -63,7 +63,7 @@ TakeOrderedAndProject (72) : +- BroadcastExchange (48) : +- * Project (47) : +- Window (46) - : +- * ColumnarToRow (45) + : +- * CometColumnarToRow (45) : +- CometSort (44) : +- ReusedExchange (43) +- BroadcastExchange (66) @@ -135,7 +135,7 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeSh Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] Arguments: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] (15) Window @@ -153,7 +153,7 @@ Output [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] Arguments: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13], [ws_item_sk#13 ASC NULLS FIRST, d_date#12 ASC NULLS FIRST] -(19) ColumnarToRow [codegen id : 2] +(19) CometColumnarToRow [codegen id : 2] Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] (20) Window @@ -250,7 +250,7 @@ Arguments: hashpartitioning(ss_item_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeS Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] Arguments: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23], [ss_item_sk#23 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] -(40) ColumnarToRow [codegen id : 7] +(40) CometColumnarToRow [codegen id : 7] Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] (41) Window @@ -268,7 +268,7 @@ Output [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] Arguments: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34], [ss_item_sk#34 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST] -(45) ColumnarToRow [codegen id : 8] +(45) CometColumnarToRow [codegen id : 8] Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] (46) Window @@ -400,7 +400,7 @@ Arguments: 100, [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], [item_s Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (77) -+- * ColumnarToRow (76) ++- * CometColumnarToRow (76) +- CometProject (75) +- CometFilter (74) +- CometScan parquet spark_catalog.default.date_dim (73) @@ -421,7 +421,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_mont Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(76) ColumnarToRow [codegen id : 1] +(76) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] (77) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt index f8e1ba2353..f592008d93 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store InputAdapter Window [ws_item_sk,d_date] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item_sk,d_date,sumws,ws_item_sk] CometExchange [ws_item_sk] #4 @@ -46,7 +46,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] @@ -62,7 +62,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store InputAdapter Window [ws_item_sk,d_date] WholeStageCodegen (2) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item_sk,d_date,sumws,ws_item_sk] ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 @@ -83,7 +83,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store InputAdapter Window [ss_item_sk,d_date] WholeStageCodegen (7) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item_sk,d_date,sumss,ss_item_sk] CometExchange [ss_item_sk] #11 @@ -103,7 +103,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store InputAdapter Window [ss_item_sk,d_date] WholeStageCodegen (8) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item_sk,d_date,sumss,ss_item_sk] ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt index 4b3a3c576e..84c04cfce2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject (47) : : +- Window (26) : : +- * Filter (25) : : +- Window (24) - : : +- * ColumnarToRow (23) + : : +- * CometColumnarToRow (23) : : +- CometSort (22) : : +- CometExchange (21) : : +- CometHashAggregate (20) @@ -35,7 +35,7 @@ TakeOrderedAndProject (47) : +- BroadcastExchange (36) : +- * Project (35) : +- Window (34) - : +- * ColumnarToRow (33) + : +- * CometColumnarToRow (33) : +- CometSort (32) : +- CometExchange (31) : +- CometHashAggregate (30) @@ -43,7 +43,7 @@ TakeOrderedAndProject (47) +- BroadcastExchange (44) +- * Project (43) +- Window (42) - +- * ColumnarToRow (41) + +- * CometColumnarToRow (41) +- CometSort (40) +- ReusedExchange (39) @@ -154,7 +154,7 @@ Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQU Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16] Arguments: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16], [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarToRow [codegen id : 1] Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16] (24) Window @@ -193,7 +193,7 @@ Arguments: hashpartitioning(i_category#19, i_brand#20, cc_name#21, 5), ENSURE_RE Input [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] Arguments: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15], [i_category#19 ASC NULLS FIRST, i_brand#20 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] -(33) ColumnarToRow [codegen id : 3] +(33) CometColumnarToRow [codegen id : 3] Input [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] (34) Window @@ -225,7 +225,7 @@ Output [6]: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sal Input [6]: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sales#15] Arguments: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sales#15], [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] -(41) ColumnarToRow [codegen id : 5] +(41) CometColumnarToRow [codegen id : 5] Input [6]: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sales#15] (42) Window @@ -258,7 +258,7 @@ Arguments: 100, [(sum_sales#15 - avg_monthly_sales#18) ASC NULLS FIRST, d_year#1 Subquery:1 Hosting operator id = 3 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (51) -+- * ColumnarToRow (50) ++- * CometColumnarToRow (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -274,7 +274,7 @@ ReadSchema: struct Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (51) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt index 3ea3165fc7..e9ee530494 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] CometExchange [i_category,i_brand,cc_name] #1 @@ -34,7 +34,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] @@ -51,7 +51,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] CometExchange [i_category,i_brand,cc_name] #8 @@ -64,7 +64,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt index 564222abff..6453facb05 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject (83) : +- * HashAggregate (66) : +- Union (65) : :- * HashAggregate (22) - : : +- * ColumnarToRow (21) + : : +- * CometColumnarToRow (21) : : +- CometExchange (20) : : +- CometHashAggregate (19) : : +- CometProject (18) @@ -31,7 +31,7 @@ TakeOrderedAndProject (83) : : +- CometFilter (15) : : +- CometScan parquet spark_catalog.default.store (14) : :- * HashAggregate (41) - : : +- * ColumnarToRow (40) + : : +- * CometColumnarToRow (40) : : +- CometExchange (39) : : +- CometHashAggregate (38) : : +- CometProject (37) @@ -50,7 +50,7 @@ TakeOrderedAndProject (83) : : +- CometFilter (34) : : +- CometScan parquet spark_catalog.default.catalog_page (33) : +- * HashAggregate (64) - : +- * ColumnarToRow (63) + : +- * CometColumnarToRow (63) : +- CometExchange (62) : +- CometHashAggregate (61) : +- CometProject (60) @@ -181,7 +181,7 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(21) ColumnarToRow [codegen id : 1] +(21) CometColumnarToRow [codegen id : 1] Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] (22) HashAggregate [codegen id : 1] @@ -272,7 +272,7 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#46)), partial_sum(Unscaled Input [5]: [cp_catalog_page_id#62, sum#63, sum#64, sum#65, sum#66] Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(40) ColumnarToRow [codegen id : 2] +(40) CometColumnarToRow [codegen id : 2] Input [5]: [cp_catalog_page_id#62, sum#63, sum#64, sum#65, sum#66] (41) HashAggregate [codegen id : 2] @@ -382,7 +382,7 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#83)), partial_sum(Unscaled Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(63) ColumnarToRow [codegen id : 3] +(63) CometColumnarToRow [codegen id : 3] Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] (64) HashAggregate [codegen id : 3] @@ -496,7 +496,7 @@ Arguments: 100, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (88) -+- * ColumnarToRow (87) ++- * CometColumnarToRow (87) +- CometProject (86) +- CometFilter (85) +- CometScan parquet spark_catalog.default.date_dim (84) @@ -517,7 +517,7 @@ Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#2 Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(87) ColumnarToRow [codegen id : 1] +(87) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] (88) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt index 751b3dc240..15b14cb16f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Union WholeStageCodegen (1) HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [s_store_id] #3 CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] @@ -32,7 +32,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] @@ -50,7 +50,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (2) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [cp_catalog_page_id] #7 CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] @@ -73,7 +73,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (3) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [web_site_id] #9 CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt index 54cbf8326b..77ad43a121 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (38) +* CometColumnarToRow (38) +- CometTakeOrderedAndProject (37) +- CometFilter (36) +- CometHashAggregate (35) @@ -215,14 +215,14 @@ Condition : (cnt#23 >= 10) Input [3]: [state#22, cnt#23, ca_state#2] Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#23 ASC NULLS FIRST,ca_state#2 ASC NULLS FIRST], output=[state#22,cnt#23]), [state#22, cnt#23], 100, [cnt#23 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#22, cnt#23] -(38) ColumnarToRow [codegen id : 1] +(38) CometColumnarToRow [codegen id : 1] Input [2]: [state#22, cnt#23] ===== Subqueries ===== Subquery:1 Hosting operator id = 8 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (43) -+- * ColumnarToRow (42) ++- * CometColumnarToRow (42) +- CometProject (41) +- CometFilter (40) +- CometScan parquet spark_catalog.default.date_dim (39) @@ -243,7 +243,7 @@ Condition : ((isnotnull(d_month_seq#10) AND (d_month_seq#10 = Subquery scalar-su Input [2]: [d_date_sk#9, d_month_seq#10] Arguments: [d_date_sk#9], [d_date_sk#9] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] (43) BroadcastExchange @@ -251,7 +251,7 @@ Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] Subquery:2 Hosting operator id = 40 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* ColumnarToRow (50) +* CometColumnarToRow (50) +- CometHashAggregate (49) +- CometExchange (48) +- CometHashAggregate (47) @@ -289,7 +289,7 @@ Input [1]: [d_month_seq#24] Keys [1]: [d_month_seq#24] Functions: [] -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [1]: [d_month_seq#24] Subquery:3 Hosting operator id = 14 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt index 580f668ea8..266060bf09 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [state,cnt,ca_state] CometFilter [state,cnt,ca_state] @@ -25,13 +25,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [d_month_seq] CometExchange [d_month_seq] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt index 4d8ac469c3..7252446de1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (173) +* CometColumnarToRow (173) +- CometSort (172) +- CometColumnarExchange (171) +- CometProject (170) @@ -938,14 +938,14 @@ Arguments: rangepartitioning(product_name#76 ASC NULLS FIRST, store_name#78 ASC Input [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] Arguments: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162], [product_name#76 ASC NULLS FIRST, store_name#78 ASC NULLS FIRST, cnt#162 ASC NULLS FIRST, s1#90 ASC NULLS FIRST, s1#163 ASC NULLS FIRST] -(173) ColumnarToRow [codegen id : 1] +(173) CometColumnarToRow [codegen id : 1] Input [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (177) -+- * ColumnarToRow (176) ++- * CometColumnarToRow (176) +- CometFilter (175) +- CometScan parquet spark_catalog.default.date_dim (174) @@ -961,7 +961,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(176) ColumnarToRow [codegen id : 1] +(176) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] (177) BroadcastExchange @@ -970,7 +970,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#104 IN dynamicpruning#105 BroadcastExchange (181) -+- * ColumnarToRow (180) ++- * CometColumnarToRow (180) +- CometFilter (179) +- CometScan parquet spark_catalog.default.date_dim (178) @@ -986,7 +986,7 @@ ReadSchema: struct Input [2]: [d_date_sk#117, d_year#118] Condition : ((isnotnull(d_year#118) AND (d_year#118 = 2000)) AND isnotnull(d_date_sk#117)) -(180) ColumnarToRow [codegen id : 1] +(180) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#117, d_year#118] (181) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt index f38a6afca6..f4df8e5826 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] CometColumnarExchange [product_name,store_name,cnt,s1,s1] #1 @@ -51,7 +51,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -158,7 +158,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #22 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt index 379144482c..10156503f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (68) +- * Filter (67) +- Window (66) - +- * ColumnarToRow (65) + +- * CometColumnarToRow (65) +- CometSort (64) +- CometExchange (63) +- CometUnion (62) @@ -366,7 +366,7 @@ Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, CometNativeS Input [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sumsales#54] Arguments: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sumsales#54], [i_category#17 ASC NULLS FIRST, sumsales#54 DESC NULLS LAST] -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sumsales#54] (66) Window @@ -385,7 +385,7 @@ Arguments: 100, [i_category#17 ASC NULLS FIRST, i_class#16 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (73) -+- * ColumnarToRow (72) ++- * CometColumnarToRow (72) +- CometProject (71) +- CometFilter (70) +- CometScan parquet spark_catalog.default.date_dim (69) @@ -406,7 +406,7 @@ Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_mont Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(72) ColumnarToRow [codegen id : 1] +(72) CometColumnarToRow [codegen id : 1] Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] (73) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt index 3216530ac2..9410250201 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ InputAdapter Window [sumsales,i_category] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] CometExchange [i_category] #1 @@ -23,7 +23,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt index 4a04020a03..ecdb8d50e9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject (57) : +- * HashAggregate (36) : +- * Project (35) : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * ColumnarToRow (9) + : :- * CometColumnarToRow (9) : : +- CometProject (8) : : +- CometBroadcastHashJoin (7) : : :- CometFilter (2) @@ -24,14 +24,14 @@ TakeOrderedAndProject (57) : : +- CometScan parquet spark_catalog.default.date_dim (3) : +- BroadcastExchange (33) : +- * BroadcastHashJoin LeftSemi BuildRight (32) - : :- * ColumnarToRow (12) + : :- * CometColumnarToRow (12) : : +- CometFilter (11) : : +- CometScan parquet spark_catalog.default.store (10) : +- BroadcastExchange (31) : +- * Project (30) : +- * Filter (29) : +- Window (28) - : +- * ColumnarToRow (27) + : +- * CometColumnarToRow (27) : +- CometSort (26) : +- CometHashAggregate (25) : +- CometExchange (24) @@ -98,7 +98,7 @@ Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] -(9) ColumnarToRow [codegen id : 4] +(9) CometColumnarToRow [codegen id : 4] Input [2]: [ss_store_sk#1, ss_net_profit#2] (10) CometScan parquet spark_catalog.default.store @@ -112,7 +112,7 @@ ReadSchema: struct Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) ColumnarToRow [codegen id : 3] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) CometScan parquet spark_catalog.default.store_sales @@ -181,7 +181,7 @@ Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] Input [3]: [s_state#15, _w0#18, s_state#15] Arguments: [s_state#15, _w0#18, s_state#15], [s_state#15 ASC NULLS FIRST, _w0#18 DESC NULLS LAST] -(27) ColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [3]: [s_state#15, _w0#18, s_state#15] (28) Window @@ -338,7 +338,7 @@ Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (62) -+- * ColumnarToRow (61) ++- * CometColumnarToRow (61) +- CometProject (60) +- CometFilter (59) +- CometScan parquet spark_catalog.default.date_dim (58) @@ -359,7 +359,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(61) ColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (62) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt index a1d9ab5183..8d0ac308ad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt @@ -23,7 +23,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] Project [ss_net_profit,s_county,s_state] BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] @@ -32,7 +32,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -45,7 +45,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #6 WholeStageCodegen (3) BroadcastHashJoin [s_state,s_state] - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] @@ -57,7 +57,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count InputAdapter Window [_w0,s_state] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [s_state,_w0] CometHashAggregate [s_state,_w0,sum,sum(UnscaledValue(ss_net_profit))] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt index 45f7e2e66d..4b97b311be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (64) +* CometColumnarToRow (64) +- CometTakeOrderedAndProject (63) +- CometHashAggregate (62) +- CometExchange (61) @@ -364,14 +364,14 @@ Functions [1]: [count(1)] Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, [total_cnt#37 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] -(64) ColumnarToRow [codegen id : 1] +(64) CometColumnarToRow [codegen id : 1] Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (69) -+- * ColumnarToRow (68) ++- * CometColumnarToRow (68) +- CometProject (67) +- CometFilter (66) +- CometScan parquet spark_catalog.default.date_dim (65) @@ -392,7 +392,7 @@ Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_dat Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(68) ColumnarToRow [codegen id : 1] +(68) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (69) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt index ccd1b66fa2..fe26a3f5cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count,count(1)] @@ -32,7 +32,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] CometFilter [d_date_sk,d_date,d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt index 12d5134a06..b1b1e31edb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (68) +* CometColumnarToRow (68) +- CometTakeOrderedAndProject (67) +- CometProject (66) +- CometBroadcastHashJoin (65) @@ -388,14 +388,14 @@ Arguments: [customer_id#25, customer_first_name#26, customer_last_name#27], [cus Input [3]: [customer_id#25, customer_first_name#26, customer_last_name#27] Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_first_name#26 ASC NULLS FIRST,customer_id#25 ASC NULLS FIRST,customer_last_name#27 ASC NULLS FIRST], output=[customer_id#25,customer_first_name#26,customer_last_name#27]), [customer_id#25, customer_first_name#26, customer_last_name#27], 100, [customer_first_name#26 ASC NULLS FIRST, customer_id#25 ASC NULLS FIRST, customer_last_name#27 ASC NULLS FIRST], [customer_id#25, customer_first_name#26, customer_last_name#27] -(68) ColumnarToRow [codegen id : 1] +(68) CometColumnarToRow [codegen id : 1] Input [3]: [customer_id#25, customer_first_name#26, customer_last_name#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (72) -+- * ColumnarToRow (71) ++- * CometColumnarToRow (71) +- CometFilter (70) +- CometScan parquet spark_catalog.default.date_dim (69) @@ -411,7 +411,7 @@ ReadSchema: struct Input [2]: [d_date_sk#9, d_year#10] Condition : (((isnotnull(d_year#10) AND (d_year#10 = 2001)) AND d_year#10 IN (2001,2002)) AND isnotnull(d_date_sk#9)) -(71) ColumnarToRow [codegen id : 1] +(71) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] (72) BroadcastExchange @@ -420,7 +420,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 BroadcastExchange (76) -+- * ColumnarToRow (75) ++- * CometColumnarToRow (75) +- CometFilter (74) +- CometScan parquet spark_catalog.default.date_dim (73) @@ -436,7 +436,7 @@ ReadSchema: struct Input [2]: [d_date_sk#22, d_year#23] Condition : (((isnotnull(d_year#23) AND (d_year#23 = 2002)) AND d_year#23 IN (2001,2002)) AND isnotnull(d_date_sk#22)) -(75) ColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#22, d_year#23] (76) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt index 631a82f1e3..9bde9ac863 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] CometProject [customer_id,customer_first_name,customer_last_name] @@ -23,7 +23,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -46,7 +46,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt index 6789a22468..b8bba79fa5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (124) +* CometColumnarToRow (124) +- CometTakeOrderedAndProject (123) +- CometProject (122) +- CometSortMergeJoin (121) @@ -676,14 +676,14 @@ Arguments: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#1 Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#124 ASC NULLS FIRST,sales_amt_diff#125 ASC NULLS FIRST], output=[prev_year#120,year#121,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#122,curr_yr_cnt#123,sales_cnt_diff#124,sales_amt_diff#125]), [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], 100, [sales_cnt_diff#124 ASC NULLS FIRST, sales_amt_diff#125 ASC NULLS FIRST], [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] -(124) ColumnarToRow [codegen id : 1] +(124) CometColumnarToRow [codegen id : 1] Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (128) -+- * ColumnarToRow (127) ++- * CometColumnarToRow (127) +- CometFilter (126) +- CometScan parquet spark_catalog.default.date_dim (125) @@ -699,7 +699,7 @@ ReadSchema: struct Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(127) ColumnarToRow [codegen id : 1] +(127) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] (128) BroadcastExchange @@ -712,7 +712,7 @@ Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN d Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 BroadcastExchange (132) -+- * ColumnarToRow (131) ++- * CometColumnarToRow (131) +- CometFilter (130) +- CometScan parquet spark_catalog.default.date_dim (129) @@ -728,7 +728,7 @@ ReadSchema: struct Input [2]: [d_date_sk#77, d_year#78] Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(131) ColumnarToRow [codegen id : 1] +(131) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#77, d_year#78] (132) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt index 9e340130a5..5a8af58abc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] @@ -27,7 +27,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -102,7 +102,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt index dcbd98dbb8..2d1cab2831 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject (97) : +- Exchange (81) : +- * HashAggregate (80) : +- Union (79) - : :- * ColumnarToRow (31) + : :- * CometColumnarToRow (31) : : +- CometProject (30) : : +- CometBroadcastHashJoin (29) : : :- CometHashAggregate (16) @@ -42,7 +42,7 @@ TakeOrderedAndProject (97) : :- * Project (50) : : +- * BroadcastNestedLoopJoin Inner BuildLeft (49) : : :- BroadcastExchange (40) - : : : +- * ColumnarToRow (39) + : : : +- * CometColumnarToRow (39) : : : +- CometHashAggregate (38) : : : +- CometExchange (37) : : : +- CometHashAggregate (36) @@ -50,7 +50,7 @@ TakeOrderedAndProject (97) : : : +- CometBroadcastHashJoin (34) : : : :- CometScan parquet spark_catalog.default.catalog_sales (32) : : : +- ReusedExchange (33) - : : +- * ColumnarToRow (48) + : : +- * CometColumnarToRow (48) : : +- CometHashAggregate (47) : : +- CometExchange (46) : : +- CometHashAggregate (45) @@ -58,7 +58,7 @@ TakeOrderedAndProject (97) : : +- CometBroadcastHashJoin (43) : : :- CometScan parquet spark_catalog.default.catalog_returns (41) : : +- ReusedExchange (42) - : +- * ColumnarToRow (78) + : +- * CometColumnarToRow (78) : +- CometProject (77) : +- CometBroadcastHashJoin (76) : :- CometHashAggregate (63) @@ -239,7 +239,7 @@ Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] Arguments: [channel#24, id#25, sales#22, returns#26, profit#27], [store channel AS channel#24, s_store_sk#8 AS id#25, sales#22, coalesce(returns#20, 0.00) AS returns#26, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#27] -(31) ColumnarToRow [codegen id : 1] +(31) CometColumnarToRow [codegen id : 1] Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] (32) CometScan parquet spark_catalog.default.catalog_sales @@ -275,7 +275,7 @@ Input [3]: [cs_call_center_sk#28, sum#34, sum#35] Keys [1]: [cs_call_center_sk#28] Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#29)), sum(UnscaledValue(cs_net_profit#30))] -(39) ColumnarToRow [codegen id : 2] +(39) CometColumnarToRow [codegen id : 2] Input [3]: [cs_call_center_sk#28, sales#36, profit#37] (40) BroadcastExchange @@ -315,7 +315,7 @@ Input [2]: [sum#43, sum#44] Keys: [] Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] -(48) ColumnarToRow +(48) CometColumnarToRow Input [2]: [returns#45, profit_loss#46] (49) BroadcastNestedLoopJoin [codegen id : 3] @@ -451,7 +451,7 @@ Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] Arguments: [channel#72, id#73, sales#70, returns#74, profit#75], [web channel AS channel#72, wp_web_page_sk#56 AS id#73, sales#70, coalesce(returns#68, 0.00) AS returns#74, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#75] -(78) ColumnarToRow [codegen id : 4] +(78) CometColumnarToRow [codegen id : 4] Input [5]: [channel#72, id#73, sales#70, returns#74, profit#75] (79) Union @@ -558,7 +558,7 @@ Arguments: 100, [channel#24 ASC NULLS FIRST, id#25 ASC NULLS FIRST], [channel#24 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (102) -+- * ColumnarToRow (101) ++- * CometColumnarToRow (101) +- CometProject (100) +- CometFilter (99) +- CometScan parquet spark_catalog.default.date_dim (98) @@ -579,7 +579,7 @@ Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 < Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(101) ColumnarToRow [codegen id : 1] +(101) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (102) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt index 01cd2f085f..34656493f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter Union WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [s_store_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] CometBroadcastHashJoin [s_store_sk,sales,profit,s_store_sk,returns,profit_loss] @@ -32,7 +32,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] @@ -63,7 +63,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter BroadcastExchange #9 WholeStageCodegen (2) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [cs_call_center_sk,sales,profit,sum,sum,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] CometExchange [cs_call_center_sk] #10 @@ -73,7 +73,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [returns,profit_loss,sum,sum,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] CometExchange #11 @@ -84,7 +84,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 WholeStageCodegen (4) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [wp_web_page_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] CometBroadcastHashJoin [wp_web_page_sk,sales,profit,wp_web_page_sk,returns,profit_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt index 1637692970..9a697cc844 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (67) +- * Project (66) - +- * ColumnarToRow (65) + +- * CometColumnarToRow (65) +- CometSortMergeJoin (64) :- CometProject (43) : +- CometSortMergeJoin (42) @@ -360,7 +360,7 @@ Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss Right output [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59], Inner -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] (66) Project [codegen id : 1] @@ -375,7 +375,7 @@ Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (71) -+- * ColumnarToRow (70) ++- * CometColumnarToRow (70) +- CometFilter (69) +- CometScan parquet spark_catalog.default.date_dim (68) @@ -391,7 +391,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(70) ColumnarToRow [codegen id : 1] +(70) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] (71) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt index 78d5fe1d7b..58ca9f30b8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] WholeStageCodegen (1) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] - ColumnarToRow + CometColumnarToRow InputAdapter CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] @@ -22,7 +22,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt index 442335b6dc..52f0110cb9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject (115) : +- * HashAggregate (98) : +- Union (97) : :- * HashAggregate (38) - : : +- * ColumnarToRow (37) + : : +- * CometColumnarToRow (37) : : +- CometExchange (36) : : +- CometHashAggregate (35) : : +- CometProject (34) @@ -47,7 +47,7 @@ TakeOrderedAndProject (115) : : +- CometFilter (30) : : +- CometScan parquet spark_catalog.default.promotion (29) : :- * HashAggregate (67) - : : +- * ColumnarToRow (66) + : : +- * CometColumnarToRow (66) : : +- CometExchange (65) : : +- CometHashAggregate (64) : : +- CometProject (63) @@ -76,7 +76,7 @@ TakeOrderedAndProject (115) : : : +- ReusedExchange (58) : : +- ReusedExchange (61) : +- * HashAggregate (96) - : +- * ColumnarToRow (95) + : +- * CometColumnarToRow (95) : +- CometExchange (94) : +- CometHashAggregate (93) : +- CometProject (92) @@ -285,7 +285,7 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(co Input [6]: [s_store_id#17, sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] Arguments: hashpartitioning(s_store_id#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(37) ColumnarToRow [codegen id : 1] +(37) CometColumnarToRow [codegen id : 1] Input [6]: [s_store_id#17, sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] (38) HashAggregate [codegen id : 1] @@ -416,7 +416,7 @@ Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#39)), partial_sum(c Input [6]: [cp_catalog_page_id#50, sum#53, sum#54, isEmpty#55, sum#56, isEmpty#57] Arguments: hashpartitioning(cp_catalog_page_id#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(66) ColumnarToRow [codegen id : 2] +(66) CometColumnarToRow [codegen id : 2] Input [6]: [cp_catalog_page_id#50, sum#53, sum#54, isEmpty#55, sum#56, isEmpty#57] (67) HashAggregate [codegen id : 2] @@ -547,7 +547,7 @@ Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(c Input [6]: [web_site_id#81, sum#84, sum#85, isEmpty#86, sum#87, isEmpty#88] Arguments: hashpartitioning(web_site_id#81, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] -(95) ColumnarToRow [codegen id : 3] +(95) CometColumnarToRow [codegen id : 3] Input [6]: [web_site_id#81, sum#84, sum#85, isEmpty#86, sum#87, isEmpty#88] (96) HashAggregate [codegen id : 3] @@ -661,7 +661,7 @@ Arguments: 100, [channel#30 ASC NULLS FIRST, id#31 ASC NULLS FIRST], [channel#30 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (120) -+- * ColumnarToRow (119) ++- * CometColumnarToRow (119) +- CometProject (118) +- CometFilter (117) +- CometScan parquet spark_catalog.default.date_dim (116) @@ -682,7 +682,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(119) ColumnarToRow [codegen id : 1] +(119) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] (120) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt index aa61bbd579..e04a9ec1ad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Union WholeStageCodegen (1) HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [s_store_id] #3 CometHashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] @@ -38,7 +38,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] @@ -65,7 +65,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (2) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [cp_catalog_page_id] #11 CometHashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] @@ -97,7 +97,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [p_promo_sk] #10 WholeStageCodegen (3) HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow + CometColumnarToRow InputAdapter CometExchange [web_site_id] #15 CometHashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt index 32e6981799..1817811aa1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (36) +- * Project (35) +- Window (34) - +- * ColumnarToRow (33) + +- * CometColumnarToRow (33) +- CometSort (32) +- CometExchange (31) +- CometHashAggregate (30) @@ -186,7 +186,7 @@ Arguments: hashpartitioning(lochierarchy#21, _w0#33, 5), ENSURE_REQUIREMENTS, Co Input [5]: [total_sum#18, i_category#9, i_class#8, lochierarchy#21, _w0#33] Arguments: [total_sum#18, i_category#9, i_class#8, lochierarchy#21, _w0#33], [lochierarchy#21 ASC NULLS FIRST, _w0#33 ASC NULLS FIRST, total_sum#18 DESC NULLS LAST] -(33) ColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [codegen id : 1] Input [5]: [total_sum#18, i_category#9, i_class#8, lochierarchy#21, _w0#33] (34) Window @@ -205,7 +205,7 @@ Arguments: 100, [lochierarchy#21 DESC NULLS LAST, CASE WHEN (lochierarchy#21 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (41) -+- * ColumnarToRow (40) ++- * CometColumnarToRow (40) +- CometProject (39) +- CometFilter (38) +- CometScan parquet spark_catalog.default.date_dim (37) @@ -226,7 +226,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(40) ColumnarToRow [codegen id : 1] +(40) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (41) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt index ae161e5ae2..d2ed714b7e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl InputAdapter Window [total_sum,lochierarchy,_w0] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [total_sum,i_category,i_class,lochierarchy,_w0] CometExchange [lochierarchy,_w0] #1 @@ -24,7 +24,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt index 4d870a8b80..7bcf62d617 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -* ColumnarToRow (24) +* CometColumnarToRow (24) +- CometSort (23) +- CometColumnarExchange (22) +- * Project (21) +- Window (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometSort (18) +- CometExchange (17) +- CometHashAggregate (16) @@ -111,7 +111,7 @@ Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometNativeShuff Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15], [i_class#9 ASC NULLS FIRST] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] (20) Window @@ -130,14 +130,14 @@ Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17], [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#17 ASC NULLS FIRST] -(24) ColumnarToRow [codegen id : 3] +(24) CometColumnarToRow [codegen id : 3] Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * ColumnarToRow (28) ++- * CometColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan parquet spark_catalog.default.date_dim (25) @@ -158,7 +158,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt index 2b7813a707..6509498640 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 @@ -8,7 +8,7 @@ WholeStageCodegen (3) InputAdapter Window [_w0,i_class] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] CometExchange [i_class] #2 @@ -24,7 +24,7 @@ WholeStageCodegen (3) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index 36d3706506..afdf8601de 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -28,8 +28,8 @@ import scala.util.Random import org.apache.hadoop.fs.Path import org.apache.spark.sql.{CometTestBase, DataFrame, Row} import org.apache.spark.sql.catalyst.optimizer.SimplifyExtractValueOps -import org.apache.spark.sql.comet.CometProjectExec -import org.apache.spark.sql.execution.{ColumnarToRowExec, InputAdapter, ProjectExec, WholeStageCodegenExec} +import org.apache.spark.sql.comet.{CometColumnarToRowExec, CometProjectExec} +import org.apache.spark.sql.execution.{InputAdapter, ProjectExec, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -749,7 +749,7 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { val project = cometPlan .asInstanceOf[WholeStageCodegenExec] .child - .asInstanceOf[ColumnarToRowExec] + .asInstanceOf[CometColumnarToRowExec] .child .asInstanceOf[InputAdapter] .child diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index 90c3221e5d..b6636d9c12 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -66,8 +66,10 @@ class CometExecSuite extends CometTestBase { test("TopK operator should return correct results on dictionary column with nulls") { withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "") { withTable("test_data") { + val data = (0 to 8000) + .flatMap(_ => Seq((1, null, "A"), (2, "BBB", "B"), (3, "BBB", "B"), (4, "BBB", "B"))) val tableDF = spark.sparkContext - .parallelize(Seq((1, null, "A"), (2, "BBB", "B"), (3, "BBB", "B"), (4, "BBB", "B")), 3) + .parallelize(data, 3) .toDF("c1", "c2", "c3") tableDF .coalesce(1) diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala index 213ec7efee..cba9075d52 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala @@ -36,7 +36,7 @@ import org.apache.parquet.hadoop.example.ExampleParquetWriter import org.apache.parquet.schema.{MessageType, MessageTypeParser} import org.apache.spark._ import org.apache.spark.internal.config.{MEMORY_OFFHEAP_ENABLED, MEMORY_OFFHEAP_SIZE, SHUFFLE_MANAGER} -import org.apache.spark.sql.comet.{CometBatchScanExec, CometBroadcastExchangeExec, CometExec, CometScanExec, CometScanWrapper, CometSinkPlaceHolder, CometSparkToColumnarExec} +import org.apache.spark.sql.comet.{CometBatchScanExec, CometBroadcastExchangeExec, CometColumnarToRowExec, CometExec, CometScanExec, CometScanWrapper, CometSinkPlaceHolder, CometSparkToColumnarExec} import org.apache.spark.sql.comet.execution.shuffle.{CometColumnarShuffle, CometNativeShuffle, CometShuffleExchangeExec} import org.apache.spark.sql.execution.{ColumnarToRowExec, ExtendedMode, InputAdapter, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper @@ -174,6 +174,7 @@ abstract class CometTestBase wrapped.foreach { case _: CometScanExec | _: CometBatchScanExec => case _: CometSinkPlaceHolder | _: CometScanWrapper => + case _: CometColumnarToRowExec => case _: CometSparkToColumnarExec => case _: CometExec | _: CometShuffleExchangeExec => case _: CometBroadcastExchangeExec => From 4b56c523f8de251d846d15708c6c3f117513ab03 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Jan 2025 17:01:37 -0700 Subject: [PATCH 52/83] remove unreachable code (#1213) --- common/src/main/scala/org/apache/comet/vector/NativeUtil.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala b/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala index 72472a5401..fba4e29e5e 100644 --- a/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala +++ b/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala @@ -163,8 +163,6 @@ class NativeUtil { case numRows => val cometVectors = importVector(arrays, schemas) Some(new ColumnarBatch(cometVectors.toArray, numRows.toInt)) - case flag => - throw new IllegalStateException(s"Invalid native flag: $flag") } } From 5f1e99830caec96ce5889656cb8fca68a6d6d5e0 Mon Sep 17 00:00:00 2001 From: KAZUYUKI TANIMURA Date: Fri, 3 Jan 2025 16:29:49 -0800 Subject: [PATCH 53/83] test: Enable Comet by default except some tests in SparkSessionExtensionSuite (#1201) ## Which issue does this PR close? Part of https://github.com/apache/datafusion-comet/issues/1197 ## Rationale for this change Since `loadCometExtension` in the diffs were not using `isCometEnabled`, `SparkSessionExtensionSuite` was not using Comet. Once enabled, some test failures discovered ## What changes are included in this PR? `loadCometExtension` now uses `isCometEnabled` that enables Comet by default Temporary ignore the failing tests in SparkSessionExtensionSuite ## How are these changes tested? existing tests --- dev/diffs/3.4.3.diff | 66 +++++++++++++++++++++++++------ dev/diffs/3.5.1.diff | 74 +++++++++++++++++++++++++++-------- dev/diffs/4.0.0-preview1.diff | 66 +++++++++++++++++++++++++------ 3 files changed, 163 insertions(+), 43 deletions(-) diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 8be1cdcaa7..b78a24a4c8 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -53,7 +53,7 @@ index b386d135da1..46449e3f3f1 100644 + +# DataFusion Comet 0.5.0 Changelog + +This release consists of 69 commits from 15 contributors. See credits at the end of this changelog for more information. + +**Fixed bugs:** + +- fix: Unsigned type related bugs [#1095](https://github.com/apache/datafusion-comet/pull/1095) (kazuyukitanimura) +- fix: Use RDD partition index [#1112](https://github.com/apache/datafusion-comet/pull/1112) (viirya) +- fix: Various metrics bug fixes and improvements [#1111](https://github.com/apache/datafusion-comet/pull/1111) (andygrove) +- fix: Don't create CometScanExec for subclasses of ParquetFileFormat [#1129](https://github.com/apache/datafusion-comet/pull/1129) (Kimahriman) +- fix: Fix metrics regressions [#1132](https://github.com/apache/datafusion-comet/pull/1132) (andygrove) +- fix: Enable scenarios accidentally commented out in CometExecBenchmark [#1151](https://github.com/apache/datafusion-comet/pull/1151) (mbutrovich) +- fix: Spark 4.0-preview1 SPARK-47120 [#1156](https://github.com/apache/datafusion-comet/pull/1156) (kazuyukitanimura) +- fix: Document enabling comet explain plan usage in Spark (4.0) [#1176](https://github.com/apache/datafusion-comet/pull/1176) (parthchandra) +- fix: stddev_pop should not directly return 0.0 when count is 1.0 [#1184](https://github.com/apache/datafusion-comet/pull/1184) (viirya) +- fix: fix missing explanation for then branch in case when [#1200](https://github.com/apache/datafusion-comet/pull/1200) (rluvaton) +- fix: Fall back to Spark for unsupported partition or sort expressions in window aggregates [#1253](https://github.com/apache/datafusion-comet/pull/1253) (andygrove) +- fix: Fall back to Spark for distinct aggregates [#1262](https://github.com/apache/datafusion-comet/pull/1262) (andygrove) +- fix: disable initCap by default [#1276](https://github.com/apache/datafusion-comet/pull/1276) (kazuyukitanimura) + +**Performance related:** + +- perf: Stop passing Java config map into native createPlan [#1101](https://github.com/apache/datafusion-comet/pull/1101) (andygrove) +- feat: Make native shuffle compression configurable and respect `spark.shuffle.compress` [#1185](https://github.com/apache/datafusion-comet/pull/1185) (andygrove) +- perf: Improve query planning to more reliably fall back to columnar shuffle when native shuffle is not supported [#1209](https://github.com/apache/datafusion-comet/pull/1209) (andygrove) +- feat: Move shuffle block decompression and decoding to native code and add LZ4 & Snappy support [#1192](https://github.com/apache/datafusion-comet/pull/1192) (andygrove) +- feat: Implement custom RecordBatch serde for shuffle for improved performance [#1190](https://github.com/apache/datafusion-comet/pull/1190) (andygrove) + +**Implemented enhancements:** + +- feat: support array_insert [#1073](https://github.com/apache/datafusion-comet/pull/1073) (SemyonSinchenko) +- feat: enable decimal to decimal cast of different precision and scale [#1086](https://github.com/apache/datafusion-comet/pull/1086) (himadripal) +- feat: Improve ScanExec native metrics [#1133](https://github.com/apache/datafusion-comet/pull/1133) (andygrove) +- feat: Add Spark-compatible implementation of SchemaAdapterFactory [#1169](https://github.com/apache/datafusion-comet/pull/1169) (andygrove) +- feat: Improve shuffle metrics (second attempt) [#1175](https://github.com/apache/datafusion-comet/pull/1175) (andygrove) +- feat: Add a `spark.comet.exec.memoryPool` configuration for experimenting with various datafusion memory pool setups. [#1021](https://github.com/apache/datafusion-comet/pull/1021) (Kontinuation) +- feat: Reenable tests for filtered SMJ anti join [#1211](https://github.com/apache/datafusion-comet/pull/1211) (comphead) +- feat: add support for array_remove expression [#1179](https://github.com/apache/datafusion-comet/pull/1179) (jatin510) + +**Documentation updates:** + +- docs: Update documentation for 0.4.0 release [#1096](https://github.com/apache/datafusion-comet/pull/1096) (andygrove) +- docs: Fix readme typo FGPA -> FPGA [#1117](https://github.com/apache/datafusion-comet/pull/1117) (gstvg) +- docs: Add more technical detail and new diagram to Comet plugin overview [#1119](https://github.com/apache/datafusion-comet/pull/1119) (andygrove) +- docs: Add some documentation explaining how shuffle works [#1148](https://github.com/apache/datafusion-comet/pull/1148) (andygrove) +- docs: Update TPC-H benchmark results [#1257](https://github.com/apache/datafusion-comet/pull/1257) (andygrove) + +**Other:** + +- chore: Add changelog for 0.4.0 [#1089](https://github.com/apache/datafusion-comet/pull/1089) (andygrove) +- chore: Prepare for 0.5.0 development [#1090](https://github.com/apache/datafusion-comet/pull/1090) (andygrove) +- build: Skip installation of spark-integration and fuzz testing modules [#1091](https://github.com/apache/datafusion-comet/pull/1091) (parthchandra) +- minor: Add hint for finding the GPG key to use when publishing to maven [#1093](https://github.com/apache/datafusion-comet/pull/1093) (andygrove) +- chore: Include first ScanExec batch in metrics [#1105](https://github.com/apache/datafusion-comet/pull/1105) (andygrove) +- chore: Improve CometScan metrics [#1100](https://github.com/apache/datafusion-comet/pull/1100) (andygrove) +- chore: Add custom metric for native shuffle fetching batches from JVM [#1108](https://github.com/apache/datafusion-comet/pull/1108) (andygrove) +- chore: Remove unused StringView struct [#1143](https://github.com/apache/datafusion-comet/pull/1143) (andygrove) +- test: enable more Spark 4.0 tests [#1145](https://github.com/apache/datafusion-comet/pull/1145) (kazuyukitanimura) +- chore: Refactor cast to use SparkCastOptions param [#1146](https://github.com/apache/datafusion-comet/pull/1146) (andygrove) +- chore: Move more expressions from core crate to spark-expr crate [#1152](https://github.com/apache/datafusion-comet/pull/1152) (andygrove) +- chore: Remove dead code [#1155](https://github.com/apache/datafusion-comet/pull/1155) (andygrove) +- chore: Move string kernels and expressions to spark-expr crate [#1164](https://github.com/apache/datafusion-comet/pull/1164) (andygrove) +- chore: Move remaining expressions to spark-expr crate + some minor refactoring [#1165](https://github.com/apache/datafusion-comet/pull/1165) (andygrove) +- chore: Add ignored tests for reading complex types from Parquet [#1167](https://github.com/apache/datafusion-comet/pull/1167) (andygrove) +- test: enabling Spark tests with offHeap requirement [#1177](https://github.com/apache/datafusion-comet/pull/1177) (kazuyukitanimura) +- minor: move shuffle classes from common to spark [#1193](https://github.com/apache/datafusion-comet/pull/1193) (andygrove) +- minor: refactor to move decodeBatches to broadcast exchange code as private function [#1195](https://github.com/apache/datafusion-comet/pull/1195) (andygrove) +- minor: refactor prepare_output so that it does not require an ExecutionContext [#1194](https://github.com/apache/datafusion-comet/pull/1194) (andygrove) +- minor: remove unused source files [#1202](https://github.com/apache/datafusion-comet/pull/1202) (andygrove) +- chore: Upgrade to DataFusion 44.0.0-rc2 [#1154](https://github.com/apache/datafusion-comet/pull/1154) (andygrove) +- chore: Add safety check to CometBuffer [#1050](https://github.com/apache/datafusion-comet/pull/1050) (viirya) +- chore: Remove unreachable code [#1213](https://github.com/apache/datafusion-comet/pull/1213) (andygrove) +- test: Enable Comet by default except some tests in SparkSessionExtensionSuite [#1201](https://github.com/apache/datafusion-comet/pull/1201) (kazuyukitanimura) +- chore: extract `struct` expressions to folders based on spark grouping [#1216](https://github.com/apache/datafusion-comet/pull/1216) (rluvaton) +- chore: extract static invoke expressions to folders based on spark grouping [#1217](https://github.com/apache/datafusion-comet/pull/1217) (rluvaton) +- chore: Follow-on PR to fully enable onheap memory usage [#1210](https://github.com/apache/datafusion-comet/pull/1210) (andygrove) +- chore: extract agg_funcs expressions to folders based on spark grouping [#1224](https://github.com/apache/datafusion-comet/pull/1224) (rluvaton) +- chore: extract datetime_funcs expressions to folders based on spark grouping [#1222](https://github.com/apache/datafusion-comet/pull/1222) (rluvaton) +- chore: Upgrade to DataFusion 44.0.0 from 44.0.0 RC2 [#1232](https://github.com/apache/datafusion-comet/pull/1232) (rluvaton) +- chore: extract strings file to `strings_func` like in spark grouping [#1215](https://github.com/apache/datafusion-comet/pull/1215) (rluvaton) +- chore: extract predicate_functions expressions to folders based on spark grouping [#1218](https://github.com/apache/datafusion-comet/pull/1218) (rluvaton) +- build(deps): bump protobuf version to 3.21.12 [#1234](https://github.com/apache/datafusion-comet/pull/1234) (wForget) +- chore: extract json_funcs expressions to folders based on spark grouping [#1220](https://github.com/apache/datafusion-comet/pull/1220) (rluvaton) +- test: Enable shuffle by default in Spark tests [#1240](https://github.com/apache/datafusion-comet/pull/1240) (kazuyukitanimura) +- chore: extract hash_funcs expressions to folders based on spark grouping [#1221](https://github.com/apache/datafusion-comet/pull/1221) (rluvaton) +- build: Fix test failure caused by merging conflicting PRs [#1259](https://github.com/apache/datafusion-comet/pull/1259) (andygrove) + +## Credits + +Thank you to everyone who contributed to this release. Here is a breakdown of commits (PRs merged) per contributor. + +``` + 37 Andy Grove + 10 Raz Luvaton + 7 KAZUYUKI TANIMURA + 3 Liang-Chi Hsieh + 2 Parth Chandra + 1 Adam Binford + 1 Dharan Aditya + 1 Himadri Pal + 1 Jagdish Parihar + 1 Kristin Cowalcijk + 1 Matt Butrovich + 1 Oleks V + 1 Sem + 1 Zhen Wang + 1 gstvg +``` + +Thank you also to everyone who contributed in other ways such as filing issues, reviewing PRs, and providing feedback on this release. From 08d892a21e853d0daa7afcd68cd97042dfb54bfa Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 14 Jan 2025 07:47:15 -0700 Subject: [PATCH 76/83] update TPC-DS results for 0.5.0 (#1277) --- .../0.5.0/tpcds_allqueries.png | Bin 0 -> 27773 bytes .../0.5.0/tpcds_queries_compare.png | Bin 0 -> 42959 bytes .../0.5.0/tpcds_queries_speedup_abs.png | Bin 0 -> 66470 bytes .../0.5.0/tpcds_queries_speedup_rel.png | Bin 0 -> 91701 bytes .../benchmark-results/0.5.0/comet-tpcds.json | 744 ++++++++++++++++++ .../benchmark-results/0.5.0/spark-tpcds.json | 722 +++++++++++++++++ .../benchmark-results/tpc-ds.md | 12 +- 7 files changed, 1472 insertions(+), 6 deletions(-) create mode 100644 docs/source/_static/images/benchmark-results/0.5.0/tpcds_allqueries.png create mode 100644 docs/source/_static/images/benchmark-results/0.5.0/tpcds_queries_compare.png create mode 100644 docs/source/_static/images/benchmark-results/0.5.0/tpcds_queries_speedup_abs.png create mode 100644 docs/source/_static/images/benchmark-results/0.5.0/tpcds_queries_speedup_rel.png create mode 100644 docs/source/contributor-guide/benchmark-results/0.5.0/comet-tpcds.json create mode 100644 docs/source/contributor-guide/benchmark-results/0.5.0/spark-tpcds.json diff --git a/docs/source/_static/images/benchmark-results/0.5.0/tpcds_allqueries.png b/docs/source/_static/images/benchmark-results/0.5.0/tpcds_allqueries.png new file mode 100644 index 0000000000000000000000000000000000000000..3fd6440953866f0808a04939d19a659b931c14e6 GIT binary patch literal 27773 zcmd?SX*`x~+dg_4GbNOvL8eINN+p>x4-JwcmAOcS3<*hPo=T;eLW-2B5ScQRCL%+I zjEPVvp@g*^?mJKK|M$M@UF*a8wE8{w-Hpq2o#%1x$G&g-wr%_Igc@$sna{F>g`%kW zdb*lM6h-S#Q8XP)jQGyFn%0l_YooiCsrydngYL(yjvk=4TDiM8I=ed_vKI6_aMbOP z^O3bO8)eo=3EI26ySORJ$~w)xK*sr~ovhF{x+c5{vy1K?H;P(dMgE~lR=sK<5y7A@1F;iom7RF(p&$=eJlf~@~OMhI49iym5=+Omp!`)kQ1}V?*ufHGcB(CugQz+}(X!tP1MdU*J^& zeiNON<4K0%atB^sH1Yd={aaf3h>G`UAE(OX;M(_1O`GElE$fnZJocMX#M|tCU?y6B zea!)uMJ$2Z)|IbbojG%c4sS6p^3nFd)il03s}_YBeZYsw7Vcm@lAoWSyf1G}zQ;h& zTDz)64wEk~th}OnQf0}8BW$d!tbB_X$9whJBU zd1$xYvH20RM9jUnHpUX%4fpq)c^0`e(XlB@ZtYr)AN}<;=@zQXl)Xn|Zr)wFJx)+o zb|^-m*s*ENXn#FxjHF4Bdc63E!CGZy{xkCyX#f1yV`Xi98rR=B`uUw!!-pKFrTvZR z3q(XjjMMfl77`Mwtge>L`r7o+j+2j%Y30h5xVooxb&I}!{mQq=Qz~u$lRLY|fA*Sh z-@bsIoxQImS60Tnz_fpAdaT8{=CP}d{NaZCb!X$p8!N(jS08w}XkW44lvD7x-Gzg8 zJ(K;x0)B=Dt>(6$USBoMcHl1h{mtP1{wF*){eGo1n`_k9+aHUy} zW9WmuxuvbG`|hb)morK2&tJ4Z-~Ci^vFhmPsLSVC(ZTnbbh}b_Yh4iAsz23wL&23c zoJU&S)^>%FkwsKFAzxBk*NLHXZ%R!4?Jf@ z&%u!Q?@c2XuVrj#Xqe~>7I>R=z~F|W2TN5|mC*+0z}%{uU^u6gS7-J8Qbmn}-0o0m*aPrJCfZuaw2sk*6pcHQ9y%crLqynautcK7f| zdag9t+S}XP-Cv)|#LB9nsY#E=8N$LXaq)5W_}33*_HEU=ol*-(i^yzT)J{a{8<#gq=ZSDl)JlofO`D7BSAqy z30tqP?>ehG&7q%U^e(=jF^HXCB5cx4R7{M8>UuI%v{SM@$LTsH$hcs+RE)>q`yG9K zeIdic!|Gp1_(UvM{uP)>PfyRZaNz~v+`yn97AoP`P@B;nBct<%;)|@`-r5}4pK2@- zedEUOQk6gsVOiPjZ2XGG;)_^1dwN8RW~O~IPfU7d9(pIX+U?U8M2}2FL`3;{Q3L(@ zI1Y70_WopJ384TEF)=Y=LBU&<-b>dza!}>io=I63FD|k=bZC{zuRf9N{3Az>BpmsW zPd-56x}4Q%e7dlV%r=Qlz7rX5>+8AO+S-KIu6<~D6YIpYaOJi|KY#u-+T&V1WpQ#! zlnUv2b-CE(X78KpD%^rz+oLIT? zu2CVQy1IJU)N=2gD!+#{umDLlDt)UWr$rH&GOypf(MmTjEXU^4^z|*MeEoXS@#DvZ zH*VZB*z{0m%hfenj~_n{!B4}wpPxOv=Eit)>I(m}%ayq(&6XUeKn`^zC}T=|4X4jy zQBi+&rkb?gcdi)<|~g^+PLf`U>B_f4@A6e^c0lhYs};R@gX* zva#APmoHLd{q+X#o11xl#jo)iw%l=N%i@ubFBakv@XGAN%GV|t)c1eLbJL+X-rd=H z;bn97;g#5w<=C(K{5DumBwhw8)4C#zH>dN<7isL}J;_Q!LYv)(+Vc3^wf z+}$n^9Tz84pJJ30*6jA^XpcUP+A>9VU3vxv=EH{%lMk-RK5PpXorQx{NFjiifMGg-i(+~I6B$hTdnq0Yloz=$1hI}MDzv5`=l(fBbS7GlekK6|Fkwp_{dc2ZId-+04=y$lrR(xs9yfx*G7RKnie zBMu(>_wB3mb61|c!Sg#Zu9#b?n1Q+D>6v+2xS+uJ_;{D2N4Zi>(q&c`(Q1jB-`#$b z=Sxd2@45nHpS?FrtE;!aZ)strqodoA?XYX)OK$tU zDZYj3Aw1G%v%z#><#rZXSy?MbM=9;7WvjmIRa5hq+pvL`hleM@qS#MZOiY{NkTgj% zKAf0r`(>Af!LED8t44EWWEM+#IV+ts*s|qfT&kUov<%I@C!P;R<1(|dxK8{U<=o)h zW}Le7S~agwfazlw1mYs$b?dS#uY7(}(JmBQw zIvo`fBg<@L8=7VpeEHs z_Ts9YpORYezS4M%r){1`#UFfgjfz{IQ2FxZ>6DaJ{&$0ORi{RGn(p13Xp(N;_cs1a z-S|LQXz1ptUq4v+ryc8(IWNgtLSLU_)}??ol6YP&)`vS?~B>uzs9LUO+SWNp5`Rmx1D}`uB zE-B22-1G3@W*wb*7AJmiuUWH3&aP_N#N=f6`%D|7Skz7-R0%AyroFxR`3n~^T{_j5 zsrd5YrZXNrqNg;qwG(vCE=R~~-~~oIcLJ)Ier*4Sa(%g(f3-C|*TRLFwlBEK%gZ~4 zJIgATTOV#K*a+ChzD&Vi(W9TDa2-ti{QP7~-$P6)mp^}Q^FG5$E2sO*mq<<#ZOfVj zt%*<9ojdybS*Y4#%BANe_vm2LLb`eEnrdp)?)#>vrl%&$%E~mni(?uBS$R8pdKeJs z5&WBYe*OA2G4fo1JO@oJt$5e&XH?uarQvVi)c5UM#>mF!k4@2y-+mA1b)>y$KJtm} z!Gm4BH%~s(5jYXB`3z%cXIbzBs&~i0KsfSgab{U|^~v37X4x!v?%ZKwXAeTWjD7tO zj%+5fcJ0FU6XUVXHo!17@f!0+zqIh)^!>){=;S06pmteizf0d+A>WY-UKbCK3gjjZ z4vt|n*Gr$WA6Y)7WfM?YhPS*TXGKeuG&JN?j_upOKhyJz8S-4bQIbBNvbQ{S%lGHE z$0+A1*4I|sFp6!x7Jw-0epszihP|HQHDc}M<6|&%_3}^Ok>O!~4qBb4Wyc@9&@*N^ zdf*yx8wYCIrPL=gH)juJnh3>x9oW$B``u`AS1r@6(9lpmIZL(pZ!=RvCu#lB3~{J4jeI!P&P)aR zO|b}RBJ~lphQw9=>=_GX4WJ*`pLFL=7y?-@_r{H7hmd*HqKn^4?!GUMoA^Ggg7917rkFM9(3 zG0m-VPBn71F~=O--dQUg#~!UbStr4}Qm?x~DXdm4R~o+PV>yEiW&RB=x#` zy9~ZOyw@@O>*I?HNo$TBJ7(qVoW9z4iC^ z>~ykC_nZ2afP_NpKRVrn6dWI2+B>sNR*{}fZJC0LnuLL#9w+KXlAPWn*uI4p2H&0ihomi8e zbXl2xU)>=pX)#TK-&8(&BZ0 zMk&Kj=LvM6x*c`Dn+@1vSmb*gxhwv_t4oi)e@GBK{q@5WI%>_@wKNn;*OJIYOULfKNY}{z|>XP(Ce`-3};{pN#t$A*D%g69=0|DLGmML&hF1=O!zN0ld_DH?! zotgt~-MXc|lI_@+<{yzVBHt!Px`kItEnd7>ujS@c?&Exh7S?GVhU)r<_jW%BLlC&a-U8xPMH_voE&dbxQo;U1HpNym*crd;= z#3)4*crM@bOW3_#sro%*V`CE&-Dh2qqdsRB18CPh*vpF+Ub1f6)5CfuCMLhGrx;5B z%m}Hf3d~<3FYd|ccc`H6VJ%vtdy~VR8m+}A1rSbHJ8j@nv%LsqCN?%>$w_32_`7%S zuCGl?P2GF1(Z|(%Jo`5D^$pHJ`bp|TZH4vLSq~qcdHC?*qIa)+ix&B==U&z$_$K-Q zPexP}JN9PuwQKj*?>*mkAJB*t5yb98ic{O*Hx zb|&`}mMeSHt0~vi)zmzF8?Uk2z0dIWojYA9KfM?(4I$L~xbzl7MwpjHujPq~X z9>SdhI0=`Ql_fQ0V%;1BN^^2@l2ihAEP3|qS>=lttBlQECbBwM6Fb_w`gt5>y^gQ| z&^Ud_=LY-^!NQwZoY-amBrQ#*`O%@o-Jj6)+S%Df2&gWbF06;+KV;1&YU*Yxf$@?A{n zEBRiEjvguX)L&{7xnv!SazjOhmg1n9VR~|P){=D&$)aMHWi7zCFgiLqhDEFr6-_F? zQdm&H!NU^-WYz(Q1b8((I(m_-jFdJ+ihaZVro(Tl{HF70;#5=_s<$5c@Z{J6q(Lo} zVB|l;6ywJp13W;wp-4ld(91fy0Zv`nbWEnJySx69FVKp2%KNw<3Dqxwv@N^JL*G9+ zmVPmV$7|Y+fO`j3DgV3aoIXC2>Z-$G-sbuP9u@}ITz-B#z$6qlWiG2w*m*$3f<@)J ze0H1QlxowZQ&-m==6M#e_zV(AVT0^0iMWRk-QvqT_Q=S{Q2uDHQ>L0_Uk1%z$})BD zgvufDs-NF`X%^* z`^G6;4t2?=fI2Rkrgv;y)gON1YinL#d$DTHkq=>&mAVmpt7(DR6+l$6R`2v3a~`>0 zTXaz0NG^XI<&E2?%4CN~O7%P2h>9I4#u=HJPZ0rOt>#6(Ke+_jQ4J`hig_+yn%AM> z5I`FXlv0|Y6^_0|1mMXMOOnpr+v0UZzq`@mq!xH7()AEdV*UE{$ktl+_Q~#TZK>j) zAFctxgl5t|FK=Vf)Yx9*)ZOz54R_*~e9`o;0rkW;)lob}EP?7fK7PM%EK&cp`0HW4 zh2S}W$~@7ze>~jRG=AWaDtRhk5Y`pbnQhCV&j83LTO0et%F@!XZ|j9im&8oO%~C<$ zFwLJ|kzPFgtCapNuqPn5camR2+J4qR^;0=cEkzkeu*EpIxdWdZ8{)Z6UAulgb*ew- z;lr|#?&tOWN$R<|t)pAO%tQio z+aNL(j(%i9UmTQ?A-CnG%2c+s5k7DcA{Rm9lg0`1#c+_K9Z01fyg%{`} z(qlQbpfkSxg<*y#VvCB!&>p;~3>B^B{+@>>`inr5M4*n5Ykutc<=SY`kFhar7Z*8n z{!B?m%+Ah3TrJUu<&yy1#CP7=65mThVdKOnCW=hSp*opGtNO=rFJB&o6l{&O;4nEn z=G8Pu!Gvd^(hBpG1Kq<+x-uB5D7Q*^@T z_wVo36Ts`NPUt{l*f%z}v?PM4>TjV${MnHOx4X%pW8mF{aKZy%i zS}!`Z^JD1o|p8>8>gm^PoEKUR2wU9lAK z1VA6R8I#Y8^P;l8_k)AyVX=dTif4){dp~dG?{|KCiv|f33v(x@8uX+ZdIL~?OdX$n z%07JHN8;N(X_bXIQcwR5xUG?5Y>f;b_zUn!$J_fCYxg(oRQ$m8_k`;z8uw$zg0Eg( zjE<54G=rCy*Twfip`pyw#e2n#@u{id03cR?h7KnDdfT==Bh(wX1B1H>qkT1W=-S?+ zRf+dxF`r~e)UPh$MX3tv>$8viTK4_Kr%#`X>O@}xvYfklGkwh4L?Xttz{@p09}%g6 zirYM(uzr2c+XG81K4sflTPKVZucJVw@PqQcqUhleUo9yV;9fO?5)0~5U$Op_yg?3U zNn`BvHs@#97H`1=k(b9RdYnFccEOe{TS%6^Z&AD)U0KC=`^lM7&u`zqkDwu?_3}Q_ z61j9khEvP>8_GUB56rSl0ke|wbDi6@6SX7X=6f6l_$jHXVqdjt)x^|zJBW{PP^vKs ztt8cb{Pf8hP^ParTe835LHOr)cggm=vfgP47$Ui2U-uzaxKN06kV;$tp^W&|-xrOwp4sSvnC!-e_e`$lONJH6blvet(WhewO( zSd$t-E&?n!a1x=TK_QDqnW7QzK%1$J*4*&n!L9U+jD&r9?AFT;8gU-lSI73Escf6N z#>VHkT{P8%i0Mc?t8)I*DOY!QO+&+T#vA127XdtKB5$m=ETI6+tVX&jlJ`8kV|Knd zL5tNF#12W#SJ&E+hub$W(2b|`{;{hFkIcU0vBX>AWEFCc@IyIq*EKaU;l;MEz`CMe zex=@fOSu~1PohWJXFZ4kOMCke;Cx;mm8O9!Dkl^Plt%0lEd2J}yE5S45VU8cv$(h6 z;A_5s;9vsQ(Ctx5N=oN0T?zu46~bD23^Y_gdd|2HbtK<)BVj>V_nnW#4N)efQYc zR?d0s%Q*D(^w6$@$27Jd>Ad{mvYm6a%t^rjjzfnIk!48Omv=M)j4)sX4>s6Bt(+$Z zUhxp}obZ-O`Z49?n_$GFJ#4J4H6agaf*cA2hbV_-@aV7OOw>6mGdeCVVUCFG?Cuuv z1UY-}#I|kQcz}?s5${H5sv66&`kL7e^#MTO;(K%M+an{zuMgkL_~~bFd~(GNw5S!g z^)3X&#>S#9^MEDbadmYiSJxFTTO>Y`$xyu$3WrwS(VhT6sAMp`eIK6WzK7lxI=T04 z+jj%QMcJbXe!}sHLtV}o>0>d5i)z42E9Z=E=CA`p)f6+8m>76Txm8~+?^O_)_#P$-Hj3!zQ%)Hpdf1kwD- z26*K!dySo-MJQQvM<(fUgvkr@jSCO!KAtpnUkP%9eeLZ~vL4(L#vEWFV=i6u9`o3f z!Qs9GJX!^=MnamNs7E|`XKS#SwH1{9R~$5wsU)&(&mQ;-`qkOBQ*X6={R0M7pN)RK1Q3A z*SU14p)g-mm^5Sm{P}Z6R#qu=%_ObEq0^~p7RBrkFz=(oscZaPFUHCvRdQpan+tl0 z#<+bNvSKY8{n6uIV4tt#^Ed$+0x&vxpVeERmz9-FCugQMAPEro4V1YAfnS2+!yzC* z6tt3e?-n9aq-U#K$$_}?G<4xgmRP&ndJKmJRStt8rTh^oFL~y17W$x0lN_<-hC=j< zn_Fx*%^L>MFaf5UC8p*5dzYs=65Mg%NIJ}R!52l0PFYzQQ8e%B)vLAT z=Nk802Ze-~$2s;=en09>6g&s{fHFhS)pcW?b+>KP#s+>11v)ZqK$HBL*`7mBD0+=x z_6j`BTgDP-!-*72sEX9x54!Xw$;;M6OVdcX3^YjJ*mR5?U`@Ty`-fK%4<{#Q^lRJe z48O5J!kacNg~W`z)j+@AIXD>c{H#ECi*s>|BpdNLJXqPnbPJs?*o(dPJwNN}jNiY1 zzbJqB$D~@4QRmH*)2DC?j*aFpK_oPa1e8EG+YztMkUj9^@WQ5pG^|2lTh61p?|=rB zSL&6Btt#-j^b@Xa)GrVHCph)doH8#-`8djK&mPGCYVn4*Kv+8H-7-okM~)h<;I*(< zWQznv1^~}s?6P$iuF5<20Hl(qPdEQs-l)A~JH;zyN(s>u+BaqA-O|!h_2z??mXwZ; zj+eK$1Abdv_u!j1Z>~~YoScPrkA99yN3QfoXi*d*7g{=-w|5r5hZx>CLAcWMlMPT@ z^rcI?lah!5b%ktiTYk&PRF22ROB#g2-Kl0)QSO8`}lY$!U5rD$Kdm)5k^wi`tj zykI3aW6j~KsLTlOAW?%$f-*Af5ACX-VOJ3%IAt3m3zs^KtOI?M9jJ_413_$*+V0&v zKt=P=o`E(dtO!887N`)cX4n@WfEp0n+Pb<-+it2v0uVtq<3*lX((WuQ{L8s!VvQU` zbXp3;v=xA`9Bw~nRg69;OsX;6LTAgC2(hxaA+KL=F*P+66cIVoOF4utQ2osTI)yOl zx3f_6O5VTcL&GRZ-9j1edv|vM^6-j!kk)DdL~YJbgkoW+HZHW#Wfbd)!>&A^6 zL7T|Q%U@h>V`~fDooE%wm9tw5i&hHuF=-9H`{>X+#+sU%C9O7CCfQmq+ACMCFj1%% z-0Pj&*d!$-w;t2Gy&8{H1666?zI{letk{f;*B&fsZ@0Mj5Kj$+BOgk}>A1M%(B6mN zZC8ET(z29#ds}ZI-mC-9@)`1hP(aCJ*KTZQR}@X*$wDxW7xhJl*Zi75bFbn(z4k=Y# z72&R>@3m`1nGUyqDu|87L(pGkMq)PELrDtcagM)(g z%_1O^AfOD!@hc%A)8V#DkMM0cvJ9Ce^W=;lL|nZdr{+gbad976&h%y5+S&0gNqSdb zA9HZ%Q%{eOtZaI@Vs%zdPEb;kC=qqQK7&!ptMr@C#zq7*WuGzK0}?STK4XK&oq3H` z3Mb_1poIb@u@H6xHOLlqyW7(CJQPbaFO)Fh(@akSIdTO`4Pgn;VnHNjB-|h15~Wlq zu?5?RtOt@pA`Al|VPU0sQJq*A$|ONG;ub|p35kiTfi{Q;1m4z&AKff5tYDAQ4Z9z| z>hDj<+rOp229`pAITsTX2y}4#COwkh(2Gk{8^}D+LxC?JW@SD4Wh6D~x2M2sHPMZP zG=c<0MHv7+O3;QxLB2BZXh^p>3reP}yC#%Z@hJU{ z9XpN&Zbn(S{loS;eJwgkWIIN%KFRx^tVedRvavaXZW-VY*^$(*h6`UfhEGP{2|j06 z(R$KVR9JdlTHKEo_ERBkJC*3Ax`iw|K4+iIg$fLJ66JsW`qF&&&$P!9iV$P$R_)=hok$=+di^F$DyUKiRo!nBdp ziRLCAA_kw5SH?ao4qVS6om7L`X~A%qm&nhxayETizQpB@h>fRUwk5-#6E6&c^b!>gg~+DD@lnn z5o}q(f&UQp-ta$|7e)5cPGHMo7a5E@w?00_z{(@IW{puA2YGjFUn=x`%Db?*FmS~M?n%PZ=S#eW8_o&kkl z{zxkyv@I$wim?IwaGD&s^ha^pvJB(*L2@OaD8L0K1Cl~AW^!WU7M|!|S589+4o^={ zhXdGu=htG>0SVo$q0fL+Ni216>eQgXat$15E7)@(o;w6w9n7|R)Q=xOT*n6YY|JN@k-&Qnkq-mUX%MSDk&lv$B~~A*T>+`>DJn0aFj1`p z1J3m`3CSJ*W((&>8NQF?xJsV(>ZMDJFuK4fQ&wBc1;U#ktTx>U^6^6wO)?CZZ{9r1 zExFqg)dvnQe>54*V_2z}{0C45ZSZtW3%%D7%>XMcjP4f}5z@y~mPqENkL)=S-}8|J zwp^2Eh7s*C>}&9HQ54K^WU&dYP*9)*M#LH%FM;Q6tx*ecmxVTqx|*!k&PPSrHL_WJ z5rhWkl$Kw^WecnhA5LAL@IFyz0a7WU#{)TNNoIrPhcxjt6k#tw$e_84cWy5V2@gMm zt-kGs!YSA`DB{?KM==;1DN#L;MK*#6`MBmvppLw}<0;*;5B97cZ}+;cIm6xEzo+;M zM@XM?(&wR}@l%c>{;&i2*f_0oYF-3_RYzT&_MS<46sR8JXIP>I7ERO0hzl@DMEO~1 zDJ}XIZFhJ1Ut?dwk*G|~%=BLfPy{$ADk>5{545#Axg7`(t{vTRM11>u74n2%Xn}z* zLHB`mmDG^nAkAJ#nf`?oJxE`JSZS*$RQNf8q@aEP^jtl z?M2E6x&UYpAA5SNV06L#JEt6^W+Ob^)_{UwN4m`|*VoT4dADL~l93g*ji?Og&Wo_| z+MvPUTVu-vF5VCg6k+(TC>_r-wga;W_fjcnD{>>C_MgQLFo56VIupXa9wn~7+_SmJ9wzjs`Wn$P0;?dS9xi;}M zI^EGR7k)Ki?_K-m89sBP4%gocgaJz4xosKp{ARQi$jebcspD&Y;y`ih8~!G$65J~`SR$yxPmyruYO8>g;W7=wMdzI)2;=)>&dbCN!7H`0O&vHgT0;FC>5}J?f!DzEyD!ExztF0cyZ< zG!&syKn1_zrX|~S^RqoAd-m)hOzq&8FK|ZA$7&cS8`2>6e@yDYGKAQ!*h|Z|Y*_?o zA`sL3DCoM|w?D^yaN+^L&i(2yyoDBx1GEtOQvCydettxW0|3bc{hFsPgl7#{6-hy+ zFN4d;FfTH4{sI~j{4aEK=)of|gn!fP+s9DjF<^@=v5MTN&vP|*6MX^vXCQeKpoc;I zk)Z^^pMf=b568$Kn1s7G%Xa9*{Sj^(Y8wqDaAK4bIgywb=bi^W2levesxuJ6Uo?r5 zWl+{ZC5B$9zdsjyU`YfZY)A+TP!TNchg14jEnB^*1Wy|4sq^5y)m03l=p&DVyLTT|0NXkiJm?o-%m+&;bDjP@`73k)k0@J5Wl zp_T}1knYy4^|lJ+#q{%U((z$oL6{RppAX(X6cSW0RwP&@uVIqfGjejY(W1Ie|Nbc~ z%RT#)oi9CWq#1jN4^M&iUrus{J{fIja`6#I59GL3RR+>v!N>ceI4 zJ$(35xON$PA-S75s1d|0i0tZy4(QBwm7nyWhR=Z5Ah{a*>|T9nc(?|TF}MsG*fixG z-*e1*IT!NtMQPn-vR zit!IyTicS>*58r2s*<0kmpWhQE4JRJJdn#zJEeol%?|tZGk8$KAH0Ko1ab-yd@33a z)!t@q&T$IIrjN-VLfuG2z9r(4|J}>R9xg5=AXrv|vZx)RkI{YC-abT!EMT-~s6ilNXVa0^2^|~=5#E+BrbC&6n~fb@EMXY10GE%15*>1}p* zf22JY^XM{aG@v&yE-OV;HloFVT1AJq<%-!KjBE;^1XTvxw zpXY8&A$Z$+7EVL9z^6tuL5%MFf!)-;-kQ-^~=#o6g2Aj?RzY5CBk4g_N zu9>;uME|y-frnStJOG|3Y(G3rRtjc^AQbG`tP)e%*0vlviNgVWramQU9|UP2*ML%WNeQ)l`EqC>XN~U|r4R!eAWPAhAE&ZG zjz$TVcQZOsLf?{5f#-^%_V3?ckATSM<4 z&gWZ@c1<%ZX|l4iZkJVRI)^zj3sOXxrG_yQkRk;$TNUXF_3O~+!qLP)G`HM#Kf(k^ z!I}W%Zxjm|fvAH}OGvXoDiqYncx-*6SiF5F2nIY=bu+UiL^1|ch)0s2O%GkuQ!^Id zQqf1k8oOLv%Q+Gn3CMwc0Jb_!h=PEX(dW+j6M2KMweIfaFy8bq zbw_q7`fEBbXrx(@U{c{h9Z;I*f;A$290Cmwk}jnHT;&8*q@KT^0XkLS{@~)`;^zD9 zSQ=p!74tuBB#8PpkosuhH^(1E(4wm$o7?;DuxprzE){ZBh4s03P4gH2hWqL7AMlegxrqGwNze_))cfmgqso!C?Vy7>#S(lYGClr zR-DwLQCZLqU{VQrLy>W%YlC6rE%fygnJu4s_(9u)X&?6JR|AVlqgr?%>jqu&-MhiE zMH93TjG_}ak>Vn8%JG64JZb?~uJGmM6U>)jU75r7`kau4ocE}GYtf0Nh;oQkJ6g=| zZsVUr3ZnkAOd+z3y2MWLMI7qroZ+K?$<2b#ir#%eng*5eVyfi7pTTU2ee|*apDPp` zh*5)b9FE+Cy~ILFq5s_hIt1FTs((8DYmUqv(7HL6En`EwbNlXHgWlhJ%*>8P+mrWO z4D1I4#)nw|=)G6SzDg+oUQjsp^~PUd1G5+?t?7n&@$o{~0e=BU#f=*~!Pf@9#QnnB zN+?160fyaE^7QFbK1Fx&IHlvXLiAArs!>EfLhSz^@#QzxWCsBC3_5LeY-}VEZb)Mb zPAo|)fpl%U-);9ZJ_BQitojUn`nii21JF>|K>baxYk)>L{9@(JTTogF3`4nc{q|As z&=U??RPdm}LS+mwfl^L0-0EF_`=Jf zPQvesf~zng=_u2O2j_z?BBd|kpQfIS%>8NVz4DDCZ{8FjgJVFlod=FZW3l=Q1^3b_ zpf}5&B%VzSbl^9z$-f&p0cy=hJtYMP9RZmOWLvz39=s!S-Va;^KK#%KqJ*`C#lx`2{`urT*9@@^SoP__(7VD}`TS>)!MHYJLbqCtLHg zD&dDegWkrnJcN^ofT&!kfD{FV#NjMl+%Y+#qN3c2Y!9%F5T5!TahES#I0IJTv){pk zpA0Itk|!eLCD}`n^^n+Dew1Q5_~)tZ3mTVi@?--d5~YZuiv~l%qkj=>8x#dLEf7Xw zz?|^G#1xP75PJ462{4b6`lS4VE{1(!BO8K6+6C{!$ZEj)*t(|EI6!IZ=%nuVe4vJ%;R3(gU$iRpVd7L_iNS?4rg0X2|0r9TuS`}0Vd`opQ4r%tGL;mRwN=jU1m z+&B$|)Sn2-i1?b3Tu!4?o2#(WG)myQ3JMA+s1!+8gls@$I}ltN|9Mddz<@uIi(n}zB}OY~T9lf;J`1Sgzfi?C)NnG51r$i; zxCuq&rMSct)|a{U-p`U|R~=`MSt>Fy0N|nD14_8B;lcGkm1$2pgQK0@yiJ=nWoIGY zUo0lQesse-d)oJ)d5jt{%l(j1WVUOiU7|eiElCeouV1%;KNFsh+mEK|Qg#A$E`Y`- zKPM`P_`^tkaeC%QqeUVRC9=yI$o^!q%^Eha8w^{JnO2BE(oZ+dq{S`7T>MSrk5>RI z!VS(cclo0JP;G!JWsR2$2rR$|&fH_(Qv&-DDLzC)d-Gcvq$uevAZz{g%P5x0gM)*_ zDmcqgS~@$QC3GUqN~VBjD&7?zj~kRYH|&BA$Ng4j!o z{WpwuPrV1dJ-FDvE~c=s5S#AaCD>W@1A>A`PdoS1>-1|-=|Jm3#90F0H!Vd>D7a?g zrT_Ch(KLQ&ZFNBdJo@#+Qt0ts6T?Dise&gayvRh-T$G9UQ-m9QePiR!Dw$bnXJeL_ z|AOU8C9=i8XN9iqIt;|H1lYInkI=cCYYlI2KKwb~uSjWeY0ic2A^fV`W z(ohoB5Bv@z5D~TAj%EQ$B{7*`RA`QVTTknSK|K)1X8O~V8oY%-JdZJ`HFrB)CAfHl z-f3k?*c0IR`Hz)EsDq6CeXNYe>shYJAEKoQEqF<0e;7D|PKaDaAE0Daa?e^+8sJ{} zjZ+3F3KlBqqu15rG55w2c=~b>$uG5nV1oeA^8QgpxR4RsFvo+=A}}t_`!a%%n8FJR z3w!=ZdFz>PLneUKDQO%R9eoOr|F29SrBFO{oOt?Hum1DU?~Z7cz^Xcr!l?S_P`d!C zQ|)@l`hwVYDUhIMqNrL#8ljfeDLb2(~5iy6!}(XX&zpIK&p4K`G8KtN?ZH?}Zmj9QMKAc^GBBbNV@Mx-Z*QNL#(=cDjkI>Gya z9o-Dwgt#2x%&p{{CH|%ix<&k9XFP?f|Lm;5S=@&W7#(x~Fm=-dwvpx)#@54%vo4=s zm!B;x!K1kaNUp1|UkU>!sUzrY06dAwWaYLSC9oo`R`Z&n}nVKTkni zoC*GaZ3hR58?mvOn0UNzUTF5$jT3_u#=>f+1_vGSnPupEDMPdZ*x)9Ay$rkv4?>A_ zq}aqo6J4xm^~zqqW}{pnF#w&y2She8F4m5w8s;f-AOzGNRDD9Q!2ft?68Vql?puS2 zmJBY>ixzw;QI3Iw7sKv>7X4IQ+sv^R7!bDxtC1H~pF6;>M!9%;Ui#B(NHKR{y+CVb z3G<$Y=}#371i0kL(-tgSaSi~x6l5RquEBRktkjUD5sLIAu_NH>4>A+hGy32mC7NaH z92Q|p(FNF?YVu@-d)w-EG5Lca_J_SKfCsGFP$TZVB@X`KwLqg-kH2Z;r z0>e8(EHgw|udb||O?ODM@klPvDN8`UyMCe4L@RvqWRs$MpFcLM z6*x**p-vRW5|@{EFcwgL-MR|+MUd1r$Yd~bJN#Yeb8;aq!zSmQlmov~7+E7SG0{WD z$;{2;%pLOXqW%_art_mh(8k2b zhpnd>?*xVyoq>N+Qj%HTKLi;tx7bNcHs)}Pu3SkLpmqv)qp^trmzt2Pj-7msut3B* z4`+{0cjy5VG_c$g6$LHZW+;PbL5La5 z+dIFx8%zl-HuFd?ASo~TxWjLAY;Ppy*-+bRzW~rW`}N{IL*xeww@@d1eO)jl3qOW- z{yI2PK<7B zH91J4X?(U#bCjIqKe|9*{Av_zGNh@k9i+`ZuR1Jm=2^>% zU+H`%)xVsAhqZ@KY%^BcT3EeqYHCX5Z6DJu{iIsk?3B1j(KcflVH?{ z+4+Duc(#>-NM~`+8+DCv@|@9IU?UI{6LZKqj_VQjW>A`U4o^b&Rb@U0dY0Tw*!)^? zRu8t6Lkq;fm3FgSj+sFYy$kRinMfd`vHu_kR=Dz==q7y*5h$Ls|z$irw(tqIv?Bz~qO%$-p;X=gky$g6qzvsO(z7ok})=YKnjOQc205@nlY9;^_=f>Su@ zrlknO_J1`=B05p!AP(PV+UWet>B3Ei5AR!y4<|%sT`oI-^2>iMND8C~`m6L}SnT?GbHimjpN57OXJ?0|OdKMt?z9M;!TqgIhvTb#)}3V9p4D zqXQEibB!0h3+Cf+nos~Neu(iP*leD@c)^UMW?JBtIrkA&H13eJK%Qjccr2XNK}X@7 z3c?}&r>lkok`YI{Y{rNeWURd@O7Ng#bLhaHKGz<2GAY0rgh~?l5)<4Z|FykSXA)l& zrrv{TvY^1gz&{@yDL|VB84N%U|MT_ry>)kdX1_#?{t7X%1g!C2NSkm%lt1|X5;&JJ zrct{d?GDCm$N=+SF9V-LKMS>q4gy}_-F`w~!%TIZVHpNF{%ou!u~W0wttY4c$-Q&V ztTr=RF!QdTUN1(&)Zr9Jfa2{KY|H;e+!L<#M!Eo*Fsopb!_0JH6Y}*{!XuHhDy9Ex z^ANKuXfmn;P+t4rNi#M+ISMHK!k=Wlm)0KMCZHMh6k}u^T2Pq`6kK^4nAx9e3Yjt0 zCg?r>H4>@`4D`2<&nbU&OiwE+dXk!1Fb)JuDRD?FUAh!%A2-pZQon<>#VK5ii1G`! zFD9E$ef(&ftq7f06WlI2N(g}4^7WO~FlwGGwE&$i0v3vI=~qxN@aV{>`gbf4*EjYYmbHb^!=T_ng6?0MRDlQO1Es%KH^(zV2FxQ%nM(lEtGz z#(@t!lpwsg5EG458E|97yIOyI{ZGmF<{1yfAu?bbv|qA958*nR_O@>$aq!hTfd_Xv zc1#DuuON8IxE?YZ8AXRn$EQbyvjfvSN=$i?=>r_v zq9KvGBa!h__06-e=@c1VM|hRt2(>`$RC1h+Q69n?jv2U>(srvtGBpc-TUC|z_I}6d zNrwN%-!+W1rzMk_U@XBZYAq`qP(p?o(98{D3KJBrLyNMK(yjFLv$GcM{~d0Q8I6!V zUAP^)70`7Usvg1KMooYgB0E0+EXMJZxDg<4Dw;Ua^t0GrkrzeCI?Khm6!kF7Zw4Q3K^W5kRG{j}sC zR+}7dLKF?mViz0l#gQl)h+mwG#_Z$c^Et<=j4`DOQuheXA0&wk@eQjxKV}Q0oEq!v zE3mi>;A21*cHuyY%}BPx@J<$Go|x@rcwfMaji!>=VU+e?O9F^H>N%tB%CDA?8$3X9 zO^xF}apcw-Ijt1)!e2hIqqt7a!Xc~=*g3Eufmm5$NJheSnnVcB5hlPhXgNsFN>W1V zZfSItf3O!Jpkbam`Fms@#+Ef`D2`$i6r^<}ZH|`Z9UUr7~Sz$xi~l#tq!D zFAgIaY{^|frm``bhM~yxoT;XDzYuRBx^B2OSe$Df?A`DexB}sAl$J(bkW8N?OMe1; z8NvfIWtgZvpOoCMM2IF__60SFb#j&^6fwU-1;^1k$vD4@k$qVR7>K`eFku^D%?CIm z*1FMAca&0YlEaj}AN{FKfI-eU85B7kUaTZza`*?v4+&9+$9Q3mR%9Y9F&;5188{o} zki&gON8mOv=S7Y*!`dBh}C0LWfi3NsBjjV}P9_41Dt zEKPGCTb~f=Xn2KGRF+eO%zp}P5C@pdagF=uWuTqslea&2W47Fpb)JXemPL{{Psz9l zB<<@s<-Px@8y(ObnRt0GtM|aJ0R8;R+%0pJK;psT@k+>IPysZwAp*IhesOLXUGX;<{IIo3J%4kLzC@3hMWcCG< zHh;0?5{t;CF2&RZIn4`7Cg@x?G<$!5h!?vh5usohB~}64A^`W?(`U8>5d{!IMz<$^ z{?uE05@UsxZnJvAtl4eRB6>`Hk|qkL&_=(U9Q_PxhiNV>V(D;Ep;&(6M04nYF$VH- z(A?T+Q?T~eX-~7@(_Mv`{ygV@y`7(nz*544~mqhRog4Y*Bg?jb-hK|5N4f%U&se@iHy z7_M0H!6M&%RqKZj#C+Bbt#B?r45;UDhM@3KC20NrRmK6agcTG_=b|FmA5H^w?Ti$( z$Ax4p8y1r;Bv4EU&fyXWJLr9Iv=0qMh(FBoQxx!jj5gyep8XCy3s=KNNCP-aW9>3k zzh&@kQ52jAi}Dr!x!EY(EM_rJfu_I=DzOlPR^p^GY@V~j`Sl;8g@`|_(a(?pcP0D6 z5O;JIifNv6ot1?`+UEZ+$1Z2Rp{;^>7Gnp*UUU2QZA>kuI2GeOkA1VQ{X^@(B3G+S zvSHVgq>kgh76hsbnKTlZKgxz5qHF6oIWs8$dxn_Ftl8C=9Sy}3!Dth{%1BQfz z2|*5aYL+J!4G5}7JxB%C)FMuOSkYYJ?8dp5r8of@YJ@`z{4CNY|FrFBBnYHE>Y+t` zK@?_AHfHANe~CyM-qR$**REN@PXmvZ6m&C7C#U(6lBryc(=dHG3dBl5e@?NgLcK&ucbrL${1 zXGHzUaRv_$M!H+TBl3@LnoMiYZAz|^jc%VTF>VP3gO0d$Ar6r?o(vQt3H7ifp2JZu zi9~k(3`8R^QRm>cr-@aN!xWN-wyBu3;uv5T`rA zz)IZl#88J}eneA2j2C(QA8&D(6>wPKW@;cIOYO;`r!a8Y(b3@#8p{Skp{%bjCIiV# zU34^!o7?vd%iHf?nRG7LnlKaQ`s04PCzCUMd#`52ySmAJ|M&0n?r6Tr+JA|}MZMV?9HvF+hBl3~KsKvRfhlc-#v<&SnW=f1=7PNe@ z?!j2lIXzWc?8(i-a8W;LUpz&D@?L-wZpeVaQTz+gn)l}w%;Cecn{AkfL%6lAS0r?C z`>o`vIT?AUy?23tN&vdKAfMwK&>^s601O0bDb9;w72Q7+j#v^whRIJiN+H-22pL^U zK?RyJaDbgyBaGf0`qD|8Fr!#hoejDQZ>Jf^!rkU6?i`6{bM)vMB$D>4AFJVF!XO03 zp?6DepPdHWAANSUyu8$-gYjTf@2KBCb`A$474wBmcTcD^vNtP!Gf{&f2k@pGU3yRd z*FXp}|Hri7sk?W3_eP-m4$&oT_&%6B0iXh*^Cw_jWtOW^!THm9>e2M`=7|UjQaB$? z4PssfWDeT{2XKU@;h5=c3j!`+hk}kE!H%fq1LPTM$xVO1^n$8{3s9Et&z{TVXmuPdL0Dsi z{xY18=M=pAHpvR`>B6|=4~rDYXjm&Uj4T8c5sMfYByyA6r!Ao#=osC|VO4-DWb_e2 zywTtR*3hBZhZrzJy#|t_fkv|G%U6z=khN?2VrwxnL=M-2#gQETLOi!Pe+SUR z;ZZwI7cud^ffMM6A%GlR0_O@=r4)?_R1b@b{Ubzh{E-v_lS&3oB`&Y1utHnTp?(WZ zZI*@3^}b#Ei~abCck{_H*`mp>o5*}U9toKUMmzN@r%W;%gd#cAr4$xKl1SjtOF*Jm zI63{BRRE4@Q6hfjH({23?*i_@ z0as2_f;f3g|3mP3_XjKAt&Q%m_HQf>(NYd|b9(1)dqz z_XWq-nmw}LVOms3ki(OaS9YYH1iB!n40W@U?!;>|yRmq_`N#LgBKg^%f!BIi1n2#sN zi^Hxg0V!G*{Y`y8#bMz)5@@yuC9~qPh2I{>C**uxXnE(pB21Zp&&jFZyVEUaxl0(_ zx&jlU|F7!K_NU4^jN^A6&PrG*P9lh-7?_9WSP+GUhCm%@4j9pR5Km%b5)v#xqmTy; zH~2Bn=}V2aVPXIj|t?z@#-WM{pQKLW@2ZtvCJ$z2w|4&ds@hzu$FT z-=|y6vv6Zos}+eAGcz+Byw?)xDoJPFG+(S{7F5HQhnNb5$G293c;wG7zcPD<--8HT zp3wFmi%Uv2Q$;4J^%2p%+WV9z;%6qhnFXvtL?bjDZ+E7rwba7MC?-pIa90v!uwTi` z2qf~pt(&f(5I!J039(P7*Uz@xu9?V;<_eGfooo3RD^(CMdWo63Av`7VdA?MK!!3;xB5pazBTS*4?ttIH#0d2wDT&e|O4tUhO90uHE z!G0W&ZBNDRqm#3CaEMDbq44~LHU%rn2hC&0D_QdFap4USlgSRlvNI5RZI82G`1k4^ zlarIQHMP&R&oCm;f_5xj2ULcwyto3tD2b*wMsr~`F)Szwp=ZY+Z((P5XpX=h1w-qj z{aNn3u@vZ=G%WV?$A~-i!C}dWx*IskN>lAvR*V+G_|4ryOV0IAIi5wt2N01@4R>;O zzBp_!l$I87o8S&2nBIo<9ucqBXRyBMW|#uZ_L3O6W$1-n!=sX>;BrX;xZbt z;D4xE9hbVraj;b)w!uhae&Y1BG7({;vWc?+hB!yh4?-7b=PAT2iOH zH1G7-k=jVdJ{M)3!BUx&=?No7_=-TuHxc44h3A zvf^RoI=cYkj~LL(t20>g`4FY(ZMP-zAhYRHbh|ohx}sxaRSIp>Uf@4_szX$;S{_8zn7NJ>PnOK$37;zmT{IXDF{A3A80GlHCEGTMadTV@_e{N%8 zBBoMBA#*DaTA%Ob#U+_D_K@po6k+64W#C)Z0B zQ*Ps_3WjfC9SGd*9^Lu|7pVgl;;!dB0o+l8Bb5i7Q1oOrOpkniAw37(^e9QbQiKG= z1)ddU+1R|F4A%yOA$=8VZvoS~pkI>X%8@}D|C@!4RU0cR-OuKq_s?)mTsB%&pF3A4 zI!WG`!yYXH(-wU;I%Bar4*v$uZ6h=&l$;?3CUYj@>!mz6+2ILBTOo&)^Xk7ll5U*fxw|_7LS~QrCf+N{<%$G zvnpo5g_Jw)dIs00Jn9js@d2NZs4R%1b=S@ZG6f!-@&A99@9#>hdfVmM!&V`@3QW-j L?+dydsL%fgR1x7I literal 0 HcmV?d00001 diff --git a/docs/source/_static/images/benchmark-results/0.5.0/tpcds_queries_compare.png b/docs/source/_static/images/benchmark-results/0.5.0/tpcds_queries_compare.png new file mode 100644 index 0000000000000000000000000000000000000000..f83a0b4b707e3b6b1d1406c88bffcd79787c08d0 GIT binary patch literal 42959 zcmeFa2UL~Uw>^4{NxZSfSg~Og8#Pjt4jNGb0R`z@1*D09h)7KgHV_m9q@z+qdKaV^ z6cG?mX@Uq!6%i3oinKS^LE|yM+WBcZ2robP>chv59kmtZ3f0T(dbj=9up5ZDx1LD`-;D5*CRnZq2DEa(JFJ zYA01Wp5JVenBD(Ibt4xS*DQ?U%a@aqpWMnXKjJ$#_8IO%U%hgEyWt$x7u>Gx)tJuu zLd3>}wX8ogHqQLtOMIS?|DB7^Gr{?#ZmK7RnS?nQXIGdvie8 zg4dHnZN5s;dpEX~YzWu6TBkXmUKg}#%B|k)-`!cA>oD*pr#rEEM9Y2q&z<3}6ZVn? z8fyM4;;vOnO^O$MtVw#3Ym#ex>w_8wvO`P zfM|u_jt282-JiRed-BfBou78=#+>a}llLTT2zN@&)kuD)nr+iAXkN?h*ps1t|HAUD z2MuW^l@*KBbuJ}jHHq52u9p4qx~3|}u6u>+Sf9iAhq{2Eo=3mVOE{XRm0=cTROVlA z-|TEV^xWR$&)>iE2uU;e^Y_^+MFq93^6Y9gO+w={YPC$C=h)2^X}c&sIX2KdB3T|L zAvVxaK3dVKZ~xq`-l|~yNxWtn9%!AvlkvkVm#%w>a^jw4LFS2pJ(>*>8or%u8s${I8EIM-E%L%`djIRVB;J9}{C9=3(mvE`$H)ipmO0IhOH|i)7Q%eH ztXB%tdA)I6MsBlT+)eYOyv*&7Elgbof}0OLxcIY3$CZON&n&ar0}964&(9j-xv7>A zB$^|$T^cPH`^g(hNoGAiuI)bb>zTRb*S#fk9t_`KZWV&Z;+M5~ z@!JezJW9pb$iM)Ws~sy}c6R>Seyn8--Y@IpEj1x5_V(66y^~#0WEUevd@h$T^+qlC z&W#L&PlhZ%a_eW?&I(<#d^~u2bnp>5AE%M_00&&Pzph|HX!PxEs~r2@u+iQEsSNWv z-f-8Ja~nG5xeot2`1GLnierzq#VSY1tyc`WGJoyPNG#|`!Q^OxQU2D+;bJM98dY_p zwg;C2w^(Yy&T92t$KPbs>525+Tafm6=W?C_tpMHQniHeVlg4ognWbmgU4|0|7ISM{ z)IDCmuC>^^vMVM$43o@Pr{`!%qX^JDxhX*_gEL(0z`e6?vOB}Nn(`bIv5;eUmWOWB zrt@1ir0c*g7S-nU-Cu6|aC7mYybrZy+h(l1oN4*2BQ7bwv|!>x64}_P@8^^T=(_}# z`f&w>50pDj)~M^Z!)i-k{CR5N-Q9Ve_*tR#{)K?yK&g+ft~Z|SPRPbs0!Acx1g%@{ z?Z4U9YpYW~-dov)H3ZlI$4%m*3{}P6^3b< zEW^C2ORA}<`P2HT#;NIU+qA?E*4G}NR#6&i-xPRvp48Zk+q_Kg7Vr5f0=iuj8-8($ zE7x^mq;ufab#Gq&ut=x6NS@56*9!Z1b#o4316A79=>~Ks6-buUi5ABuQ;sTKqxt>pr5EM-@&uhn2g@(*S`~QvZ4Q?#KQ^llwy{Y6 zgSEDI=WN|CZ1N~#+X(LF-xZ|q{6SSwa-@ykP;VS;X>*^=BvK)GOQ4wj5n6j#Gha3) zeBgHTIB)Sv`$qnRk!lqszurW(IJE?;(5+6x@5D#nq_J(W&=`Df-^@`mG1?nmo1op9 zJ5W)E=hl=QO0QBZQ*!C|^;{vsuX@vIe3vZjt-=ahC*;@@S<`2kmOgFuCgJ6qi&yyK zncJ*fN40CU%oAqT<&O22+4dDJ;qx)F9-5e(=$%aMPQ2am=M46Tc)aX;l}@8QSut{c zob{Gj&;5n1tZ%BuWj1!cyE|a3m1;DSIF@K!9^@A!WbV>@R_X@!d6e$$+}P3T!wBpJ13ck!JpL|o>|Fn@iaMi;$wGv9Um7Le24DhK>dvil?zhd#h>b=TGt&T&Fj`&*I&Eb?*bLTFP!s9$AVyu$UW#Z_S#c4ey?oRel-N+>t-}wyYLzdH1&7 z3tO8Wz_5Hz&siRQ^22M>Cy}yqg;2-{yMj;*}~filNo43pG*B|m#Am?w9TCzTSsL64OOb=(=&fRXIUGpy&@*u zCFfq_Jn?tZFaSM=cEA3V+r9Z>?%S^x+Fo(Wt7@rqofzmJYR--M_4Mo#m(iX8j`}0F zdz7(?8yEH?IIC$KeE;fky;)oW-7wvvo*%AQ_dxzSHnHOt#~VyzsuOj^GR$h^Xz1|N zeEe4TJ=sLsFRMI3^xhw*#-k7f$qNOqBn*t0g6{^$mO*vTHLA$@^rs|Sl`(VpVS-6-gC8^Nx zv9=AeKKriki~?>TV;ITxmi)MJ!TLQ9`ij{Lp4tH{`qP!vb?sEiPAzi#?#n7qRB|{B zly6Pz%^&SJyVh%WZ&RKve(nUAR4DdZS?1d8UsEtS-g~U4L5Si8n@PwexcQEc-APJA zZ=Vk!8X0vbWGVVMA%=D}WSHwAl&x?Xeo3gXA9wEjZN{nx*xBI}vjatJjgRDXMXxyV z;x`J21I?2YTV1iHRdMP`xqyCMkFMUeF&f=6IjSpy4O4wnmCO0#ds$$U{^3^d7}v=O zx`fN1Ab2;7EA;U7PcNAihuvX27#;*m}5iVjYEv?ACdRn})x{TtKj8~X^A6OzJqki{HV zMac)eNiE|=sP0j&11UggC%}U*|5mQ}z@6#y+O8b*D#7z#M6_+klk7oM;D-aD&yde2 z)ImU)x76e#@K;rWc9zWj%Q1t9g9N&XG~R{NXjo1LBTWzy!iCt=+}TKT*)+JoKF4t?A=yGVXl@*~wm! z5T(Md{Jn>In_bU!^Q6Dc={5#zZNFW^gHT0=2s=!xj#b$}8>82Ca+I*=qT_3BzxaKY z4GfCUx(5q7_OY&@;`lSm06^i(`mG2iittBON%{rAG&ZqpCY532WBr=T0D{FfgdS4{ z6m>Q7RoGGj$8UrEVocxDF2TSzjN}Ms5{@~~mH2?*C5OnsLxHM@ZKpg}N#x{Oq#?U8 zKC|-pjlDiAU`1rEjhID3`1>UDpC5l_1R(Yh&&KKLS|!>YPy5ygtJZFmj@@Sz?mSq6 zU-BCk{*i9^EHgK^7TcyMu`1Q5jP?*cbY`>5XwlnDA)sCH@%L5j=7cFs8`4dWWVidb zH`%4=7q|o>-tY8Y73$BUywCy(w>cV>ra+S3^n?~y~P!d zz&p$4Pp~P3iDwrMxVBx|6^+|xV<88Mr6#PiQ+d+UdrQT7a|MLVsyBvD;=`f@*=>6= z>WuHWO^a1a&>BhEq!NAT-HSg(5E9CY-(pxaKsMo{28vSS?>R~cBVu~qinS~Fk83vA zR!Yz+^go{#5V`Ht_ad+!`Z>N};^U$YK6%3)fUXiU)@jqfzue0mz=3dLzopq}fC-4P z^a0(YMlS^hu$wsJr_sJi){0Aizlphn7+g*CQ8?x(pe0I3nvg2!9IrPb*yhUt0Jg`6 z5?L<6;=+}clqt=JS3;)C6gTPWiAHrpf&$^13vXe1S>+hvWex<@g;EtD#- zd|TK$%XRUr-kPl$03z5BT&_HgY`wwV6ipr8Ki)O)V6Ar>c}e)L)PV}I&JT6^`YoNq z05U#Ky5D|`c8`kNwaS45&OxhXnTKulzFmGxN_>*aR4DjioqRbQ;K}+adqPH-aJ1=% z4I#$~nQt|@3l;p0JcKZp&_V-Z)I`TZngGY)t4m(42V=iJe zAJWH71zXF^CdNkCL~ITXHriJ9x9sb;(;Qe*Ff0Q@*PVFN3@1`=kyg%mxn{A}Ft=)= zPWG5dCYXyo;gS-RDS`L6jLgy@TdH%r{Al8sv1SiKla=Jea4T&sai>JTfBb=4#;>~=s z3W68EUGoFe1mn2Grp%JTt!04w43_+7?^36prNNYcr+IXU=7)6fFkgCf#Kf<29xYx!~US@y|V zH$c#I!O^=+jy89CiM`#8rA-4p(TU)YxVQAxOXmwU2U)|1)u(whMmhT_Zv3B z^`-50c2}>=7e-Ec7NCHNr~V|M=PJC z-pWG9w0KGEvxmrXQihIFj7OB}M~)VSWuc3t(oO8jVUa>)G3)nSxd@=`=RMiW`&-#T z(`LWPiJ=1WRpKz7_U~@HPYgL)gGNi*`^TPqmz~Rm%+h!NY-c@Wr0GodasszdR#VG- z+U(8|3!2BLebhyB`1$+tZ@8I;67ThyQSjT28D6}J-&qBTwJ*oySf>=%o9J|5fbY6B zx4$2^D(PZ6Zn@OMBBoctEo?;+C}qIlmu9h;JM4hY0m zk+Qz)5=YC!i?;G~3#4((vR4}Iv2qOtqsM)`qUHF_QGG7{)-26rgKe+tlER4^ODXQI ziOae-kOg9cgBg6u4{2MZ$kgiCmVb_gl3*VxEm)%O)HhVBdEPON`v|g`71+4-2rsrD zyJBKo#y`fwhM9XTddz1O>2yQHIgU!#Ekz5O)N-n;+OO)tCq_PyvqbhpfEGaZQ4Zl3otfdhL zC+e+TE-NkVr)YexJ+Hmy>aS-gkHC0tB>UD}*?dqL9xNT6 z#7WO5ceur4k=!wMr+3=`ohREsFv$5(s^4yuH`MGZ1wv|KsA3qwf#-N}rj=Q3ye7E? zS>N?MgTr7OgA}$t7aafifiP(>F?QBRt-n*&KW?aDxsTmY(%jr8fY^L zDKDFl+5RS^fc?$SF!U&;a7iMgIx54%@gt9QD7#tKO^V@oyGP?0j=*0Gx5FOy>$NrF zdP}sQjzzz`dnN`1Cy<{br_*VK%A{L=7QBS*Kcq|uVK}DHc|5Z@u&CbP)SQ0A!gl!8 zq`eJVjn0}#Hn^NzdD4$=C~)`h)cknWrHLM=w6v5EK!5L@b(;RY0q#Q%*uZ***-VUK zpt{$zOJxdU$R&ZKWrE}?g{|!%W3=0b8?$X~!6vL5IT@22CgG@dg2#hcJxbGvqDVKZ z;gsb!D!rTwY~_*HbkJMEAGx7(_YsSdx*~VVX4=J;_fQf7UgG0WZ#dcg^_>?=iFM_m z7s2R+!!b;j$ZiNt-xC;6gc}FL`FDglzTLz=icCiU;g>9rP?=98oEq_TM{d15JyVQB z7rtcE!kP78U&@N<`p7HDS?gImws1I}brNn&0MODRv)))fL>+|4vDej`fDR70_XP@= z5LbKimrS7kXe5ewp@P42cng5TZa+>{8+a+zr3C;`{wI$h{Y!g!dU=O^!RYe?y;Zx| zo!{pTfO^}|jU5h}Onvy_+MS*cz&-gBuUt}BuzaQHu?N^lT32@y!O7bn(u)8aDCO$H zoAc_LOyUsi)d-x7e0dE5C%ImSm+nhSXPMLIKsox+JOiHY#;pTx-KSCmN7=Ew-s?8) zr9|@C0Ku8`x0STFQocZpo@HkJdWyk5NWCT5q+FcdBH5Fv)f|4q*O<{ATrdmdMG5>! z)^@48lX|wG46+9s%mW-Q;KL$>Dcn1qfk_A^!(HNOjgg=&3Tp)w(1|D=`-_!BSQPSZ zTldL$-$N)KDIy}<{`uV!?tW)g(9KGcBOE3D&IP%k0k9@rjoAa|=gec;7MI9p@DS&T ztxhQgv529_Ab1F;(bEx-quhAl-B8N#$mt{+wId$WgQXNw>THf8CVgCZz_MN^`9s8V9+Ay{UA z!0|}r7}>9`dB6tzbX-Q-MYi&@h=%L1@WGDD3h;qsb2k9pv zHBv$#Rs+oYXaONDhYvOCX@KYLD3+9?XhJizUt=JvVBC^I+Z%8o%oo^xCQ^Dm! zoWAN)p&Q`a2(8-`{;(kuFhS&1#HlB!Cp-)l+e(}f@RC=2%Kc@gPm^C}HH{Lnh}CBk zKXDYrjO0CHO0>&FK*96dc0BCwe?*iG{*xq}rj>OhG%5chCwXa3^~4ZoY`R!;UIirl zHcwt;dZU1LdJDE|@br1&zkDWnFln8aRTMXcJC>{qIE0E!$(CmpzDO>_DXqp< z2*t(KH`Z8!f1-rTmfqHQ%bF}*8QT{8%yROjiMjoyGuPcO>H@n$GlH5NQ1TYpd|uV@ z_c2oCYrVyNP$Dt8AXKZ-vJ6{kH`yqS3VT8x7wU&UC&V?m%HE=niWJz&g3m0{6ucw` zp8_7QtJ8ONCK~Ox=`;K*#gOp{psw%+ZcAkB-Q6lsPK3b848NbZDx`eNGj0G6^S5x= zRCU6Qbct@5y-HlTry)c8^^xD3YhZ{~M7wo8zim%A0%L9j{3ugh{T#I76F|BU5cgCk z#X|*|{wY*k}MI#M}RZnW-Z}ukM9B{>aWE5*-{N`zjtPefrl3Fr!Dsq5pKbe^G z`X+HQ^+<^Yz@>peh%OA8CBJ2f&e3g(phc^Zmmym@j$s!(_~4Bi>an+tBI-KUGN?fK zg9(TPM^=w0O@mp1njd{%@S$mgES5P9TxbU@ULIt-|HnXAkii{PYr_V(cxs-cuxu(| zUliEX5pp85bA-%mna&3!P8-#d{QHjZlI{3o20K7oJ!}Rhrx8$~rJVw@-VaLbz^{*?rdS+D+T?I^hIlOit9JTAt@-+7ea9TfxD1?c#Y^MXsR=*emh z!`K1xu=M-@Z6WaS+3>u!_$||t;LnMJZ0im$zPw4r2I0IDnUL)5<2@7?Oi+&TXzV$4=c!r-J7lg0MFVVRM}SrG#^mnnod(Le9zI82 zBXI21BPxRs%}TeRDtd)j?^s2p^5n!g&{*%G7k7Re0XY|poqODUVXaRa7FrHDscq3B z^~(YF!CNeYRvv%)GbJ)UMw2c~8XR?n_J;Dg%t+`GKhvmepstIFz8J}k6xe`CBN5eU z0jh57yIxeWT+-uO_0v4E8mPbPW}WKR=NTn-^cj3MV5#K zL^u*_M)^XNSK1Rc!NYsMpS$7;fqCM&>{c*GurXFnKAc$>VcZ$ohw72|`$xYbUSmrr zREQ5eB$f+lmp!p|#Jq-z*(-t(89=fAog+2;+C^{sMX@E3J0|jjK>d{qG z#E~CV5|_b<{p-_ACL`){!BFyJ*S^86V-XsO3zQn~Qlj8M?vq_`Rz@2|A8hxhm@jXRs5Y($34{;!pR|u=#GjaH&KNt4+jEffTErrN z+nDvPK;S!rqPYF9zj{Q*m@r1d`qJKoxCralLK#2h87wcjwQt7$e;B+qsd}@<5Bn3e zgg`p++RbLt|m#J z+~@wiTIGIzenv?4C^Gs)f*e_=_^tG{WD-7p`ntnu06^PrlmTubtJ+#a7W-$dEa^9@ zaTOYn^%@4JpXlfJ)4$y1Fkl}JvJ2~<)V9nDKj&WgZ;Ys@udjQ3@5alb-*B;2etV}> zhV_9ME|ED9k-*JlS8rx$T=;6nx8 zeGh0z75cShgCJGtuzuNlIx7POHV~vi#(vAX)n9msO39~}AGrdGCK7J09_bhD+BHry z8FI0IiF=q&cgMbFEPjUTqi%)#V*%|!1>l4ZQ3imxN_0F$WN(Q<4d^d?7hQg5j!dw8 z8HyZ@oybo&Ilf8RMOX>BYdffyhbRFAfF|pNL)3XaLQFiBFoBNw>?l*ReY9-`rNe=e z&L=CRCdUZ!1`3)gBO>QE)=$$AdHU6iskve84`FGtQKd;TOJUkN7yh1$lwcC z-?)&{F{7B!<8l|42|dAxNmo#)Mbb4ww}d>9H;W84+6tgD+TTjGf6-&FE(RaT*~Fuj zz8el^NpKf-64keDFvZ-1!|<@72fakrVJgU7kxXp`^o1 zMN%8XR@hbV>w=nu0wF{(Yo>oc`vI_202QRGMi}uSUzw`GcQ@dufEoP29`XT^Yqo&q zmDma>r9?@|skx#XXRJ6XO%=laR>Ecw$oWvQ2&sk95x`v?>!L-AYEPj4NYXX%0dGsN z=!>)OBM%BBXblo|*Idz85k?rHn-vJk*{DtsjRMg}FvJWV6yzyJ z1Z}a*Y3!Vy-2asr)@@M)us$3Aq1{JN$Ocl_icnvI6@hAdr_{94dQwRuwYqOZ;-!lCPWy1^LE{);RDZ&o;hL1%SrbvywoJX|#013Z< zIlGn9*iir_!2n6ONVc#Q)$yzf9@O>-%ji#jtm~?dtx~R=h$g99?$fQHiivj8Ja|FK z-+T^3jrS{Gl>9s8zf{iar#z1eRM?Gyv}O=f#Qs7ohez&Y$6Rxu2KAg{T|Oi-0k_aq zm!wY-WM(H(js!2)u$`XA9;TuwvJU0rSIYKs+OMck25LKm>HsFqKJ359$lg;|BUl#8cyTIG*)V6b{e@b6_mKb|U~ z<$#399BJ!Rf;*Ec6q*=pYNw|I^`?J+$dK8k#eo@DHy1N2cr~BW&1Jp#H-51v*$zlaut337 zza*xGxW~=1Pf)+l#TI4i6nU~5`B4XYVbi<47Ga3ME0x*^7NDZXbG!*Fa~WoDT*i~d zEdKxqicxw&MuZG^gRI_(m+mWU9{fzzs#GYjVqn&bYRMJ&Bzf3S4VzhIBbws$+?7EI znGKu}mmbG9NF*mo%gCfS0p1nh=crc3S7$QapnWQ=y^|1<>i$#a{(Ai!`5%0yR)Uet zPnakZ`8v4_BK-N`0V#5zq;`nZ4Kr=|ZoT+@Y^2h)0vti`&I`+!Myp)q4@Y?a0OE0(dQ0TgoseNb`W$9 zrLpa{q;msZY7B2s=G22q1L;5mp=|MqMEoF%l)xgSXM&I)?1pfFjE2B|*@tX~M)pF$ zpi;S&-!RS_0*7I?6`6JMMBUrf)(qBw8h?Uhz~JJE-Q15n)yc`))%fn~-D7w1qM$jQl@w zb=YaP_vY_v!ycNPCiUIt0pQYLq(Wf-8L2VDl;Rn{^R!n;p$C?nDb$*h z=^P&kp^H*2hvb_bBwY-1`PfBqOR9Po%as}DBWMpLzkUXdUuu=27|);v#!9P#RWQPi z+3iEXpQL_xk^qvFB;^fit%3_7$w}T*Uus3lQL52LV$h4cP*+?>YOs-zBC1d*V~qXk$Y?x5R#~B{yd)%9BHan0YfS8D#NY{r#ewpBW2)7 zp-B!-axn6E@*$H`x_j>aAdu=mRXRd9{sd+M>z05Lw*1a-(*r)VGLnC~p$vdNb}QrX z<25Ap0`60ra5W~BkK~XvFtR?x9taped_X0BFed9|Nmxet2dzNtwNJutj6Cl5f$p(8 zb)y#9(Mm{;5cOqYl>^V3C$<1QnG)SewRyx*9h44}fB$`P9YraEAOSiyvLr`qgpa2R z73GK~ke6t-WG~pD7@xd(8B2M5$q!A$_P7?K2Q)>ImSNwuXTFiC<+Xs0xf0XEWSgKsmLOQuL9hB-&M5%ThVDG z7ZruAgnJ3Y!Bd)mZm&cLDf8xQ{GkU$BSk{+sBf;on~2q?Oq+lNtT7v4i%Z89);v;^ z=#^@hf1)!%ccl zeMJ_)1q8r+IY4WaVB}#Fs9{T-E#TIzA8`o1{!o0xbh!M<1cm z0*UFKT;JtQJS<(%_H}FwDG)|rQB*tNwJCQe5AzI1QbZ}Yylix zx+%_W08`a@{_K~OckfRCV#bIQMB1;bfK?RtqeLb^DLVX+F9!s=3Xp0Ic)O0R)jpDK z1Vms-%ST*5I=UW!oOMa(PMjs!DmhR|Hbh*X+Yt5ua*|N2M5p@_y$6^If^Qrpy^wu3 z!c5t~7I}%jzCi*(WNXC31zfR)rz8@?Jq6zK5PoPrsffB8KL|Q1fdO@upe%ZsxN?$1 zV@~rwh$2zaXcL{~od9-8{^1YOT?j#6mix}kWL2%FJY+P*_H)K;gl` zFag0{6S+)jr{Lj7tC3!V#_;3xgy-UsB@G72A~gZJxpaC-9Rf%6Ayld^)eIK{f>4>5 zh>;pMWeS@BWGH)48~!kD51AJ3E0df0cF%rjclpyU3 zBYcBATIWy#!V7O*2Ek!CakZtv5X~st8;|OfYyc!uUSpl+iP>&``cY9zEYHdCuOlfN z9CJJMpit^0<5h_gyDX`vI>Eb?0Tipdyg}Kb+>=d!CZX49oDvL`=smJnyHqrusGb0` zc~aH~g^0CKG3me70+Cihp4#vYRFevw6`=uKRlvJXkZ!Ki>qJ4c0?Y%aP7LOYYPkX3 zT8zt|Nvx?=h+pkGe3zYwRf~)owQRV>0g%OQSdcPRbzN1x!Pfv86irBV>j11a0Qc5* z@z=Q}RF453ckVPFdQ&ezBO#}W^wt=}p##c6wn!`5ik9f_@j;)LI4Kq}G}5d>+Ardv z+a2kvSW4aj3nxxGS_)*)DQqSWI}!rV*@4?^a+Ny#D1s0^>1YS2!Ub?{Ly?CxKkc?e z_(8XReQcc1rO$)PSxjn^a5N=jRNa9G?i6dD!En*|TUJLRcZkny@F=%Kpe`W2c`D^i zz_ex5n8JY=K^$rtsM8q8%?WX7Wm-_{Nw=f!ZL|s?pOgdOWVT`P~>i*tN+%eR~-3I$yCM8eORA2+8uB*T(;}!O$q}U7*IW zCu6VHUv%)4S^GzW{iX&1q9{3t3IIEQPGz4QWSo>Z(91*%;$2Z*-~OEwBnnNoRNnj| zX~TG2w=gzJeC09Jen6KZ9KRf}2ysKo#fIzlW|*mp#Z8Mp^!0F~K)7ql{=iUE{y(5} z|F^gJuh8&+(Pw?0lO^XGJ0bYDyms$Qqz`{xAK(0Mr}mfGh}d}Z@)yqf-CBE>&>~Og z!MmOG76bRRyHkaa&mWNans8!B@B50BW-$Jh%%`lD_3MB3C6Wuv!=a?qCO!9l&=ZE8KGoI znwqjs>aQWqJC*T~ryT-fX7(pQy86@lFG!NX@ZUwnlo!Rl)Qd%r4DdHYS^$NiL#x_&Ccr4eC`nkr%j(5yu5+JE24p zC~7F@ko8#1LtXFU$Sd260h%Zij|9~$by$t%3NLQc?&NAjGoL>wN@3`U<&hK!fGZvY zb5n`@C5&*)=UkWASfKI;AoqjO`)&f&61U|EmJ1pL0;4Av>Vk=<;)gBL4pO$8LObIH z&KlM10w-pp5=r7o_d)&Zq!dFB@KF>&^$Ocqu6BoN0tW8d0P*}la?otN((Xj1PzSsw zp&A-9@Gkdr;Z z_9Xz1bWH&^uYw9S6|0CT89-6)vgjI?&n3@P)|jqwfl3clJeeys_Ku1bl(bWoOB~H| z+0bmJB17yz>flHqpUOm}vy@x;$ssVWr{TVz5_uQl=`-d&q!S~kiP4Xfr$Ljtc6~WW z2|%s=)J{sRXfCg=u#Dplr<~y&10g1VP?bw<2sj&{Yo;PAFd2Kwos&105f=r=XM#;P z^1ez*ekCiBt099O8Xrxq?PGo3u6xkvVToM7e6hC0B_P`lq^jJO!YsEn@QY0TJ~?O^ z(t1dL(l|=!*{JoK@-FJhL9P~auZ!gqZmO3Gm)$XY4ZWWtvVUI=5?WE~DiqYBR2Ct| z&0m=HxSV88nqvhaBb{$R=${8pnFf9(hc- z2GA@5%${`W@cPG5q5YVt^a3HzM^!}R>_GXreg)F(pJG^NQ%we8G%f|Kv;CzMlwJZ> zPc@%RJ?p!J@aW0QvR~hf^LKa)UgXv{zAUrAklIO~zQmPy>ssL`HGG1ilVcK6AVPPO z*%i1rf0Y{;WKTJCVI;hquDW2$&~*7L6=jKCB55KidMUGCqI+y#>~&T~I&L)?cQtut z$cRO$tB0LBI}&fToCkT8^hXKH*xXe8m@9-7`XLzPj0eL|h6jlam?(y*GA)Y=f0P8Fj4g+dMdT3C1H}GN!=7^SS2Mhg zk}tO>H{klIBZ$rkARW37v3<1L&iQ_|=$!PBI$%frs#zb04pP!kAlO6=>LmMyK*b2> zt1R-8LXB~q9E%Er5q}dwK~0645sCUFFheb#Nb+LWeswcp=8S>`>_j`ZYY9_cu#x4q zcf5B*0T@LH)@mHQV3GcKC#xg|g!owiTkJGs;X2R=qYahIZXwIZU+iMaaP}?(dsuLhLdgm6fzANq~)tsbFVR1{C!cuG|q9SoGDyC3Ph<0I=7b`4`|D8FH zMB}0<*RKGwTWap6h)(5bVovGV|9Qc0fh{5_D_U#voS5xTRA2;M(uv|Dm+sdN=+GH5 zZJc@Y1Zpjr`y_HlR_?c&)w?I(L|3ndf*tA%sPO7R$fhef*dCzYL1#UH-TJ$&GAUKL z)>5t{0=+oU!IxC|vOb{1W58OFicl_uwGOATh-uziK=hFOHyh+WsarTo&_#WPYF#91 zv;L9@A@@-w&$%oPWA#Fm_#_4UWC1Q5Or=khHKfIDpuzPHjp6?l;GNZe-@#%fcWx?@ z*SCY#Fa~?V!xJrHlBl=wZexv62C6Z%{;1Q)At^|IZ~`ixexNu1f0`p96k{u)ms`_{ z|L=iTM(1yLOr(O`-epv+eWecvX>yoMJqWLLieDMSdEoElAW^9Gm=#0_ zR}@flA~6q>)<^N0j`O&t--~ALO2pU@YOunTJ)o8tPs38C+nV2^PCDW z3W5CV;Chw7)R8Zv;{V=_3qLHQ_gvHKhJ8_uGnJZ|P|l&^ zi7hfdfA;WyubOG4YY&=~+VS0&e4aS`yH}Ly&PhN5;8PA!XfUZ7y{Qw9YArg|=6PT+ zvq4aj5P^UV^YkV<{s?@bLjMO|D_PotGdmHeBgUS#khqqTvynLXTbXj!)9tv!r>2k2 zw(%D4-%X`g!(7~#1Q_Q}CO~vcN{^9pD;Hl5rXq*D3DXl&d1cPl#*5IZ{EQ`P%omyiaK_$8GmKE%uP~FdUo^= zNRYOb$~si=payNKk3~VCD3e!9r(fWett5rU?XG~cr2}H9+oQBI#QHuPoho93qA2wS zfo`_!hjo&FlqqKg3h}~BpoD@(8x;8?+Y&P%@-c=$f4sI8bSL`k40&;e1Utu6|IO3{ z?R6qr)9uL#l5fdo+3ZT~^;q$X#wrkl8VZ)EN76`0d4q@jnW_gqZGh-XRB!uR7^!FabFeLu zq;LbQCW?j{^JRU207xOm{kNxJ#=rd-O+U5mT9-pxxM~qoXN)Cl!YliL$f& zMk9}TJ1POSqi7;WBm;tbQ3aif0RtSOJfD0UwH~tzuiI-;UU~B9rIeC@x5~eBe7Vpa z?@&UFbOhf+QsJRID~0|gO=|oG6F??GxdxpV&;bl8kFpb8aRq=!lou;XjNeYEVJ&sI zqF*VL>e|Q=%82iP@TP+*aX3pL0J=_bv|t@XpL7gp@a$@976W%v^`ybvKwzbs1(jU= zMd{4Z&tLD>D#v2sS-*y@ztR`}UsJdm7XVZ^NzE^6|2BlWCuB*OZ-hG1Iz?7U7Jr@z zkMMx@m>fSir$cB5fum6LX@g>+jE?#Gr*tD0Enw6WN|GiOY7wI2&d64I4f$Cf$Uqeu zU_^X?Is}f)2Pc5H`B9-}m;90CUCE_PR77p`B+OgW3aqJ!xGaB>RW+_r!-O^jQbKU> z7#)xUGz{t4DUzp&;GhN@94mKOlr?%K&%8;hkIVP5EBwfr;}{33j$knU2)yldwi%8^ z*!%hf%W&>+?k3|`g?^R_d@z{GV5|HdzkgWp;&;WgC7LT#O-cf%Q1)tA0pl;Nft z!pw|FuS5ySv92dk)%#fAz~@dV~Y1bS#W>5#Rsguu0xR>;U;KVlbI{^lw=%8{fjTHpqx9NNqq^ z7J_w_H;nDxPh_Fob%4kb2xH{`ao9LBk#-a{-gF?`lz>F}DpE51)u-YaX+t;&ws3j| zFcHxqNZX^%vN8?vh0GyQTVCv1mXR=|_?b(Ao|VkM5j1QB?jCR@<@zXWspp_38GN65 zFPpj=T_X0gT;z^7iUN1cA`!2N)}_vNFr_$vEiyjd;*tY`4yxRI+qScO#6UF~J7yV4 zzac77pXg=z9ixZ|)v1kH8RvUat2lixw~`g(4qu9;PK2@K($a~7!M73*OH-TJCr$3Z zbh?~&_N)VkiVAm8gW37B&q6mG_YaERXKWbshPx~IasKVd)O7Jkzb7S6;>8z6Z8{(G z=*Mko31>N037f87eQo;vfzj5{gZVA9)50>s9E~>%B-PoK+lf@p?sjfA9|;`Q97>Zk zH>z31x}1*`O~0;4tv;7yHjm*}I|NCbFb%MYZh5TnJ8g??yuzJySfs7jXfP}c{=;(%DS*NdI(;Rcg%02%7 zG|E>7klg)*le^*ve_{EI+wmKT4bvGBb^zE>f4^Edt1yzmkpAhbK{6P-xBsoGBzqXc zj-VTXlh*m$u-D?#qBrQo34#j35c;&@G*YALx;k~o-7G_rULIpOC*l+x=m&YU7y&C` zABci91izZBs52dUhd=r#ad!*u|izGu;in9^`}I>Rlqt5EVn z!C%8{5TU7liXuo00;U&OM_xFY+@}aMt%%N^1n1C$`y$bh~}X76h7*0GZ8W^)!b!yuK;HVB9l?K*Z$z{UI$sNR zfbZBUCJBX_M(zn~zn!dem2Skmbi*y{gLF!j*|9OQRj7+p4R^52khC^)D=dS;pU!of z(_B|weGkIJgV1jE^`c3%X8BTJ&qc+H{t3mYtqU=X)&S z^P=bF36cQKcq2CUT&43{+d0LgLB0YELw%0$T5J#@dbgJaWW8c#fZWnQtuJJ+eP9-f zkQM!igO5(c>)zf173hz2I2`z@gd!W1LswG1^M+2ATPcP1zynYPTH^=_8Dx{p9YZG) zu5xJh;%n;ab~!sc-yOn_vXaIu)4f`Xd;pQXx9f-KS*gef z8)iU**jP=-6&4ii+zjV)3oAx^W_*Pk>%$b9#gT9929Iq53N^anF3WEj4b#Yy9szwG zfPdW#J@ZmXqNy0R87;0?a0)=Yc2*+Ap)X$zH>RLQ1{6?BaSk81Uaqhr>Qzf|GSQA5 zWAF}E#J5?4=(!Kz%G{|WD7OaY-|ukm%Y_)y+E;0&M;hw#7efW;!agr7st-!6Zn`SbUw zs4O7fkZg4`xOGl%)>eYU9%OmaIA=kHWE2TTg<*Z&uTV7jo+7Zo(Q_g&TUmD++I zUU7PUi&r-mZTK(GE9(_kK6G~u63)!|FxB17(Fv%)`t^@cn)MC;EzkU);`wS-Kt_PZ zi?c=rtu5@jy}#Jzt6!Y{CI}<+|J9J0$r`r+i31viDwJ`V{0 zMj&_tpY$~2Y?1u>A>uI(fM4GXMV2t=wS#E3WhP%Z;=N255M|Ap1CUE?#t|JSa8h{- zI)>x#C$o4}H~tYAuGVu97;D#DL!D{h_wR9Y108Vdh!#yZZ?V~8TW(|(CWyWC$=#LEE{tGH*(&_UA6}4<`eR{ZK zd%s#D>EL6A;p@E;j*MF~82f*N zYpJ%&d=4z>hVTl9h3McLv}L=VMTwXXJxsMnosfW2imU-T)Hpmf(`lrCDNf*6Y?Xxb zCm*9WaV=rdRpKpz&K62a%3vZ3u#;zUr zq5|}mWn^S*hGzdpQj#}Lk>aN#gVa`S`ZW9W&t3|zS~~tl>s;yB8*4P@-Nymu^)uJ9 z0wt~O1m{`5q(>wEf9Z#qDINyO$zn+i#$-SzC^vjjz;H{b`bRQ?;lM(MxtaBE3dIM~ z*JjYxWBuwmrNoRAho|q<`6cghii~dDFMmNfH{<`ln6NtQU!$1YdNMH9FQ0dx5stL% zSG$Y(CX4caiF#85&Ro`%@TKI_u;pL4GnTkxU~c_opJJy1Onx4L^diq&1}Co{^vF>& zTum?jz+`RcyU$Vp>}J3VA^jmSs2A7PU_62^XEg$TavGMY<(i z_yooe7w4Y%({&x!d5K z0qvPzqpMfK^d{+D2GhTorJEXg9XJ20+n6)|RI9$=TZUWe$YoB<3*Xc&8IQi@yYtBf zyY>Hr^)U^(A#lCach}4peg6$3(%vwfXNCUroYsxL1D{C@H?O<@0+^?^J;Ti_I#$V| zu~aHyKIYAB&}wRusxxLzzdD|_OKa8862{*3Q-eufWV3G-BmN9SdYj&UX82V7E8mES zG7ZAKuRYrIEOLGGMWavDsB||o!I3`G^84yf3u|E{C^237_BER42zR0S)iod5?iU=s z?sv`Nh04X#iZ|Q}@;<$l5&W3aBkA_s-Ruogc5^MF1nUdcO0%cmnQXOM!&n)bL9R-a zn>oCTSCnd|8cA&qhWMjVm4*^9bAZ>*#?3 zGMK~LX!A#i@YW)Mo-f?{om*sXm%K}YwZ4NmV@qf6f#TN9d^cj9egDmI16^S@4fNfl zZOUARTZgc|#dO>@&h)j_fgNHWZ@=pOVaFc>>=Blxe6DE?W#ae%+OL*Yi!vC7z3>oP zr)Xa_Bo6ks{65G(&u#yh?wb?h9UPDJ^b8*9bxi9yZ#VNMpY_Vj?hm!i+=m|}ny-Ay zZ~n7j(wy3}TenSL9lPb$;%}t2qhKJ1r5egkZ+8rwk*;d?BP>t)*~{KV>0-9S3RVI>9A<}HpuU8`WX9o`+@*`KxRznC^aW04oQ%D5JLf8x zoveHbLw%KRw)JN1p6KthBv&n97;wQ~xFy^Nue%hwhV!H|rFQmPI4G$FeeRwR9T!6$ zbW?)4p0r}3-;}d2j12VjZ<&(<7j-QY6(2W+{L$yV_i@Bo2?#oYme5`&6FR!*ah~JQ zEohH!N(jC3HD_i0QY2)}d{!uAP;I8E;$q)nUg=VDGDB!~nSm3R=}$>{ui$=BW8 z%k%Z~K2GK@e*~1Nbv)T((U#gI^J5kY3bn&aW{pj*2A9aTiC>Fr7yjixbnJy^ZRsWa zDHu=VT=UYE7rZ1M+ih>v4;ed9#MaZEZ&t~c;J21O!hgtuM}!W2`}z8GlXNvNdgZ~z zeeNsR(su>xsRtQp3ujGVogMU0Uzd^j4OIgG_$7&pzlZ+PBdSs6ls=ZYs$(3) zNnBLZIel>X)+5MqjA=sSgjwn<&@q|p zcRW00Q_TNM&xJF4YUTvNkr%du$OJe;G0J|Zi@gN1o2d`L#+4 zBCX-axSWK@Kc9T($Z z=(FenD28@*1)YHG;|a~DWy+uxMD2$LC@rKmaAM72pu6aZ-ab0Cq^_s4T;tYJL;7o= zoADQ@Uf~dj5F9*ZRAbPTVTBECjbn>;K$dmgsyTl%>IfRQk6S+Rgc!2h8VA?O;I9nP zZ?ZctZas+8(&!yeaZnCTB|Q$ELwW$M#Il{%Sc-c%CW~q^DJUct;i!wCQJrgzwwFLg zpMY)X);jEK@`T1r@TNB^A?w47k;b+{mwFcG!50zHOQ))&dP1XgB&8OWv`~vx^f&qF zOcAOK(9m;n2Cz4n^9}XlYu2nn9ZML8fnLJ^$u9{*@nTzNup@tw$hwosN0?JVD@+}Zx{`V-O0V3qXRfr_Zz33yGb+bScf^EVlD38kHnwfOul zU8|Kh+OaU{gE(98eywf?{`0jb!#vNnWHjmQJTF=);C;<5Y{T`#p`G;}n{a*n+9B3+ zpm`<3d0_StzGT(jpU9b50Hzj_hyHhpc+YU@9=@VT$Y1AEJBuX1qPS2?(3 zQ4Ven%gbDb~zHWz(ed)Dj4$pT) zO}$8WnwlkC?r=h9$!_b3xeUQ*0uk!eE^-3ruCO~yvUILx;VJDoHcY%(V zX1WRDJ*bN#C=KTe;ILSo`6!t+w?J!*_Nblr;6+*?Ia3T;Fy9(yCluqTv4wPEh|9D2 z^HI|90OzvBX&eO(xs#)_mhgJwu(@$^BljQ}YQ-@P-bpx|?=gx}huExEPU2M5R>=B2 zu)&COOZ9{vvj_ol75$~077{H`F_`HkAPVwt8%*Q&fV`d;2eG)`!?_j9(71R36|>C{ zp)R1qJ&Lk|V>0mlvnt$v#4?0HA z8fUEE!?2d(F%^9JAk1KT@#yIFZy{4V@$SzV2eHHZ{+#FX%a$XJSGd;pHrT^y+@p%= zE%<;|Iv0eFUa`jC(^_Ayfwm&2SHH5KSHf8l*+;S(J#h{(CqB893gaYa^`hD!zMsdv_j5n;-d`C`e8NRu$63FTI%T|itvWOvdJEnT&%a)ILL$x08m+EK+Er!q z3h-9`s$L3F8Sj`{uY+S<%#B5~%a&Qv1=6W{`l$N>BO-41INH|MyMDTNeGgQfX++}4 zdeeLIbYfOrzL6+yr%r6a!U6pqcl6^`MEBPrXv&({4rF$F)5n6mJVkb+~q?R4(_RA&!W zv8886e2i|%)A|lO&pYv5QB5*@V$ocbFw4%^>AnKTe^{1X zwb6f|z^^OKC@~tZ7!UjsQCx!0-ut&AO0;OvHUyJ4gask2%*BSAbFpRYpNk^!%JBs; z4}XeBY-z0v@EyoQ$ZgJJgvw)t$`c~gOS^RlQ}uTTQacjxTHXvGUO?AaCO%!JdhcKNToynF1zrh~kCG)S%aDqS&8__gp z45u#i?1>T==lP;GP$1b2Vaz0atp>WrkOdi-y)$!``aXn5n9qiwiuxBWx8hgz;jG6h zU=bP#L6$`hbtb`hW~4M{tA6?E{9Q19QX-O7B{y@#?MPv9bC~F$XE4`9DZLwBWG`;p zf0YY4l(2FfJ&QGmFPowJDBKlS24r}Ctj{WhwBcq@%~iN8Yu@K8^V+f{>mddrg4=_$ zYAcesFFD_FgEc1bk8}%ORChW$f*_sQoH^=+S-w_CS#Ces?^^b#`)5V17}KLAj#l(X zEkmsE#j(GYCYV=oMt=(DJnH4l#*{Qq)qi&+d-1LC4&e9HySR1#RV*C{#TaU5q}z(T zQ5vP;b3zE2e+?lm&)jIq)9=Hiu+YllQpQPKnmf;Jskm9ZKnb7Bgy!J$fSf1aq$zK)K-n>Os(*MHCG<@oru7$^7RQslFgKhS2f4Xfj zzA$2O$yjt@yO-x+U&vNZZGy+$%)?l^T5_rh1wzycoW?b|8AWo}XN_kssG~KB%gfa+ zniDPfYS|T_GK~>VaFI5{=T4r)6Y<>gSX2juyRT|2@LM-2${v<&6W96<7x*pH?HiW$ z;b58Mnw+VNY2B)eW4k=wu4Ojyi?%wsK9fnN?r(6l^Nt$+d1I;SX}qX%VUmbvDrU?g z`0N2u*H)*es;5!yVMkFvq;E`74e-zWiWHuE#o;7M4AqjL;)sN0ch|N0-!_wdw9lOZ9}TB}i~C53zv47#NTetfm+&dL;ju2Nwln^xQ|qkY=CBZBa#fI&W(G&_v51So8$`($e|LDItn*O;mUal#&) z-s_0u`e4x9GO27?P^rAytWYSN&{(nTxUj#o7{!B$K(k5HzFb?wasu^iAO;VeEnkEk zWi;8yLRlmdIa;#JVa6VU(7X~_EaTSCYrpbm1`?PMw4qC;?C#icX~27Mrmr9LA4!2w zq6w@`W-#?fqnKnNTG0bX+Z1$rO;|rHR?f7R7^Y)|+jY2GSYMo4sQZ@w-p@KAQmcg6 zg;#*Pk)X_vx=uG><1;`VP#6@U8=l{ECGHt%o3Do}aF$sa2sGMoExOmE_wV25oJ&^F zEv=SarQhYKxG*zHgtV~rYLov13%&RQfO1^ffj}vwV1Y(jo zo#G^+$bj@-mn8v7Qn-|53-1O~4fRUfopG4{G%EU&BDLVqIgvWYrm5U`uAKL6+2o^Y zYvbw6&NwgMA`i|gs^+oS9>r2SH^Ja!k;1pPE(GUIkx-|L9D{6yylmhYE|kZT&ZaY& zHmvgHsEN1V&VP{VTz#(g@j>@u-R!b~zMxgxR3*`#WvF{S_xd9a+~9t&MweHPZ2!_}U1D^0(21)ixk!IF^6Cq7^}UNhdNj)SkcCQA_BY>2jCS+S z-!$bXDHa{tn(6Vlj<2iz4r11ex`Y_@rZGQBNm>W8wLyAO`)x$ADLXu{P&HC4di&gE zx#HH{56s#=8n^Mqd-4lKukg~w7iCDD+zZ(@wkFyz-9=Q@C;O=S-1%l%l}JajdB-J% zS(3#M52`mft*Z`jcB-ppS5%&Bl|I{{FcQ=%)FBJk9YWF_N7B9UN%oSydGlH{n4}4z zGa7g0B3o4MbcEe^t<#9j)&Y~U>?|mSG~B_!5%xH&E;MN;9bGu~tg#tbjwVSMe)QjCk6>c!=L`E2FL-~n=$G~+kY-@Ve?r9E_T%c z!>IT*m$wSHb{sBGDD##E8bG55@Kvbd79t-hlyG>RT+YFrT}oO)q6H~z{N_XZiOy?t{MOWuq-{kZthVTv->%b+u5!xv)HMnC?pCZ%uy`lwQj05 zxr-cwZBjFkJ?q+>2hp*)qNM3b6ei zD5{l14|6?(s1O~+D)UArDHNtzVC$m{^2w`ExsZQ-#679>k-j(5W;cbyeux#L z8R-X%2Lhpfu4tlchceImyUp)_!KgU68pi#LIQ>c0D7W7TzZ;8R(-C|Z%ilwD(h1Ke zVVa}?P};^_yLNHpt`cJC)%Ameic6YZZn_l96IJ-e;pLwu9-FNH&aM}(CFvBJAk^s4 z{DWH!Mq$GJ8)qqS8Uvc9V3$mttgbk(31XBb(wC)iJt|N_+$8@s+ zvsu+_o7-C@|d#A>-j zo`Y&6wgzzvH387a!AJx@zfd#P#}z0W3ex%&FS(G+d2>muWSQvpHGkiqE}T`g$}Zsj z@{9u)ob8M1(&Lx0ZfJg$SdeC?8kzS=aM{^;^A_EEx$_Z6ONL<@Kxp;_gzO2|VOIPO zQwR5Wdw6^60QKLM`Je&L8oiJer6410oE6>kR+<^xF|!D(A~$8;Jh897SD!HNew>NS z&aT6M_p2hv@w<>_AWp(=_)*-0CbIn(Ib}laeBs1(lgzGROvD=yXY0AzJ@wUr=YKFv zgPLCe!6u%6v;KXWa*I!4+Po1J^-fIRS7M~{GHeC{8&ejI9^Y5L)JOVYzOJP)f|VgA z+wTuefNJ8%l|{}2PkJ|y6nKd(-t4+5+*xl0&_K8*TqTN$xoRU0 zV{<|*5H{Q-ey-#28j?a4pTb5rDV|!mj+{Lo#h8&D>TkceXKQLTd=KzrDsDSCLzoMG z!#Udq%PSlDWgJX6@Rwg1CbP;Xc~P@TimSU5JG)XjkIHe435@N%N1voj0U$#UzKF~d z#v1Oy=mN4pjsc}74Fx)@M!mSIg*%xm!)_KC>$=@ZU=O7~vVWrwt8DtvA+AL>9A(xRKZOYlsvn6kp?uzB@oAUf-M6hX#hgBI9yVvZr>W)m8rVi z$3zU!da%AOZEI}6aIc56HqW?E)-PvN#Lcx!R_m}0-k2bs1yfL{bK+xzGJ z=xPbyCDf@eAhSCdI6k!-vGXh&h#J#KPHts|Z}C4|iHi&1xaYbO#h6`8{6!bVSihyG zXwbK1|3#zOK0y1cG#gTfW!2J#2;f-2*s?r;xgo(K*E<;WA;D|UwcT>I09tS#hu1Ok z)%^`Ev1|lRfE4g&wc+3G>C~dd9#$v0>YCyusDbLQ8Q;$qB_G(neY*g?Mfm-}9I**M znl=aGyBMZNE3_{;TIC@F2FHe=KF0KlDEG?d!k)Z@i2B7JAc;pjWgW1E!6`3cS^CDh zyB%nOE}sjK+bx%zn=5}>)cfZBWrkXZOS;gR#?T_0#PN|pVpZITR-RWm(D-KvHcLSGax*f7v4yFjSDfQWY9U$@uKxi$Ky$3fJtb*8^T3wi6z`*uCRv(g7I}&JW ziS9^eL1p88cl&fq&=c_J1rvB29%&_yem&1r(EwJiiE}Q{56o{Ovd(s%=$&ud&v_<+ zqdkojVoFg3Q-hmt?|_%(c4$zo0HDPACnAEYJ6;*iOf)4YUj_!yi#(Z-+7^h_A|AfJ zdN6u%$t02RUZu zb%84W3nn;Ko&XpXt3NtFTP&)G8j+x4+c{_l=F4W~_~@qUbJvnqo?E7P*)eG!Z{=Ar z5Z^zlKrwy$oxy~Hx`qCCy9rgiQiryEdwUi6Gtp{|Ke|?B*2TTQoU#GC zmf2G#>WtqiplnE-bs)G*#ubxAB@yqM-W|)iGM%wz>76UQkFRyX8K&)qaNNSI*T+S2x$|Ix1ees66De2seHkzD;7S_4LMNM8rKK5=u|i zhKKvR?>X(zw|cC;;@A7HZq+RjmGGTcltdwAuO5r;_ZV1BEU*3?0rI(r?bP>f5mNip zKhFE({B_vfAtvFJkD?EF%any}|H(ixSm+S?vgQQ))a-0biA1-m1R_gB7$IEvD*)oG zAU3?hKfo;z3>Ll><3{(NZ$sB>OgdcoSe8`#J~Aq1X7z2SC=V*mo=K$m#{d1EPfqCWI>wR$v#R z(8~>9o0$&M^~nha?j+|rxT(LqwZ1v&YF`kS11V{}PAFVS>^Ks)sn1=Je-+odppOI% zJ%k^SEneYw@*y^MDY>rmG!`{{{gst_+i9#G5ZZE*I*G8%6627C(qkfzX@CnL zPD0F0j(``=ms$L)~#ZwTVD0&!C>`fM3#fnb^pX}Iu->sK`_ENr2$M_G#q@TCIc6jA^!n0Wd& zsn2YO3}9WJ0Pc@m~EvGu~KOOVDbF-6R&X+RvDydM8nb46Vfpyk8vnW2croK$*Yw^!fU0%ge#4Lka*I zz9V5vUV_2rXnR8zmS`WPK%hNwhh!VDG!o=m*8S9E>8-y$3uwZJsiyzpH!XEPeg4X| zuF}poI*Qzw9ouHL1wipY(hnV7{0kG1GAj>Qt~|41pF5}lJivm$BzL@Fx!A-TkRJ+- zcqjuNob9MSN|eyj^VBuq9#3COP#>vA`uxmFU(WuyxSz4KJ!#u750DIqN&5cD8-$&} z+6BnQ?nmE!iN*mH1uGDhq#~tkpxE+7*+5CZrx4x179gU};wb{%AshU#rF;zkD4gO0 z5PFlPz!_U03_y?*Ac7ZEgWf-ZVe{3KPd2X4Bf6)DSx-%QRFau3)-o)y)}^(*?T{h8$87~Egjn==**R&4(jhB z@fzD0?)bv<{lhkAd>q_z;=@Cs2yxDqv~IOfOmlaTF>}AtW21gH$*w`P7^rl<9|Bh> zF95}TcKpr*Nm7g?Oo}q5P#=@yKG=REhAU4HgG3x$wL+;ea!99zGUQgDDIE03GzabRc&_)s%gpe%Zx>0xeH1C%(hO6QAnCz}AZRL%Tc zG1J0GfB_{lJ@}P|9MdZqzR2)p3uY&;e#SG1S_gjCM48Gk8n2`g;uJk)^EkR20?h*9 z)XpbS&LwH}szQ|dsDNirDO(Abg11lj&awGG15S*4qfsD0PMUwv?aQa}U%Ux>X`@2Y zm=|aT1Q3ByP`r#DzKE(ucxC`(a`kYdAPm57W2FOB9TdgG9Z}o9(jFO#krb_U6^JsZ zD8hlPrL2~86of$zgy$cBEKu{aG=9=p+=9Mmv&22<3y6iZTml;jK|$o^0!CraG!O2p zM6eU`?)=tGvC{$a>LQo+**}LHn994=o!*Y^-6-PL9$K0GpE>&I{~-XFRRMreXfdr8 zoxe;*L4XH%(n+RluXr5$F0KcIfWWY5p+ayIuc3h`KqK+T1CTl%wyvntnMPB+G|9zCAoUR515w<^`G9{G#TAxxWkl51ahI z23P)eO5|S^G`{g9|IVo7lf$%^pwe^B#p4rOjPGmzap_-RYL`sPo2piJJ44u1wFhTk znzG1j`?5nYNFW4QjcyZjP!t(IX zIhn{Fy-XXUwNLcqBo*5H?-R3p40m(?*)&omS;lvM{O0ZSGskU~0}vI_q2mBnv>#7S zp4A^(Pu9YY?^xmM$0qZ;uK}3f0|Q1U^fPCmBM@YZNd8L>qE|cLggQ}&MlTdlR-(kh z@EG)4Jy@FwVZN{=Y!4+!l!E)@WY?F|+6Al7b?IwDfQSYg zlsu*Btr0!S>M$~bP$}VTI zU0iWEM|6WxrpoHaalmxmRkh>%K;Sq+>2ALMAy8_1UxiZ7OcfQW;3mM&J-;{n`7>IC zE^E??h(Bo(21|HBw%;W1hbndyX-f(PUiTA~yThv>dRSt=pdP++Y{>(5@Xxy;5EXiR zFmByX;^0hq?|C=WDq(S-@;fvX!sJbyLIBV@8&5cEzPSB+-uV?Qn zmYww3;r$3Wa0kh)J$ORFc!%zfug4F~?Ax$`*5R;9@V<_?(2E~FS|ztJLqXB8+m?nh zgz%5sYwlepc8XHv7<92x3W|x7508?Gz}7B(YG%32^f=_ejpRH87YH6Enpq2?Mv5X) zZlnUQ{4G4B^kEqD>A`O8)w@wM_a+W|)u5-4!q2XkSRO^fmEakrC`=adRokJR;nZ?+ z@&q*0@8U53{`G@vQ@+$ZjN%12pw{WyZHwX{oy`_@1skx)paCmAa-!--oqP%}Lvr*k z;-O(L+N$Fe2Jo6rG;eeejI#(v5a0P6bTcW2N~%LCO{A>AZ|_Y<*~n}PxL=cJ-YO1% zdv@5=<_d2M%AW*9A_tc>yS4A|k4AG-gBX*8?;|BCv?c>);`{88_Gdm(&fP=Wzt8*j zf7kvp+0DoP|F{tRKTK4!Z|pxd!T1l22L21#^51+BGltLmC&E5Ujub}roE29&N;DS}FqCISw9nBR5ABu@6-|K2~opZEQR&u&y0o_U`8KKD7-xz2UY{CG%n z|MJfSK4UN#%T*8jTN{6*G8l{A{B1G5^TYT0JMf3R%U*q#BaW6X?#G=h7zdBLoVIgx zv9mt$FE*#1H$M?cAe&%XD<88QxJZFAl;^n!OcjDr{ zTB$sJEU7DO=CX`M<+W_PRw?&R534kb$LpLmWINAxOInn2O?Bx%KSaIh3lTfcoY;5i zi>pm~W=b_TWYl$ZbY>?c&eQP=8J zI7OX9S$3Y2tya?JPkwS4sNaP5tf~~vh#bjnm*6u!dwfsWLAC8G>G4m^4C6>5gU4L6u*UX1CTs+n3F_P0_Gh#dYet=)z z$-HFl#poTK6kY4iA`eg2V2BxeP|u@3$~{!ymb1WCpR;L9Je-`IoQeJIf2^`_<}I$E z&=RiX<+kt2hA38xnF^NAz;~v%AWR{c6;I4qEP3^G(JlNVsjln3nzw_<=jfEq*Mk1o3SyP=}33?f(v{V~B-`t!Z4=b%MU0`Kg;+MX2 zvnsV=qE02|>c;(d{5T7&8^=#z{rPvCJ{IlK7fvtLfB`sRDHHHxkJmp6XZGE&l{m9D zBqSt%e!R*yI|paXl-23lV#2@u)FC=bOC^iGu2Y_~(^O(LdcXU5Nq!GgeK2QvsM1Q6 z`O~>=HYut2k30-1vtCWKH!ZKTa^n?6CKkNkN@dPqPuTrNoR%Q(`LW0R%HC%c+}}p> zOI!BT`7JoRwOWMDzjZII!a2|%wO3l0?#>@fPN`2d$nwmXXgs17Z-f0lGh(@6|LG|8^u?+5Gn60<@x^2x--B0!@A@CnYKK~7_4*i8|Pa2{+=$* zf_v@G(ISD7k@h^vPrFy_9KLIp{cK;jw3TKw+y|CyvwO#x{n0pb+RHPFCk=C7xV0C} z^hLDV=e=-x`Rx*NIWt`xb{APuK7KGUUL`3xc3scFA|&!mIlpS(#d1?0R#S&d!@+zk zrIgE?r^e1TGITJKI4^%b=iFlAZgE9v^2sMl*8L>!+}BgaXI4Fu*A-nO+isWi-F@t) zqeSuh`VK!1J6yrlMnKLX(&0sc#gm`DoPo=W4$(DC>TI9&y-R_lqe*e^zX!_KWx*m7>k z%34X^vA|1$D#|cIeaq_WSLsa3u^}ya9FJHv}A6@vt3w* zRv4di;dsuxzQzmd~yC@LJ~qP_6$qKep=wN>JI#=h@w3Hsb*3%WKQ zo~d;&3HExMpYXnn1y>g7J3ByIR&w-{f0P&~v)?x$Xk^w}Bt*Fxwq;ufKi=S#s(bp? zImDIEl2feDUF_%(v%K_zxk8Ovu|#dU0z~j zcWA06WGZ726dW2ubYU%2MKuUB`5kG6XE@@H0C-)+_4`W)L9jc7%A z98xp$(m=_Q9T`RAY83G;Zw1M!F^BJWUmj>m(Zi9Bq*Ye!TXO!g%EI)rbvwS6>pXjs zIU_w;6Co#gYhO?$h1Au$`d_M!C#bNu(s!QT+#~Ha{M_*8pC)Q|X6WwPmNeJl4zGT3 zgXgJ*u`7P&h0eyiq1AqVo$wyxZMGQ}urQ9=5Z4e( zlmuj4klEwQdz)31tFAJ_26+{khc->k`X__t=lfM8*=!sxE1brG&SEck`vxoxt#G)E zt!@wtVCe=z^;Uu=vK^-%ZFlTcEcTkJrOgw{AGi}yU+6x%-L_fGXZC%?5iKPh+b-D6 z2?98|_U(5JvMgIqKU(fQl$IA7YUbLcLtc(4WMrSC1DFIHaJX&1i9^v?xy$hLZ7TDx zR;t@$OX{$I6Je$E@y%z%Dm(I>rEzgjR@&tJL><=?kn=tn8iIS+vG?$`?^Xp$S=B~f zS|=}Mo1Sk;L6>4(gky;Zg?Q~4LN{rKIhD2<#i5@pT6`PWX8>2F*t=@);genyeKl2B zgP3R~FS_A@=l4z$sJfy&o7h}3S}>Gucdu-Xb!(=DVV*zSK<^HxKUV*R4RV#AGHac|;Kt;njEvA8~XI}sE;{e=YRMs$@ z{>}0(!jJEt?r{bVvc&%-d@d|3OqI60nJQoF*aJ{m3yX;AtBD}bTn$X7dt)Z04Mr0J zFgriu=vM`+id?Z?!8}Aq*T6i@v1n``*~R<8CQ-oDg|FtC;^A=(ux~@_FQ_pQ=KAvc zqpPeFqH>SUC9AWQ<22@f;z*Sl3L% zyB-8D$sDD34;F<>S*Wd0o_;AXBnU@5gK%Zr=!!UW$E46L|8;=qU}RY$oIW9|AssVT zsuL@mJq$!u0W2Ja*LcXjm$2JVVVijiMR)JqLv5g4S?y`FHF!^zm-+Z^=Oqrzb z)nAeM%+D)*e}ad-tIOFG)ZJ8mi9h)5i^2rF=}hNTNf@wXOFjpd+Iu@#vr@n|^=f5h zWop4-vOFAc*z^jlF5hkJo6MvS{MT*VROxh?q?)~Rf9=7c<}};t&b*iMN_fzTDzgQt ziuq$FS#wnnyPa9mIhmPKFw^Q6?p?*tXV>&E%#BK#q#$@I7rrg9H9h<4ydk^OcWz|9 zM_7f>&=CWJa6pG%I6iC3y67k&IM9XQTbC6?;3G6)+S)ogA?w{cw$;JA^QEUrEWHw< zXX;^*3v3Gu{t_6F>|iJ|Ja?Au3@{ZxRoBIS>#oZ5LyV&MY?x~o4g>A^VdLc|upFD> zV8jC%O5@V(r$)O}*mF}E&vawFlJJx_kUwc;Q+wcF=4d5*TZwc7^^`OQ?Btub0je)56eo(~k`%e8~ZSE>e6G6*)M znj&ENaU##Y{;~Dgx#dos9{urVrM~jrm!zfap5Ha}K|<0rjHA(tgu}U<->T=<73E6c zz)1KU+5PLz{F{QfMbDF~;mFu1&SNh_n%OO-3n2$(TZJTx-hFc!(KQknk{r$A{YBqi6twzT7u zFg``#a&<+yz$0DV295GNdw$wepYIG{SS!!3c=ojKRAH9|FuCOVZlFI?(QIaQX>HyM zHzSHVCyQ0FvB5ir>OtyL0;Gxd{qIzkIG`+%tUg zb79SDIEru_?tb`FjmMYQhv^RyEU5?8lnD43FH=s8}NC{nKMXKgGGXg^tWdTMlJ9vd@p5lgzFwOIi`2v*cm&p8F~Q4^<@@ zHbHCUYBzWdU!|<(=EV(aQk-aYCM`Sg)eK+TTYjs^t6`>|TdydNAK1EG*0Hn58<01? zD?u~r1`JaHzSEan7iCBmJK|>A7bxE_?6!snje&t$n*|DZ&0(YW1qp@R-*N@$*!Qn9b>|W#Q&y1f^Ml9Fmoz={q#9EkRm+FQxN~2F+ zmfuVS0dnsQ#qlQ_S;^K2U^CO*SLn2u|Z9CkeaO1@so@hG>9bg5UJ8PM{((e6;lg;qhzEdIyAJprQWPk#8vMs9AHnpLaT zRpOIXB~jReJTduRF%BL;yIX~$??l8fl9&8<50}!DMIy+wf_vaI8eL;#ca_ek@_tn* z-}&k7AP)BJcuOaa5+H=T%-WH8;PYzQ1M`JcpQcgCrL|k85zc!l=-xg4$O%MBFH)N- zHQ(2Tk!Y6S9QP8Gc0=r- zC^-YP7+K05ag6UTC-`FIkiQ8C_(-kNR0lT=$9qKhvS)jv+&c{h+tygTx;M^LmrxCp zFmdWQ^SBpBrXo2d{RTlLx{PhwT)ny^H#>yu3nQYanAKcW9WI>(H)jqH%*7M13#<9* zF^VUpr-N9l3tMG@=x8 zbic?!G$lw$fLBb%hrBC8J)Zl=%e7XB`g@rta46FBI5;W=2egL)IAq$cC=4ckrFw?L zo(gL#a2e1R+PcAa`jvJ&e1Cw@ULdtV!0%g)3x!?m^aI+82<-8>_PMQh-n=MuAA{k9 zv;pqu+2yucmff&TH?S#ubz!YdTHy|Hqg*jEdBYJv+x=H>-^W2iF2C*Zx+C+0dZi)@ z?-Nv3<91B8c~8IlO7&TuW7mn=NcljMso|_T3Fa!tk{Qc1i?#OIhwl+ucVmyj3zfkt z`Pvf-qGQ;@R`I#8nbnJytPr~)Yj5Zc*RuEUs

^?m(e3pf{tFZ0vvGzB!z7V= zR}v)1ik{&YPmm+*tBo?UDMdmQAXLVguj5?H%^R;2T{KWLR z>nAT8nENa(=#TaafVbp>H<6Hi;nMK>&8;gE_=a9}v{IpNm(X73YalntoZ_+awR^KH zYXSu4UoMHyh0|#?bbt8$-|=%~lRil!U;xFKs|at8!L_%%@vB!A#+?#?!7=6wsEyjm zM=LiL!1PFc1@6gj0jw|_9%;XJq}4*{FzwwFbVbfWn!tBnVH!f;P`uR zz)M5#V=tT$Bm!PP*?7g+t`s4q`Fp;mhFo=umoDMvLF)K_x!?8c{{O)nG60?QzF#OuWOn=K8X}Igx za0{tpSuf9;`E1}@AS!VTj#GSW$0%$FFC_GGDx<`*X{s_#%=>b(O5ReHA>ViB1Xqcs z-#;-h3LYiyfc83^9GTe>xQLuw10_w38{3n%1jMghIiy`&j5I9wx}2kF5*x^B3n(CC z&&!SO>taChJp-e80$xEBiH**n9KZw?2Fs(}j>ISsAz^j*<0~158^FgH!mj9eXTC*v zUW0Wid$wQDWL4R`*Gb}n;9~OP07T2k=8J1(!F65TN7BCk13|juQ79pNsz#A|E~+HD%0l zoy9S=F%BH`(bd&$o>pwu*>5tqQ@5F=vG(g@yY9^nrnH<&@#Pp#`@BWumSnC1oeaxu z&{7engvW4Hh)&vtm4c?We(YJ++`}pDs!;Kvj=ngdtt;eveiE>EEmm1Bs9aR;oLDmI z&RTDV0ycRj>@2`-3l^K-4FR!xhfNJN+BgJ*D!6e#nek&bsT0U4D{(|?RI_8!1j(t zI1@>14{$U$jTr4VehT*uByNyWjGQ38H;@ZJ0WDn#XdZubp%)RL5;6~Ram0}E$=q-}}<4tO^AA6W}|kB@}G z0ZW2tuf@?dJ-M%V(TW}af&s!H3m>c-buA(-}%lq#5500 z1DzLYT>tt@V&l1Jz_r8^beNy?pR3 zWuwj@Zo(|*ZYcYB7C(N;*~hwcyeaFl&e5aXpv-DKk_AS(2V_JQ!FdUj0&~Kc=d z@D�SP2u*mna-&3K49VhJRhQL2(C`%$cCXy3-Fnk!dEDlS)mYy=OzWM@u}X8j&^j zCaT@w^l+N*W(mbY{HviJj!8g;xYZzKxY8QhU_>c7dZ=0z&b&s1tT?IzC&|~9ZBS=j zGFaN>h!yZ=_8`J$Yzx-+Sr5c4v(R}WP+SV&r~V8XgKA%0`u$jnlrV)lR*esXQ=5vS^Hk8f`YAk(OrH)sJtqf>Cf>=&3z1aj!Gg$1He zksQ?*YWwJ@#7qG*WMl2hC&{4d3ttp@OgO>Wo1<`)?%elqaOfybR>2y&MO?i;Stlj6 zc(Tn^*=W#K^-;TqULvu z+}=PcEScT!yPhNEb*;lNzgs=}K8v40)+r7$S73(ME$KKBte=vfHM9zCKd zzljomBE^XlqV&iYKBRwV*NkS6FXDG45@gd`_GvlIYD}tjsxsk9HlaSCesRV6!HyQB zHHQYnZolsxIVuaMy4gTslSx{PcNU#2@*{*X;@5|Hn6@oaH@*S*#&S5{czEB z*yI}kxzaJ-Rvk>a|jhIp=fpq4GkpZ* z`K8PcM1!~$__?&8b^`YwLdg=sYBen#sxf_+q-&qRiH3t!-U$|QR${()!F+*lp3eeX zuZbj1+M(mV;qmBATrR0rVMs`o$Z@+;_9SKb6F^JNe@hk{?pNV>ps z(W;rq?JVKz4rZ7Zlg$Wx`DCxbyGWnIh|DK)pPvZa?$EIV#J2jC4N93EPMj$%TlLR( zfs)a(`Ceo#3Nm1VTD)G|w%nBuN(SN6o3ipEEY3`O*Fq@y8ZcX0!J7{{VMWE9ytEX@ z{yobY(kr+SX6EZbMKq*5VyaTeY^tMn%ApP-mev4tqX@c?mF%v{bpXUD1uj#S*fFUa z6Y*)?87Y+qZllFho5KYgcYojEXG-tA``luSI;E*py&s%kS)F!$Lv)B5OnD%@2df0 z4o>7{2`!C@FkhHSQkG-oBsyOgB7Xs(HICwKU$B;P#ruJVh7`SwG6#5n4}K+27b-3g zCR)YRq+456=1KZSPiRJwMtaUAQHnOu33nxi1*~#gcbW*FXvDYSpR>(MlRspO{qr7YJ0xO8%th9gb(U};PT?UBDL60Ae`;E0{+ zYML|Lkv)ab+~?Ib*Lk9)3+_r9%1L;EXdQ3JUNbM{;Uhk{VDL_T)GUYSDDwKq8e-`D5wAjZHKGI?t*qzeFQ1PL#2HzTRl`9!q=NQT^amIT zxoojJaH+-$L?=BnAJ=r8e;n--Nms0Q@42w70*U)L)jOrOoJJ1VkH@S%DEH#%#%t3^ zEhu1-=Ar?m1<#8ol3hnTmB23e(nZP^H@D^5M*zLVuei_{?>^co^PR?0BT!c3DC<&@ zI9WzQY#3zF!61e{^5ZpY+l&vfc9bUG%Yrb1xE>;+Py^`0$*KcMp7W$`8ZO`FQW&xM zvTdhs3HYqp&_HE*j>yx@SgLOzOEK;C0Ix!B9c7 z_o-~hYeTK?ESm3cLHesc+F9)UHow0)-&-~25ipS=PDjP}fB&4A3OX@J$6L#<2*&W4 zd2bW+oj#t#4z&5f`G_ErY)>lv55wQ_14Gkd0n7Kkd$clRqE*{0aIL7YX5@9`(wm1; zvowidAWE6=0T;=@r{+w~?Oo_zwA8S}tMa>+-qPZ*l4UHPQllGV@t$@zE$Jp{BR=n+ z?RU-PYBH|{J`f=Le<0V0qTMy*N9ITx3Fz{*=ON1rNA^Lm^F2^=;{upwE9AZdAn6Qj zlB1W$BijvJ?kU7tS*L@#D~lelIzldE-}N_$a0gNURg-oX4+1B@BUo6=^tHx*l$cDJ zJ;+!y{?T(5_CF3-uoF6pm;anc+2i!`K(hlXm_+FM0UvD;i>1NfmLz*W2px4Ya z>7k}xkwKuI*>b&g0GV5gYv)im;&d%ec^I)cX-0XqU*UzLMj?=d0dG6i8*p9fjdE}iU>@roU42)`_-xENt5TZrZ z*B>?2XG!}YlE_$Y_98HS|E-?PhGzG5X7Bj3U#2?JIl-okL!OYgZ-lS{NTLYI?b3}? zpxw6Hw{1sSTL-@8*Q&^&8SsRmBy))`-Qz8)mWflcAhnoaUecQ{9dnHeL*4jh ztyM3;@8ND`;~QQBDFvN4QYpGIbfyg7z3}_lVySNlog+r2{Qzeomit$4@pU+9R4JH5 zS%L&aIiXZ81e>ABtmk7xt2Hm+lWZ0G1#~zDHiIxj3d>YlajlJ>K*|;Bo3Ub5qulqT zfnY@JNnI3P>slCNo#oSP$YI;2%?YW^(?5TFQH7GK&0x6ci+$Z^bWu&FMZlj{L`%lBsZ zFxVz3@FW2FLXR%4Bc!F#7B-617k{7+QUi+(D%doaX-Q2F)sXe;aWQ zap^cr!r_8_P!!p#tN6@Kn!KJwl`i7Zik)IqVj?k8eX0vd9kzaPKbEp~xMa$2*Q|Xw z0#LaDR#o4nGX3*Pr->T5{=8DQJt?U=fOqAQ%Lfb55y1k)klsOM_SuzCeP^HLcxoj7 z`awgGSxEN6lh;wH*`^&UMv>assbZ5%b15`j^~iq&PW}wf%Pk%FFItf};asvk%ZLiJ z1b#!m=+j3?$f8uHpo{b*hI1(^TEw@y7zj2=Uzbpvq#7j4aH>jRCEwAmGv1Gu%W2F_ zjyMm*hgnVzx5oSGnYiktO`QYS>4PJz#SzxE^^>-$ttOKB`3({W1j)8-A*Qzza8^J% zmFly6#;11|xR09N&~ZH+I?3)0G<-Kx^s!;Af)R*CX|kZ8Z<3i0c7LvjO4&WACxDa8!*@XEeCw==#MV;H4I`2vRfyu(MfU zYOh)T!jBV7SOX=`6~tAj50wB7(L>Q~Lvx$l=nGH8t)P<;614@crfM&r$u*RqN^8SK z*JQsf7$P0w0JtB5I8;+3!4Am>1lY9wxT~FXNP*=e5&uM~9&j0ahBkx9Vg79={$0DH z`w~%Ct*CT2AY>FF=;b3;lj@I-IJ^=m634p*IXl+c-QA6xITZO?Kca9oc%iGn4WSRs zts>9tTfAZ&Nt_gk?;-f!16or9^s{aC*Ns@WVR$I2KOF%y2077B^ojJ#qg~xe>SWZz z_a0lKWZZiE;`&S~V1aP0gqAN72ArsS+Vb3T*(X%|2V}&_&PI|h9iOWpas$m`G2y=2{qk#z}Hnf4Bl<%1Uo1N#S+Mj&E%vuYh3>c zua-pG@tQCB9wb#Fx zIbh1p3S=Q>pZ(Zq0Y%4?-AmSAONjOqp%laoa0_|NS0wQwTgyK6>?;xBlM(HD9nna3`q|u}7eI0y9 z{D=%f2_j`B5rJ^^;#6!0ZUK;Mp@&p&&$U%fU~g7g0(r-PFf zNgM-CgN$XsTpS4lem=K^OWq)DB2ilpY9B_{i>u0N{>HMnd~s>==#EaDr%2DiBsyoL z0wK3WmXVG_CLn!c)5PdaET=NDe?%{nu7IdF#LMgSeF&S4P9(YpiswGT-VTwylF~#< z#jccO-8dm4#d6}BG0Hv=O@K9_qY1=IY@S3_weM1*HL{4CK~h?yVriqHNen}Q?`xp{AABcC8FJwsRT=!0d7)ZmsK-Pc8Us3nk&Rn zFc&Riz`CocTue)}Ucn`CLdEV-7?IqBf5=HIj{WjECHn-I*N7W74i*B2^}{PwlZug4 z<&X~0u~U;?s%vz^D+#Q7{%FaZ6SBWMSQFA8QC9$Gp-1dCY7*+wB^{t?iJP(w`a5xX z0da|A3*2hNA-7MtxUuULj*Ziq2cInA7RjhknGOBUoAx}%Gk`K{2)-wEQK=1b?lmt8 zTt#uZv?&iPgF4OVme%1_dyzT^Q2DJ*i~0JeJu8x)$wKXlniM>9%PCS1+=Cn76HUw3 zeulk1(Ho*C;nrrI@ET8+`fL=2ElTozDhr{^LrT8kdkf&uwqi44xaF&7KnZbh#buCx z>86;``+@Ghwy*%kEkKa{W`7yH-eDZDxWv>{G0(|i15#*~fr%edyhE8&+gtctV*-kM zp|`tNDUz}ZT4r{s;ZRF2PMSDX*;f-`{R{~HM3iguR!S!XhE|}ya|)Px8`u=&p)JcFBw1IfB~ z>g|g_C>j#Z0-D*rpF<2Vri7R7yxGM4#cSX>5h^=%E226`MdLCk29O9Et`beX`wi7e zU*oWfGFMTFhEF-Vz}V&4-ZJ9Bk&)eggNoc(L(D?v*CQbq#UlA;7)}Xd=D{nS7LV27s@RgB#@9 zUJS%%>K%{2Yd;0XVH>m=dnc*eAkC|=ZP7}_Lqc1Lhw>a51?4EBi9;3#DrhfOGyvty z)d+qfml2R&BgmM%o<*rolzGk9FvmtG70+LG1v0JIKz&I~AfLJ#iSjl)4Ua!$p|8!WCS}<%?$*nwg%Z`*=M4u zhrq%ne1T@2ZFtI|oyvhcJ~JhuDo7_{2uxo*%-u*kd;s+doH3WCZ zR5%kVId7}>DhVfFv4@gqD|v}*+m`fcfN9eX+>UD}Y1K6EVT0*i_dUe_44H*8_9BLAJ{L(#DeluXIIFxos1r^%rUMz8jOF&Z%|^|a6V{RH_{oVOYTD>J{JCQ%2FPa9WL0xtL@l(lU(b4G|(#_sRfZ+#2e6D$n8au$b)}fIp7(8pr zZrg7q+hLm5snD@f@!c*k3tdlpy+=rHFaxGDn$!p7%WOd>nNl5$=o!lo9cv{=gMRBU zVr)sEP9-YuECBe(x@5 znO|8G-3Oyf_t&BUXK~_y3OEx{@Y zsp5saUH)5=7NWN@%u4r9B3>T+;uk*6R2olr6ryaCgw$7O3k zTw}tP01|Q&_NJczn8*?%wn&$EwuG2C21k^*kjE!FlTa6teNKJhR(;5*E8(T}2c1df z33l-bwWd*{0#t_8z(~}%BXg|xw_C)Y59$-QtcmPg%qaOQm^DKUd+ z563D51b_3_6M8V}?y zj)wikn^k>x@q;i2P(@V~wu_+eQDqWivr_5(AvY*Og}59z(hm!0v&7SY1tSQ`gnxbf zn{$oJcOEzDAgY$2jTE!1EU9T|~_vPDzz{dHv32{S2hF@fNPs zHbla)Sud{a3H^uW;E8*cd9Gb@+&f{_%M@X{)wfh^1N9$4%|F!JLPdLuB;-*5v=+|i zd67aGJV;L4D4;NK|LfAN{J-C+zr_ReR0j!ABpGy>2M zWNREuy0bb8G7dQ+vUm?j6uo;$$3mVo>=0;k z<2+Jv0?OJkqz$Jg3W`&xY{_t2r~pCG-FoLia(oxOWfVCyZY1njjCSlouAJ1eJ$u1H$S(WiAK=^7gbHgZIHiQNaq3I z$666a@oEu~K2BF{RPf0w2O+?lVUPA-Z1S9Pcv}C_8W<(3KcqR8@mp;2SfeT&tJ(J% z1zeHFhgbPK3QEK6R$Y>=Gk%|$oSeb?PX3OffJDc_@3AMBoA5l=gS-fM;&*9ZTWajl z<9WBc!=K}EOICOK7Ty%)`M&<--|-$@pKaVP>%#Ncj4EaB<+r)o{NXb1MmGPp9BwaU zP=7P;EV_`+6D(e~??>XIZ|O1=YL{;@o?^BsC6x)x0vXYAlhq&J_(9%9+$R!_Rw?=M zCul+oUhAl_SxXblh3^j|Q>Bh?20y--5$c0@^FYcvtA<%6snzKt$FHnN(=J3mLN3OK zC#kv=m-wPN(*pFR5KvZA%g6Y~cxS(nd-DskEf>;xuKu$9AorfdzR-_Ql;_{^HR9($ zmx{+k&0X=qBQG&)PiPr^nQ*;kMJMGG| zt=>iyfvYZsO9z{{vrEI97ucg{6J}aBFp-w$%Km;ihQH$??#_+P4!P#ZSu^Kn>ddUS ze*8J3ORZ1f6Go_qxS`OC0vQ*cJ^8TdJTGPZS2)l6g8#kL=B4Nu3Uu6lL0xpw|JDhQ z@|&q4h%ACoXy4gN-D;#f0AU;di71buGD1UPfMx|if?IwqP*GQ4FG@;v5K{(|7NoJC zk_#F*p)OgxoiHOcz>?k{0sz|wl{_2dAAcAS?Ag=3jJtL_QLY(B-`Ga#%y9NNyo%8C zWi;BY!_Wv-?Ae`opmT1E9okhrM{w{5@Xdo%3`07d*~D#r?dQd1;jJD+h9{wHefj z9SC9%Q0?mW)RzsC^d70jspFD{CxOTphSXGq1zqVWuxT{aA*ptxS>{n2Wffr1A~Jc> z4=R8XZ-wi6m#%s#dg21*jiglt@osc04eh!!VErS7sJEXfMAM&w9<*VcNDSyj$E)pW zsi`y-AZ-$5D&zy3EfUxL{s{h#Qe9g=ogS32fgW6cl7(WJWbu3Lpj;xz(6FM)q**~! zYJr0(4Gl$wh?{3nH&;fs?(f%OFkS{qKO);B$_-tp2BfN?LMb|!R8T0sPGf0QNudeJ zfFe>On`Y?3=;9A;@i=1&((@NKY67F4NpvJ2PgeDgjqU+ z&fvuR(PYCNJ4JF6s7mTUy+E3eCOa6Q}w=T>o;m2ufizVKn&90!2NC}h)m90gO;#K0fc4D z)FP+xr+bzg@&v^PE#KkT?oxt2v;O9z`K~MSn=!X&=9AoZN3F|>na6E;{Qa~7-9}mGl_(NoieMRqh?dKo<@duEw>yZ(WiHzyZ;WDE6L+L%Nmf+q{b>X83$h2Mc62@|-D*dx6Gprd06suJMOU z{}@$UD9PNV1VR`Y@qCwK(Kbk2jt}wd$-dZb-S(c-hD^M6r({y%AI`*wWfe_7AW^Ca zJ%Kr!)t?7;9t8fQ=0DimvtO6Vc*gh%6Y{q8B;|YGU{683s27})_#urR(p1{4d>)kQf-86LL+U!La? z5BfVi9?#dcZ$7ka{`M_@?Je+SJYR=CJO9Ul#A-L;N_f6@ZCwRx=K0!k{vl5&!4KjO zeW--}_L|sLpZ$3g+{O5FCI0_t6T8Zwn)66XNvZPVO!-Ngc>w`RW6X!$-JcgzW1SPU zNHm}@wew4!qvr2^@c457fC4C@CfK8XDwr46q&7r@YJ6EKX1yfEg2ap~Ym1qwwn}#4 z1$QrtamHd#m_k^}ZDfWdsoBhr-8)1=GPE>2;ttkmOi&J?(hpUqCundRDjANewRp-D zjH0#clQFlV?J~qrmZ%$W$NtgWkg66zXw$?@i*}&0J2alN(&x>dfv!@&JD9J6N{viG z7bHxX5GeK5_18p%2^kcfJ+nsAv_m7~yU@57^?B+qGVaxdtC|0{;S5I2W3MTOh!#Ec zeYzyNbo)pQ(c1})N&T;~chR(|fd*kHV+=>4(L&&WcA>n=Sv*82^ikVqsyCCc24I&RppXri?{E zi6x5lYg0SYOacg~HQaON+}?J`Q-hmDF;Y{Sxt5giZK#!Xp%X`-;d#HtCaK?7$G@+; z*ozU8aCh;ptad26rBHsB{&;*p^aQ%pcu0ciCi(yQ z{TUgIr{*JMZM`5n}oCcb}33hvqZj zTQZQCrA+>V$7cL>YrmKA7sxyP>Y?{UM3)AYC~@NZXdWGn0V1_IiLGd8<}!ipTX3kY zs2j#ckcSgwoGR1MUzB);@_iD}qlchRWuaT89^{FRZ!M&1G&ishT5&tlprTxQ=QClR zOn_my^E__C9n6mf24~vN%ahO;0!WBb5_(YLPJ3FxW3B#%4e7vJu^#Ng%(eika8naR z%~@Xj6C1iU^^@!`;SX%7@20noW{i@gp6awDauJE?e@77UX&+}VP z+rsZIetLn}yQOQy4lp|){Qoeh5>3SV${xJJfS1Km$Ct-0pMhp68eWqt1gB;}QYWi& zRX^wjOaG6XqTQ73p$OOjVJ```Xe4RYGvxoO)b9qSYr}uNrT37IQ-7m{55_U+?BKnT zL$<)0HXV?#c8=AJK;|GF{a2o6N_55A$vB`ZL*2yW#m?D$^-wYEz6PFGFmg2TB;{-f z(9L|=)zWq5sDOhS-zbd{NJ15mM>s>g+Xo$aTJs-_Qvj&B3NHXvPS2Swiwe(^mU2SthakkH7hhYa&h*C_x!Igs|{m3DUZ)*5tkU#(%={WN$-8 zhC=tzl)wjI4p44dfrQ8@Yu%*$Y;o98?-(jt*7*jJOr%h6}Zs7ZP8 zyA3{rM=GM+b>~*`+)~gH{Y9rv(i{lxEkIG9i8<*ZZ++f6$E4IJ`LC{t$eZ=jQZ~rP z2eAD9Rb<4q^9+C+LM+@CXDA37o0`N+_4M=*_{4wl9NOgSs_NtOs4cT!)Vr03G0EP+ z4ICo2T>>_o^!}>9`@q_+EgN_)E`If)z^=8l)HZ^joQj(&&S(?n`LTYn zTaz;WJ=(!nsy{-x;cs85(e#Tnp$!5*Wok~&LL;Ol`U7j=m~{}ygHG9Pd7YA>yYP{g z#f7fXuc6{IFGMX8)YA!J;5HiW(2UK;%y0>6gh}yHe96JeRK)JCXtNcb8;z-sesS6{ z%p}>JM-bufx1T%;N&wE|BreUz9E~u(=Je8qS(;=a4-~__5E!`eQclGDL`=m+mB``j zBjfWEWyQ0zgV8*0;n+d$x}mkioW%SVKJ+MHU`@_{{z(uF3md>(E_c_;r)*_p-! z^rET{4krK65QYwoEMgpzM_8(&ekSy-xWuzCJWYuL6|C4V=h*H{7wh`BoRXXe$1iSL zzjHHcQeuWjUu5wZNa~N=MGDFA-b4#SljFUIPYS9tsilrauF`}Bk(d?w7*Wscn%~^& zHF_cE+)9Z%DPZMUwLFpN-6-Hz_LnpTlNfr4dsS5CTI-lXG-4da9)mWq3>@>&nP*o7 z%rFs5><`8fLidq{T*3;f?u^N8{xT5kV78rkU$yx!KlB)NsR0Mym-9X=u}Npnh35h(EyQEQI#Hnl zf=}6$)RkcFspFFF!SdC^kQns#*C#bp2J?~+hF@_V9n?VmB-FNG)7gHcrLNWJ`@;f;Guv}U3tu`(^w7k%WG0Ae93ylKOsBfq-g#KX5}ekOgCKvG6V z9bpo>b7~!Y%519Q`4=|LwQ> za62`aJ9^{OrE5Usn`vPtMT(CQ&x*(H%TAg6gpn9|ekmV;T~+SDs#g!sQ@5it0Hw~1 z`L6gEE=N$LFJnc@SL|N0i9IKbLH#ID=v?7htLzipMF1cRqYqtCr5WeFw*=XzXc868 zvLGOp;;j(NEWdov^Zk}|E%{9|50)^E91mh@IR+d2!8@KR=6Xj=CHk5sSTTDL&@~{> zmqwSiF{WJ8O!;KG*x*obLBZ?UR~LgFeMHd?+mm7@!%GgH9pzpK3t*0Z#DmNZ)JH)? zC+3Ymv9FZkgy8Ru4ToYW#7jb`9O3!)AvOM%02o z4&T^IZP!$96+*GU_!aEu)Z6Yk*D&&Lp6htMoubz8MbnJ0QYHAJewxXb+U^AsN*uoIi34FdIFSysZB zq=%OF94($7W?RBo81_&5mBWhOYoGDykE-a45Wclv7$dwpT#o#QVSNCPL+D1YrUrc? zFbojM)uu{N57HYrx+`93{xqgnoKbQx9xv?LgpX*@Qi<~Jlg5ymCPkk7fqSdjUs4w- zx>>?#j68;!gYz=-1!&PczP#!O=@vzMoaIUmj%yqgU5P3BzrB%6V8 z_Sv>!cG-uWUbt4gx`>MQrq4GEfr7}<;_+MAy4=%6jjU*&O>2A1F!a}qVvcnOBUU(+ z$a}o}*1S3O#KX4O-T(2Pu8G|94QQ~P8l5Tmv_m2T$K0(pg<({`h(s+*1TTegxyBy$ zP|*lSC)Rp=+<5WSTq=F5Oi{6#mo2%fI-k!^b`I)rFMKp_YuuOP_0Ln0jwBxc_yyf{a1-7L0)G z!6(^hD#^fuIq?*v{X_TQIm#jb6+uUl8#Op1D@+^fH^ubc)e5T**<&Jh_4Q|GkK1Py zS5{!`toiKwo*KE}&f;K$EVxA`drB+!MQ$Wd-WzLinDTb0DXLzV1%{brXquRq#9#hv zHskZnSO|--kYEfd9qTr<-4tP`eXWUk`$)p=bEW4$o`hWo$90eU3rwLKr{tNff1j>d zjF`tffp@aOK+nw_xtH4(E#9757a1OT_JnIKkCkqhgp~?R z#Y_5Z1pikt0OWyaJj?qOP*^sgy>vajkOTnoU+wA>IoU6*Y1h-lJ7!4odtVjqs2tY{;eq zDPR;3DtD?0iT&xLTQFCi!rBBT@qZmN*U?X((O`_vhRO$%kVTUcd@rD*h-B&Xoe9iV zkO<_HtfkMrqHzN=bu_tbBE^g?BG++x4-M_4@sAWDj?5bPhaRuL?KH_RF9nSY6;9E# zt*M;^952z~Qut&9E7klfz_L>|<8|qFX zmrjdC?y6b9D{|?5IuQ$G79u9iQws_AUbPi{(0(?-%p%D z7JrUiN*eQ`pyuEpQ?B!X@?5L=tsrzVt6x|F$&l^(k4w0H<*4MRiA5_Yy$Z>JJy5Gy z#vngHV}uY*ZeaGH#rFx!I16(HBslVdvrjM9m-=yJ)-UW#=@52xiy(NbNvAcxsrYU z>Mx57&_6%`QYMgM=JDbN4 zP9hX@|8O%!9tSyngN&k`2nt85sg(d?r+ykoPO7fnA&Ay$Xvr{^);@l#Eq%@l1^6~L z351YmWMpODi8})o zS-9D1+4{C=&&QW5`juOHMnpq@2LZb#PN|NjEIK86V?p9LV_I%-Xo&my0r+GX3z`!G z=JQni9L;?owvhncOdE|4g(H^5gpZ8(XTD`Nw#XS{F!bQ4Ojwmr8NE+fgOQp3>y)ozwdsg78o?euB9Ele*r`ZM{hZ6H2H^7ZxGZX#&aA$Qik71K~ zsE{fRh9!j^x>e5L%{s;MF;>i)*#owrM{S7eRn#N286p?K7{JY5jL~^Q^(J&`_d`Et z9Xi1X8Efy7E$NI`cj^wRiBWMdwm1Li^>*3Te_-KC-kq1^ekvNx@~2tDAZHBe@5JWJ zU^@MCyL`BxAmz}ICMKMFGk5w34fv#Cq)_RZ-3HF7aoZg0S>Ub(Lu^YpHDl_V&&$2| zyQ8QTyF6Uc!-@M57~uM}Fol8|WVoXrP@JQZp3dCaT|wx^ZBps0lN9bwN?>9JCuV)| z%ZNY}d%^3`#k}ZC<=#CTYx3kiKFRj54~{46=f!JpDs<&U@i$n~C8sd{u^?`1HS&^qQ#ly9j6J@W~2hPrZ6@l$%Jv_luUSX+iy{<9Aoa@Rzzy z`2n}{m3nLO5ft<}dT8o`k)wQ8i?Ih(BK4c8^TFVd6jzPdeZz&0QYZHvE}GSY22Zp6 z;dxON$o&8ka5H|sLlyTYYR%`TdlGMvv{|-Y44`cbJERv21Vn=Q2APITOh-t!!t-auYcrp>@+}9j=1zXqA#j76f@G6hgcpnkkLR#Bx0G-TdmSJ57L}c;eEa^KKTW z+cC855qT&UlcFupzTV8@w`p%kLvy;zNKde~LPe--sEYhg+#Gt990>tgYsxssCCMyo zfzXti<|*i85m_2yYg7ky2Hoq*Ulcj6TYngoO0Zg!rspQjMOn=GZ4Aau0tCv};DW8H z1goVJYYM<XyWpdmO7{)Fc<>9PQ{lsj`R+WNZ;+~A8mpc@#?Q&O z_RTUvSRlN`3oancf~u!H26Gt!9DoXbq=}E1y)5;4E+uG5{r&awM1PK%^h!QvI@b{x zvgT1JcrgN2kFI`EoxWB0?OoFH+5)6DI345*HKGr^3r~UI6(de z%B(s746}7Vo9bNbG2bY+txm1eWT6nlPEyD*LK&`B4LRRoyc#Jbb@gE4J&7XQ=~p=; zCGg?&+1UidHQZZ9-ez^p-W>wGV0uOj(smc&_CVx!#sJ4~OWP7vCJ1Cm17QT*bT!xf zuU{MK5TF0AMj!-RNG}M{sZ)HE#0N66Yrfzr1MUd4CwfxQ4|5{A8AO9fnh2Z{HR3Av zb$5m2LjG`D=<=a+nN#GT(33_gO3Xhs=3K?!j3%q7w31HkN2ss zMWI!w3^1RFK>*6dK@0WWgy;$9u&vP&q z{eKB%3K$|GcBP+Nzg50Tuh7YrT?ZA_050Gm~GkXgA6Y+z)IwrhY|)7^}fkG5{o z1VHK~SVDcMD_4_t3dr1bqwzO&Y7~WVD0W7=*)~qHNB-EF$IEUw_%c??w{s4s)mFE= zA)UD|a)fCj-zm+k@o%#je#w) zy3eyQ(8U1%3^v0`w!ZKe3u}(H)H~$$jg$TrBI+86ESu<{z-J9vDd~}K;z(AcoDOTq zF^4RFP2RG^PS!(`OxJVtiv3Vf1yq<1ovDczVZ4raPz+ncVVrSfOfSM}z*2vLDq0wl zx*kF_K!7#S3%PO!lIcMQE2UNH?^^fA<7+tVOY6_L1{EA=w4d$4z@KnOt}naa{Wx;g z?u<(6nAY+V&TRI@IRW?O!x+?x=79l!XnV1*S*a^j0%AjkLPa6*>mt@{LFvS}kjR(e zB%zqKjMoL{HLqZl9UjYO72e$m;%KGQbS^FtLlbmw^=g?Zk(n_`2}eU5t`o=*U&ss) z2zA^zcLT-L4gs>Jb#cXr&z|OmS$}qR*OB%KcVxtfeZ{6SR7V!84@8Z$L`%RJZ*v`} z;-I-7oPTntdJRb;F=%D^XW_Uzs+nmNI7x_7n$WaMzI>2kCV~Dgk9pS8IaO5GddAu1 zfGL5ny?(3JLdSN60DnF=r^k~(X@FN0jkwdcdYso;9S{ciY59a85ZDPW&yM3rA(H&W zb}b8=sM^Q{lqs&QhksN6BB9LD3fkNH9_Au7lc~Y(+EWDaW0OB>QuoC=)HtjcI$F(x z{T21;m)*;mVV0%rRs`#**_OJhwU4udabA*k52Bbxln#k13o%R~R*j_6RkAo_tv>HW z$u`g}WJ7&=?4=>iN07~~_wXxA^?;YqbPV^dbLWBfT<08aqDGLHs<(pH3L%bhwiTLh zXcW-l)${|DX;VC0ApQH6o;A6QidWv5zh!?%th7myh?u>S?S2gVVIL&1YDtwMr+W9E zb*ICoGklqSH*EJCNIS<hz-7R0giuk(r!pv}J;M)UW{)}0MIm8N znk;l3<*?x#scN@9_dF}s>h=Eu8W&=TnklRDg%ny7q!T4?9cAx&m#`-dt8TA~9^#`O~go#2Hq1X^8m>3i5fkp9*3b)z?mj5LnEyMzq z(urYR9p62~nw#c6%?hLL(V~1g0^ohM_u1ve8fzqMK?}{EtS|%(x0MlM`844jzrjl z)DE@aeXW3_4g$CN!vzqB@S<6*p#DNEf^E+(i<-!1HtR`G57XGuojh+L!~)y&iMO{? zNLFpNftzi?XcO{F|-* zaJ7mrTpSCQt8?nV9~cD<-i4X|WSXW4cK+CqHiyc6n_YR8^`js-EyUsj8^SU6HSIS& z0Lo3z5NnLX5U$h#`jH=7?-z|bJ>`AgwO(E5cQ37EQMEt@>GfwcH&a6E z!mQa|B(P%`8ycC7bBz1@L&cZJs;gW2`ozhpP%(lUk`UY`QszR0vfdPILR|uHWT&o* zYy)91K*yu@-m;FRZI`IG6-JrZQj$(f&pvx9d3!=%??nm_6pkfHxNA;?25&~ z>5xcKB`3=`DyS$SZetkJ0s~YZ4WL*)c7D+tklXzBp6Iz(UR97(t!O^wa5O$hAQ)V3 z089QhU7@^&cYr?J63u-#-{x$>@8M?KpjaLmW~t^DzJ3!@lWLS>_(q}R_#c%LZ~f(h zBklHB(fh&xb!1-nf&ZeclD@utg&%8qoNn)mqD6=2 ze=Zk)T1ln?MLiquZazu0y0DsToqw`KQ!H^jBkI=A(C%D^z@iNh(-@P9O<^UY6&SER?@a(G@4Z*b6n24El1K{=fDZH+!xL>(5oLE z_=Bk@GgC%sa&q&Fw{uf+R;8k|6A4~_)6mrwVu3$$z)*#^^!hDFo1oQ$6b81Q)M!of zCl3*I{kk2vhu?@h?`++&Ec#q$Uw?&VBlp`DOPAHfS#ypgKTZfCZ9!?{_4L5zz{rBj zZf^HKOqe`yV!n@3mjEOsjb`5e=BMKKG(uaJD>>P`0v@Ny(90 zIjiNOm`zZ>D^D|KO}A?Iqc5F+QD%3#CpUW+j%?T{I9uZM@)f1huB5Vd#rlH_C5vFG z2l$W2J$W%KoHroPn+glW5TdlM;Mi)C+ci3Kvx*+JAidZs^F?!rj&re zwW+%?3=RW7}=mD|C~^R@X(0ZWr9q3Uvn zv&2uAB?eOb+Z?p{fBKgYi@iNTp5wj26qO+@JdO&eg5XpZg^4y$UT*!sCz!K0k(fLJu@`o<3xj zjCvX+F21kWdLQ#q}YEEVBI`83A!x~IR{ry!niA_v-#w*qEck`R}G|kM-)AdEF z9*o!eq&p6UhI_aOvB0klDf3{S{+Us$-lx$T8ck(|GDby{YKmH6;^No|(Kg`^om&)W zI^43tz1MB|alhMxhfBSJUI$zH#2HmjKuO96Q_fbj<(A!#6*DhnU z*VW?oF;5T0e@5^fLisrQwc{AA2guwuspFoX*26Pl!suNe4(U|}EknVrb33Em1T@e1 z7hiXsXMoWl=OlM-_~L5sdG%oMiji6|#YOY08c?+j_1tqp_6iUjO)gZWmMKrhc+*}D z*%34Onh2ZLOXYp$y1INEgFum@mPN1{qKnjiV-{CL^tIfpsr+2Hjx`1@uV+&3S>?X#F87H7;HI4QUapMNQApD(MuL(rmfMi zPX0|*cpv3iTkgwfrvS?qro4?jrjZ<0kR;ygYy{Mj?@oV%vX>w9#cAoD5zxZOcdk~6 zW>Tkhws+HN^ozKT8J1@@KKfbWU8XGK+*g!k*M9IVycK4`UFL^6>$UiW9gccf(8U1a z-TMcKr*1U?_oRF)nuFyWJ}q5SS!wH^{&rUYTBY-x#yO#DM^0Kuj zW7>)pggIs_1j$A))pm^R!((+4I|J1XzyX9Ugb^GcN|yw1+@P*V{oAvNcd6`#a%^%* zKHk;5ye3fzkuvdeTpqfp$-=Tk!*jB+J0(j?L9)x4NDF`2Mmm?6EomXpTH&Lsjgkq{Y84I1#j{&mCV>8iP6 zBe{xOmUH{zFb!&9dB9@?=4c=X4;>cYm*zS?-qHCJD|Zw)-0r`$SlyER=}v<6WF&9N zf^89UBd1wU;*#ZX+ix5?a;mtK#z>~-P#GII3G8*y7=4SN&|^NDNytJ&@i^CGEGL)* zj1`Z-*^aeY=f(G$sVQ~s00<ODiTV(>3`jnwduybAszo+|hmV_IlDPA6$gXcx_ka!_&2y;_q|VSJ zz?^tVTyYN>FiEx(H=u3V!B@7`-3;ayH+xKp81RaR!>@^@{=XY%jr-wM*BxC#Z}L92CBCI1tD^=C~M#k1Nzl8Lf-8Q2jH)IpuJGgeR*nV z58H3Wvlsck;FiXGXbqh3obG+f!Im50sHa#kH4huwL0f*ch24b{&$|b+5bg2K{<)Pp(BxMi;_DVWI~7#V zf9%7oGj(OqFtG^7u``Be%Es}h6{d|&D#lPRDH)quP_+@A%iAQMt;kf;Q3~iRnM^fU zhKKcnQX4FQZ(9WGQk7~827=V7+WruZD4w>O6YGmm5BO3&yH=QHna7--yy?vEJM_Og zF#j@XI!MmZoZ9ZlyT5Qaw9Utyy%EkzEA2 zTUnL5kM0r|1BOH8Z=tcL&kZyv9qS+1W4qtu;ZFx!z?8$(o>&YvK5DlD`N;b05z3qq zIo0LebUP#EWnOf?-JT|WX9>wvIK*HJU+dakSh+J#uScEtJ_{HqJ4b1)igmlspnV-%v`gwpBK z<+F$FSxuC_-p`lI(=%Al?e^FTBq{ccH2s&9Z&=nQRA|hk{kCUE^NV;|VmTp+#mjQU zYwY)MQj`=2MU2J{6x~vk1(d_>>}**kJ4?5wXqUrZKJey8 zKbIROL{R^BQPb1oxT7R@(Qh&zHq~rph7CKRKbaCmD`E~;rZCzcILumR-N=(dv zHGjM0A!q+wI=M8jw8zEKv2Ag;#XyrE`;$D3Wa6KGHwn z^-c+KeeEeP4o9hdfjS<}qvp>tx-@v_-W4eK%q7haJjd(7#)AV!il z1=;j~z^V4UkkdjJpC*;R3<}R+Q`g5Kddpkp;$Ba;(OKb(vWkDmP&c^3YfEI}{HT4) z^J8Xj1XMUzPV>8|NE9#>t=^4tm(Nv&b*kO;*5W8nrCkgQJQiVrn1M6lK`xtdU}vc7 zazTMn>EiB4&+bFzF|tzoHHSSKt6pM{td}DB5-|f$pB#w_cXvQ5kq`^WWn<&UFP?$s zCp7%9$^Z6Y*b)uqYAgB)zrNbb(fPzwh-KfY9pY&@YVfj z#86n9hpXai-OSh`j+nqFFE55Qnv_oXdCQm}pXJ+dS>oDPJ3X5@lU3T=SaRO)<_ug( z=$|!_A~l7Q&7Y%0WmJ}v139c}bg9YLpX-`G)o>l^ej9^Bi!W;CY!@guL4u<=zpGJ7 zg;_Dbm5y_q7z_8Y7$j&qIoA3MOW99Bit@#llw;0kd8ws4Zf%B!sfeDjR|n^w`rg{a>&@ryubf6#b3s?7P@L3P0ZeEn1dziV^7VrK zf>RZ*gY|h3t)U2+`c$dSn}wEDx1$*qBme7W_pYd)V@XYQQm1)u*+=ELQX?%nd5ov{ zxMtDABmSFrJ$oQ#cM+P|n0H0ve))Flda!3CRih2tQGiB+0D!4VvE2S*;}`8lDm-1; zFQ}#SNPr&q4Sy)sKX3X&abx;Dr4M1VWyDHLII8wlydUtVg1x%hZobqqr;o`h5-0x5-vqSGp+=nVyEB}fhMUuhVY9#rV^x|AI zO>XHeBHtr(#kLs8QQjycnaVU0FmH6+MAFbK&Pkn%CH1}&dyKtN!(Zl!Dl56uX>F8Z zF+ES$B$YAc_41TY4&Kpmx3_(JPrA^dy!ClpdF%Q;$AQi9$34`dC9d3Y?-Px9HL4@X zdLddI9W*SQwG(~^>Xz8do)b{0fKEW6_N`ZKRgh)b1$U+u@e@FZ*Hg%uK2Oa-5hAxG z{>Ytxdzf)+u8%Lmh#XbsDnG@7XlLyZaIOr2=`Xlh%YsXwHz328$oP)q>&?Y30`4Qm zE3X8KX;kmHYT_EwNn3|eJ+%?_W3AF zs&+x0W~?z7mMVAt&_m=Oa{Ltki~AQ|x&Nz<_j7iRkSGt2IHH3$weo3xE&0#S21ag! z>?W*9ugD*(S!NV-2d|SIHPg>tBuKa9c_;z#xbS~}F^WJEe#a=g=(7PyX21cAYkc^l zVLg~22;(x=7V$p)!n0g(l|KC&{^396P#9Pu1b~Bv)*w5pg!JsaD<|Kli~QeET=@`p zUGB&hiW?SmF|AXY{*(z)I#?K^+xn|aKqu44r^|kL*LRr}i$4ACT~>3_CA9YE{2ovQ z@fV!6t1OYLtw_}IcMY+hUsA1!km}a{+&!@}9M68av&yo_W?OlGmf=dD8Vb*^R0Y?5 zHs^=6-6j|?B}O?Kzmn+QCl{vN?-BF+)KPo!kDs~wp99BuQ^VbI40d2srSpra!Pm6f_1G z(AV!tak)sYZR$m~^$%my6!gQ;N*CS-Ws!rVd!BG{haYy`0!!$iG-=00VQQbyn_ssl zxvn^t9QpHQ1}Z{N{`ey25(rtiDw%NKqEnQ@{e;Y!vTagE&>yKLJwrP1dUOVj2bBRS z)p=9$n^4J6*Kc3wVP(k)vuRYN9jiO+Iuc=#CHELL6E?$&CwC{EW_x1uzuMLCFG*It zb%Er>=pntouOm>Uy<2_~!so|8n0JHgMjf>c6R%n19z(%+)4d~gP;qcfaG_q z_D5lRKSdgYu!Kqn4xi?e0@2U|($X=P%K(RrPrdCDn(q!fh3v_ijoXV5@qF}K9u z5fNr3kxIJ=mCMZ96|iOAwC$flap3um1ILq5r!V{;(%@nI`B?)vsuRwkS+R zlnij&?e)6ysnQdb3Yd557dw}(X%n7!~gMSDc7`+WR}dg>VKIrW0RT#tol-jTl0P8 z=l5bAe|+N9H(Yd#MO+MRsIZdphaIQwSDTKQEF(@SGuN=8zF}5B4100Da`Wyd(U-L) zH=L^|tkzI#aG!Z|kX;%`8s~Yc)q?~y(D|-9gB6Tu58)@O@S)D@zj+c(MW?DtWlk4SczvG|K0lm zod{iqOjnPv2I?x4qxz50j&Mgkn3PwUuM|UIWWledqOUJ&an4WLe|HnzqtNL|q6LGs zYdcyl!7}j6Mm3&ww8XY0E_ptk%-ACrSZM^GI3F-tJNeFEHbP4Wqzk?`-%Iz_S4#P#2=9l4R{(BY*Ff-`FIP>*CPN7F#!L)A#mLG+F*s01|3!sEwlbnN2#{RDIa!G z56<5)+3`X`;pBHhOEU(o9c4%lK#7hn%X=X&aohU!qs}?Mb5-+gsb?!fz4vipkGi`F zT`@JIrSB+r$6(wPOsqYI+w$M#7FZj9&6uC6AdX~CckmlM3m+~Pgo9Sd?%e2vNX~eT zeRkf+JX78d<)H>iy(tSMAm6oFNz!hJ9(Rs!HmRf}gl*E-lj+Npxw+`X@wYKpN__4v z{03+F-JJLu-`_e#i0Q+nr6CjcZ|}*~A?pK|dru^<#vkaZkhnAW=67}Of95rKr{?iY zC8hILU0Yr-d0Y%8`tSSA_OXOnM~X~o*c3b2j1Hj5gBC6%A)V$ejr4J*bEnrYJ;5an z-#ZV3JqDLNYDlnthcFOY&t-6ojZ@yDJgk4Mc$%kVIGE@+J`J(uPt;ur27Kh+`F|H9 z#GfeIq5t)HXI6Lmft7_yX@W#=FE206)RiSisfdfq@a2PwX8Q6{r{a2+5%ee%YILb} zs`2sWrImzOxlGS3Bc$qZOeRY;)w)?vo5@?!`$!clI7br$>hcw(9vjC2@gDL$x?e#A zuvv%4Db0fHBadss)Mf8!JGMvv0bN@_EAI0K!W#2av@q)(n$pj+Oi-X;FwcRJsO#+! zIoV?9Ym`-kypI}}{9&&(lmKor-KHEMINh4;$QfKV=Ec~I4%L;DMqmw1>{m6UorS_34C z@#V2D^~^~*|2!m9MjUB7ZUgcI5?)lAFI1e|`jsW_zw0`H#1C$&B1Ca~wl_L9 z8Fusl^%=&zP-_2rwgQ?c|;Ye#_pvuM&A2d zv)+YazvgUgY!04u=b_v!Si8JIQq1vXgea%?Oe`yBn1fm=lzE?d8Sh8~W16Hrw_ zXY(1EO~1n}gud+PlP6srY-iq-$JTCjM=OAuaEySkb!)iNw0X)C_84R1^R7juOn1jt zk_x=fBedI6JCwisjOQ-K!rxK6lC1ur*uJQqpa%AqQ=i#Sa~G6Ol@~Sz@m;>aCz>RL zYE>r9(uX6aTlE}EUqtZJh{CZ};eo>-O2NcV9v~H8MymUK=kzUODDXQ4++$#!sQiN% z?Zsimpo{KoNo6&}zaDJEvA^(EO5)yooHsJYUmXa)GSrH_hrlN&CUqdv=6w9C0LARL zk*kbg|8$P~`)jl=-_m(kev~q&Ua==cwmNYPgJvGCq?2kya23f{Rp)yH>cd%bQqf98LIOO{-1$55Gi7Wb=T`!4 zW-#}NyTIERT23@3v{RVtz^mRyfxC0S~2-vbIDbnM&v5j6Ky zbQ>_0XEa6DC#&=QoaQ5jg5wA+(b`+dTPSL);h#)d+a(BOkk#fm%I<2WdHr;t^ju6e)jd&aGq-pp-bbAJB= zcEU8Fb2bEGQJyr)PTl0c+xWPYB)9AVSu+y`ng+<2RYByt7;fVRs*Ehqze`t)jS-)q z2N6BtFQ(0&QkVDd5k7wBMCzLk1p8;U3Wyq_JRN>&_{XVMuGznK!fY8@IiV$v=6Gk^ zfz!c%Q-ORCUY_`o5BIV(x!IQADBzv|L2~L_y&U;lFaE}#u!*%?bp@x{?`$+V?LaS{ z5eZV%W-*UuYR<$`N!#?wGPuo7;d#DYrjw20t#f`jCw_X24qPjjicHV(#qp@}!|x6( z9?{HB3^_kNTgSfcs}rIzkV1%w8zV0YAo<;kGheN6nYrsp)5-y!Gm`m$g%aA>&qx|i zr(ZXS@^+NOs`1^1n6QSGb1# zlJG$CTn|~FBvG$tNF2|wmXzms z_nS=gtBUw_Z`3!91V`$k1p{8J(WU}-8)7usb$xlic*;-Fi-pmcQaq=m!}8lEO?Qsr zp5SVlhB8N5inK=H6AgcC=mL(Ksooo`q{oJ=)0gx#MMDUU?K?+8W7Br4;ng@VtHw1j zvKgIRvQd#mvlp=x_|4ToHOvGV9`*{LtGO3l7^!^dWPjC~c=;~xw zP6+aU5h4@NSQXmYlw&^j^jLYil#;B~{E)R*=WG4t)`>;`&Sqzt9}+u+vcFB{(Rm>5 zB*zev`|$RlIY?0FEeLS&Vg%F-%=14! z`ol^y=)Vg$1^Fw=F&H49ou_KDRIbNh-Fxla*}STa01h&v_v_<5oK8N^Rn9m$IwQOt zfh|uVD4iu}+k5N(jaH z<5N*6%Rx#y)m5fG<%NtcPhDry;t8YR_!t3IJB7>VlElj`pT{=&+tA%<3Z9{aMy0>x zx3D-uNCcQa&HtdBU*Y3?xMuc0Q20_VGH9d#Mbj~}78ZEy4ISFi~? znK5Yuy5TWbGiWX!u5&s}@upIEr*0SNSmM$;QEnOI6I$X) z?uqkQc`9FeNvoTypt@wb9xcZAgviKCK2*N_)b*tIJIA8+lp(V5=`pXWkphOXyp8!V zmW6TrL)ayfUQq;oaPSN01fv?HsG?|fR(==}XoA$)EPUmNj5&m?z>t2%zXGg&xSXnD zFG?qouY*)BQlZ+<3d|2xFNjvML246pRoi+A2{m)t%=_Q{a6xo7Rvjh|yNUgueUL3s=uhtZ zPnq(V&p9k>-Qr5Vs(#a3z@mvKe&DjJx8>wp71=rUiq6(vkXq!>6Nq*=+GfFQ(wn_JPkOVT*?t77 zKYU0l8U1fz-+w36c|-NwMQ)+4-Y|9DQ{zv&-AB06rrYzv_fqZ>8u#jVnEu&6psinV zV;h>Az-itL7>li|lHHyNTAr`|lKU@|`YYO(I{{1WFb9zzr1^Xy z%om0t>FSj)OoIM3X}s57#dau%c0Yj)+8_j|Qd>tci&{JBDRFzg{}x_TvF6Q2mHYf9 zypr5uO!Cc#Qp3j!HpBz@k3Ln&P=MMAYbd}MHRu-MuiHc(d!PW2}a zm#m}Q<@#~$y8%|a@!wSQjlTOBC9XW#)uV?Q1@2G7@W*vz4r_lh|M<5u(T;D{jv9sX zYTDqn!+shXQ=g5D20Qo( zj&$_6XBAqn9nmBKr}v?B?aXe^l2Rv|=8mnD_ky2)asZscGBs@pP>av{?%&2C;WiPo zZL8SJcE|OU0~VqM^J@!tceSS5wj)H}R=IyqhMRL)e3;}!KQrIOr}U+Dl!D(C*3O6v zsT1YAlCmN1^I~((Y+Zbnfahwj{_N=@wyN5f0RsS67WZbs`-taoLsy6snxfA%S9e*dTKbM}*eZR9QMruiE}VAQL`eZFFcw#7F1GpMC#WJEU(z#?oLDZ+v= zC&p*ikO;G*@q%}^J{Ep4JFjX@4q}2|XrF?JM_H0;awkM^y&&PmPnwNSLSd&19fDM% z-FomKgyD9hfBFV6a6uvkHiO;|CL-@j#LP)QqZQAZOHVa} zfOP%`McD^DaDeDv8A?~h71JZAoxX+ALPg=~c6`Bd0&Uxl)7%ud*Ek06A}{c8A);sz zB^P0+hi&>*uA3;V8$!_C?}pwEZ)hNuOo$S5JR2BB=eiZ65W7pjOa(4?e@dNLjkNxW zzK)?`ZOY7}+sMo1N+#lX%lQDBzB4~TcN-5{juQ9uAhO7OkS2JD&M)m^D|C8|!T~UQ zPH!Arx%b=mb5LZv)lux4KDr3q4T*HUVov7OS9jGu90qo8U2qm9(x5#8k+`3(>z%^Q zzJz{_0hFTnAiew+I$_CCBhWy%>Aho8Q5DFged0?fADa-hTQcfKrJr~Sm}z&#TvR+Z zP#UZ{ikvebzE+9PccBN_mq-|qd4MstaIV=VwRan41fTO*w-e?>7wq`F&=~!_yy#?u zRgvB5hDy1sC5D1*LBV+KDMNUb^O@9!zeH-LwRkj+Y1=k*fO|t*>X5wfyVzZ)94v@! zUde2Ybp>j-&-UegBq5ONH6(r~V6u_&NXrW1VPNz>(g37&TtL8ndT90Qp4Bimwt@O~ z!PURSKRqh=;j{hg|MV?s?cn_H8vlKE^x)L`KQ|KopOILZ&;OGheO%Z7GZOzZ5(GQA z%9G_bJNEKYLPYevd*8wPwWHqzmb5nr%dj5S!6b4aJ_zl0{vQVn|9`p$JIXWT60`}G zJwcu|vjn6Z2=oSIsUc1aqm^Ay%kyv>(56$Xc}t7-41crc!-=YO!zEu(#CCZs&(jfh zwrwk2noRG4UyXZmlrEZ*GYrb95Brmnqcbx2a_JuPA&_uK#a!HbyZp&}VZ&?1V{*`Go0oL3J>% zFM`QDMtpYThDN@vF$pH^I9BgeK0-2>Z=>N;T=jZ-j^=tSJ6*EJ6pWvRlOHmetOdr$c>dck@*%En z(BE*s9*Rb4;;*1`*#&(y@2JH=6F#(}JE0+0Yq4l{VA~zUs8=^haR$|Hztw|f2PXAGrdnKXL@6U+d7;QCV`SUOvS z)Mk6Z{93f@D(CUo03}EI-znV>NSuE1@&zaZ9S#i*Etq#$z3|(M9-s~kOnwd`P>^qZ z`~L;_t)2m(*h+IqoGrsbES!~Jy>%yX)1~6)z4T*W-d)0?eNd~BYoU4eEUXoh!baA- z-)Vp2>Y(yRKd&pCKS2@u=Wf5_KT(>tC99=U!L+ResrbO=&6@#*6m2DTTF^jaCT7?O zuWx==p|#CW_sF$f1~tD&stG%em!5=KhJC;e|T?I^0vRX>EnO?ZP!Tfu7Q8C zeE82@i+iu-oRi@Q;?~}ANa~h1?BA{+Vt4y?21~mMyyJ-*awl413$3qB-y=EX0TFyd ztAqRh`tk>2*IXXA(80^sbe;dx%?YhH-x`&&{#j1!SF70=2JF9%4^y&0n%+lpai~0N zZ{lkT{ckRg$9Ma~AZ4TRhbPD5JC&Ac?5z5hF)(=YzFm~leY=%p?iX5rwgRr^M12PA z{lA7lE<|M5_EgREtC#L32757(E9HYR1={~(43`#WQ}qNdJ&)amF^2R5`w2K91Cfbo za*|^`W>V`72Xrz8$cHk(CN3wgD>rU{2)q-g!@c%pE40ENH5MH$+a80{Vrh}ST-vKI#KUP!^7YczCtwnCj#riyJ`bz^Z z?oM>pQ<=Hcn0Tb5>=ywql=cWfKhR?Dm1jTjU3vS=)Xa?P3*}iG_}uTfMY7*AvJaDq z0@uNE=xZ2G4Ys7}jl(YEL@uuQ7RCoeLVOEE6rGUJW(LKy05CwapF`41E6ZXSblAX~ zs*)|t;u}aR`w;g?8#qPfy3{~eXAe;{BbI3EH|_c-3{yY$KgI zZ(cV%dqr%$y81b~Wpd1@TCV;+yQh5S4Ucgz+Qq%6=(F;~nQ(4$VCS{V^lGhJ7loSCYN%;_)FZ;R)oH0}&Gkj|W!n|XU?@TchYFs5 z86jb-;v(!#t{pX_BG$>F9%^UnPx#(=2r8o?4id&xfw)}fb}(4qoRsK&#P6DTa7fp| zS1miOs+P4zc=sNm3VoAtN%fR7VLffzRbB*paXz=M7K>7A)(dlN6noHN;*G5o2VsjgL#hD@62jq|e^1uiAL`WJaMjP&X<9JmTRcwVG)PTtRS?LBCx z+_Ed#+%y#xp(lr4~iS47X z$Y6s4GqXRNoY`an#68%F>SKt2x!M9ePi{fZzJdr3?zlztGiG2K@D1rmAXaRUG4Y0s zPsuhYUf#TMgTyk>Ue{W5BlLCO0AMR@%3(;lgLG#QS6Q-*E1FBDa{jVuZJJb7*JZ|$Wdl*3 z+@jtCOK%QzVhwu%x??Byq%RR6bdkR`fH4kpdXy5CsFa{2P*)Bmy}hu1@WNe}LCJRq@R=n;vdsmn|U{MtKPQ8XrKKy0U{xtB|)x`BP3?vwEV;#)xROzm_r=IZX ztM(TPRXCsRDG$bk^8nyYWtrJupXCLG=q$=`9ymiTwXGuJ6WDqIIMLPolW;#93ZD+C+NvO zus!A{=4mH8Zyfr%l7s)Nrh?x&TRV)WOt+N}a7MT3d{aLg8ap_XeBAmp!r<^GG2;TdzwN92P8U+Na+1z_9~`p5Yp6t5 zOec&+ZxRtq8_1oO{q+039hi+=IzhF_iCPauw}~hjxu3tdX-9lQifmO3YcQN9=31)Bhu8q)98yOeNIOF3p~+SY0_@R5;A8dic~q zPT%V~b+>NR4%{CU`KI1kx96nS{RV}?gOX=wC$H3xM`+b?dZn1#HK?BL5s+>%k-9x~ z#Lh}JHqF+1T&CyAHbeWy$nhPDLmu~o>=o*F8P{=!F*`?DK0H7?QQ~j>NK6ra zZzjL2jKD(+YvkUp_^#V&akTiVpp!-d! z`5Z?aE-fZ;HuQw0R$=oU{K@z&Y*$KWW3fSH^zW*nW>;c?2A_Ckkrir4B9(WB2j(@Z zn~8!y^~M$W+~v7h1BB){_#FP*ntT;_srWRKxuhQYGIk8P$56s2Z}`;FN^A~y4j!z5 zhut}$(8XcL(sN|jbOC7IY?>#-NPs7xFIm_)iGIA`D+p=g1C$_|AgBbm|2Y)Wo{f!T zPr}n9e5g@Vj|eRMQHXl$&rJ8cEE1i>1w6;Pe22UIO7*Ce42K+q=r->?`5>tvDQ6@% zBHebfy;1d4EmvdX^1SCVuf$mwOhfGw;$9;raQt{IdiGRRZ`JmeK`$qcERBdDOM9XD z)Ai;%SPn$w=(jk>iZ<(ex|>;8$Rg28(e}<1^x`in9*{Q1g z_*_J&VNJa;fs|q=kCcF&s%Z0Jq1K|*+{L6+1cI@!@XqOH7kU+=8E0=QY7mr~yw^Q1 zG(R?|MBCE)qH*($p~>XT`Cp=Y-db!@+mjU}s#;qt#ciH^F`}jrC%5Kqcuvm^fV(zU zjP8Kg8$ZIXC2dzs>!fyZUwdPYd!)LyV~%%lrez7@x3Kdqua+X&Yx4QwO4qx zE5EhqNw`Mt-iY!id}1WFC~0nsJXn|LA}ZcLTjC}YG4f?~?SMqHmGP?>rP8X5Af7A} z9$~@2_TU7S_JV}{b&t|KqLR64XUyD3;+2HVoDp2)kdpEvt)A=`@?PQ#Z?@XNk)DA8 zkqDDS8T(2+FnQ{qJ$v>>s-71n?`b7-{kz0H+$Q=PlMTGdnU6~5ZgS=fLTi{0Zn?YA zC_$2peI1Vh0_{gHVfxHYVXi>S#w1l6bT5`;)$f3Trh7s&r)}{;64;R4yt7h#$%ZJZ zU-6b1s$U5G^V(O|5U zmU6;`%$UqCOfPp=mBi+)vq^jQWRg!VI zqnbKNIDWXcK4)&@6_Ovof%p*7(od?ck0gF(P>P7R6T~M_={AEA8x45d5%uw;@)tv&gQOBzbEYP6bsv?_AEI zdsVp-w?(yX6<~?#_{|qZi%y0J+NPf|Z`sat^Hqnt;|b3m4~OP4x1_ilyODLTk}6r~ z$TvxT4B()BtQZ=F;QWYwT{9rzgKgwG^39aOGzir}t$CkMzPqpA8HHIK)1j8qGj#>W(2XmKun+nQ?Os?x76)b!kV>ZwFueCGU^ zS+nS*(|U)>d7YvAl`Q%KeAIxLAacC_OOUnd3yO+mOQ!LC;Jvz@i<8`R?QB!UekAgR zo*aERL8Xb%vM`ox(@cvxK<5USQ8R73gppJ5g!#|c<)%x=e+C>7M-Y!o@L_ltZdWUf z;Qt-Mck}5IaVP`Fp;DiAr3I=yBk=%Xc~dd|aSgNjZ&Tvagy+qvBsdNQI8oi4(aO%z zZQ*+n=Y~A;#CI@AOXsKnmK=8C<%Me#n0(oDOT-%z89^B0x#wtL4kk#-1X5cjDnYN0 zV?Ip~m0)gcVBFXw$kaWz?KYci$TQAbcsg}8AjQWcV0~Vi-Iy;_m0{MHblj%1Y{GhZ z%9^pmt+_*A_^ALA zh|pMzumqKS{nTNkjX!?1_wwmC1zNiGrA+#l_G$Ts2=$Yl<08-fxDmDOF@w7^luGAd zB=-k#Nt{5I6Azu@!<#{MamRN2RO?o4#Rcujm&!dIt__*FGWW-d<{=vpM|>nfN#G@K zkG#cw$N>=2IF`=04-VwH)D|Y2&jYO3UpKwi(gQYp2sF@lBoA}#^yh}?sM#XE@cua3 z+p~%If!zfY*Tmi&gFCs2*;3W2>KyhW!QEo%iN*_!{MH0a8H&7X+SaHVT52m+SS=Ej zbZI=GKWnRnTHUb{vJvAaj#!F9148>9(oiFm8$z|))SFwJ6;cf!{OB!HH4zoz&N2 zBQr_gsnTEA*rZ!;+vrjg?fO!yX_G|cHDpnNy=|HEw=On%U)PG@^ zfRl_blEiP}^|nV{$3}uU!j3{*zXF>;>m_fLYRawQY)5ca;^0DD^rvY`oVXzPtW#J0ZQnH z6g2=M3Y$lKq%z5&Y*~HD5Jo4Qn8U=`*}FF|ZleZi)cC~%!QNDMW!Fh7^oqn0=DKwm z^KH4dLQLf!e*ZWoXb})JJ{}iY(RE9hkePCObIY5PiJ4vPsw?TQPR~Rbi77F=AvZ7)S8!IM*kf^KGT&hbSt$ zNZEu@(&MnH`y^t%HKP{wv*=~=q<_pf5VSp8t;;n}n6ht}X&S^M84;oMICym;LQt0M z5TWH=^?Ldiv}T?=4A(7XAzHPLY}qOM8C;tJ2`de1I#qrbu1Vx1tQnhY$?YT?P>rJ? zkU5f`|EmU*`SqAtL-@u^!C3ZJvk2K6h{mH;zJUN*vk}g3kG{RT8sTC~N4t+8AyX_n zqTS4i-DLLq;{BHX#&8U`ZjSkKt+u+NBD^a7xfNb{uB|i3WPrG>K%dh^iGG-Lj$Z zIrjAT^!N;hRQ@VfH4F<7@o9Is4q7awD7zW?+S~Hl_gv?64|terKiPZ@8^q7@)v7-( zs~->e(=w4(v?94!crN6+w2RA_S&(CnR0r3RP4jBfGx-$W*?l@ zX&ZWlQ<=3uh77$^bUyT;USz|>ux$EY^s8as@IL?cya#x%{g@Q9Njq#h2Ht}VsU8qu zy6uoy0l7z*6NW4>PCWQ2>EgO|L`)3hH2Z;Xk)_?=oUs*UgDEBHly;|#&R`BtlueOq zjhoKIC5?ItWtx;~&UT~1uS1^j0eO89b1+_%@w(uFpbpR`s-`%qTsQ1I_C5h*7}&F& zgoxJs&B)`wfeFS32b=J|H9OBD*8oPXnaKSKUNg|EMt&+iC%uaK zArAPUCt+hX;96GBbQw`@78Ui&M5Jo{^l6U_v&NLV-w?jr2B*&7S862Qk-$*0miMk0 zuf_Z^=9S%+A1Yn7s~Uk9F1cyYlUIkzP*K~f8d?fc=C1#0CzNqkI%@=Q zqkw9fv?-2|Fcypi`T$7zj*}xEeYR&s#%_UDC1axf?{UmbVhTO0ufERjz{w=Vq{-ad zSj#W=j;lb2AHBVB=9*QDb_Yo)Nw7-C1RN%Nj(5*nqk-BpjdzyGYzQs0)pu^2M+%be zTv(*(BiWe+k^=}F7@w4l^IJ;l+u!wCINWarz6M{&MOVe6!oPT zIuUpPdNd+1IJ-Tdkv){JA_g!hO+MI+WA0BzqGi{G%v)t^dtLIc7Fz8e%L`n!;hY)Y zEkq!ZG+9navyU*pv6YHLd$(Cz@a#NTd)msTmCH&zOv9uVjNXkKM0E(vLJGc+8Dm)sYav-)HQ(h`;2VqB;8w_Xe2r8;-D z+Qbxk*-wVo?e5$DLFVx6_QKJ7fo!v9o2;4#44D;b-5vHXpzPq}$$RzLmIsQq;6}n> zJ|MJ9F?>f|dg;T*mYN0v$=y)x<>m9?JlnR3%F`t~5|w85yua-EMR3)Q+#4_l)B`Hl z6K}Z1eeQ{bU$up}%!YQv4nzYsNKShHh)JO-TA`9(} zT>EX1j?NFW4m03rYVLwwpiCzqlnjt3NsCC3F)1R*_A!cnD+hiQyywDW5fe6IF#i#3 zyW?S0&rW0elUx=*$rd{7%+HaBai~j-!U0WkE3xj51O%_7D4I1sMEp8H6)PEMs3zzQ zFKUMc9YiVa3&2otJa}E?tE{91^D7ZnS@M_)71smD_HW$JcLS+}+#5azHNh~zjb=cF zO*#*|%q^6HBr!e<8h%8N5}o6!x*0?gAp)|s;Zx}yD13aA9?G3d^x+e~X^t!5v4}a> z`W&l%d_2YIE|Nk66d)3md2=d2qi?%h zX3;>1fZ@4GO-p-w06YkYlpGZ2RYK&AXyni}vl)W zqgaLP*y(2}dxt95aDz;be(dc(e09WvKXqKAIrpmc_zxQjsoe`Io#U%$V)@E?!voc+YmjvD-WsElpxP8t~}Q$no{V!=!b3{Y+c->qz+7K)Zby+@?)hD1+ks<-4&tmn5y&U3 zf$$0v!3`D)+~7Vv`2EUV*XBZD(QO3!#~8osiHZ+-Mvi-HPBulR?;4{H;n;M(A`3#a zFWKR%c!>Y(x7Xqr%%DnH9ULw&)xFtdLGdO0zCVGFg?}s6zmwp_vMEQ=o_z4-AZ&<^ z{B6bBulN-&jgtI|r_c+K8+Ay^Siu2Tz(n`iNGmRw3fikw9SsKA^aiL5vL}fu8rIWjwjbDtO`J{jgvY@&Hx!?w;x4%s1@9MKiw=wJMbO*Pt$^ZcqB1JAVB zqz2dMT<0{+@_;$1!xl*$9V3$jtglA>O9fho*tsTJ#vRc55h`_BQBePgUY!d~x{C{h zF?j<<*%>I58Hsm(Tg}K%cr&ZFxAOBl!qho^@{H`J<}kmQJ4Oui{Dqx9D#2Yc%7)pu zJ7wJJSOT?cZNVIFxlvz!UrJ(sm%Qhh60y#Th5~MM`lh6g;PsnUmOd#TEJ&{h*m{{y zEX?R$1ieZS)}~Ck%^*oRHtix?rhaF|u+H^EVF_9{ci1vrnl}OwHIMNSZ_5@ph4_yp z=d)(k$C^zS6mqAh8(r$n)D1eu&Ax85@2~7fRl_?Ok$`aR$uSDMBkyLdFY?wBH6vpw z49_1<*UGN1CPu4}ln6x| zQfoVe#MVCMlmIDw`|}^%aSrD(`S$J?;Cf>S@3>H@9?Us#L?MP5O@K%M6aZQAN`^T& zkF&fm%jWfOD|8P*^ypDqiyY#D&f;8i?rP0PA5CF=wF0;u?wXPYk&t9@#i)i#V(w6j zU0RUX-5sQrmC!04c&W#{cbIsykITFeP+S@y-@l{!PNE1)G&9+y6@UBm7i$JOElq>> zpQ$e~ENbe8MxKJCM1U=79hK26Oc zteY~aR)>OSGoGzDvuGUYMpBv~Mea47ujU3anm@?2;l7?G^qeqYIY?@Vq226NY?h+# zZaxqD+ebrkE*T1m_3@rMcC0YeFEO!2P5U`r?UXehjk)SC9P;BIt}h8`iAXeG`cUBE z%KPdym{vt=xOFKELslZnvi=|O;mxV2Q4E}#?TyPNrug*o71C8DY&m?UyxVK4L8i1H zRI|}BPF?7(x_xTpB_5p0kQrk2>HI*qnEP3UphLP?b54G$qQ4XPTE*rG=@T(;!`39p z%lyOS+X)mnL=KU!i-84|%-8(8ycAxQA7Qx@bGxJQ6RlbU_IP_oMN6XOfyeY8wZyq8 zl5ES;?*9FEkFh#6$D+d>nQ7zUQyd2q(rtrSWWqX{D|-H+R)0JIBuxbj^px%%hpGtA zgSz!1D`v4FTjOy2^r5^7#zjx54s6~^BP`BUa9Js*Z7#%=$2c~4u=rt71s8M4i{;a% zx_J8zjOI&G592N?|4;3o{m;V1;k;wI+~Al($)J&Q=CF;6h7|ZXk(%7KVuO3!?g49k zN&fKM7XT-s%}icpS$1mqjJ`G(NYWtzOSdYqGXBMuEvr>)n5_|ZX(r|T!OXIkb7nc( z(FNT7oh$nvd2X;N?>nX&EaLl3=IsvaPl#?bad9pNw8o!7=J#YEe`b?|QD0Nl`J_xS z5i|#+ge|}5Z$Al?4Vcv|=8zGodY!$_wa73eEOubFDrtO2V@r8$iLzt-ixMLiSA|ig z*k5*($r5gme4!;ulx3|8X630!)xd&J`pe9j7#a5ATh;LRohV$YJ+bVAFJlBF^DG%R z)`^u2or{cRr^J(g{)dzq|NHg7H4eYiZ~y0C^VIz>P3-7b_`jTS@MN2P=wS9fOxksN z_G>)V7T{6N5j+p4-L|fqZ<|MdcsDiS)XZtbid{iCkg2w^^KaCkCdXTo4~=_A%g3lB zQlfM4<|E)B*E$7D4V<%%XbWBXQe(?Wm8!0(Sxe0llv5E{v}WILr=(yV8ZoJJ(?IW? z*p~Zd1YXZQq{Xwn9RX2X4`IIGd21j+=>6$$YoZZ3uk;Pn$@?FH<}D*??Bu`!5(yJP z-UGBKXrZh7b(h?Xp!KT*HG45ZEp+|T%>Pq?e@CG!eLZ(Qzfp}%WCI0BLyR;&{`wmr zH@+|_t3~RuQ8_!h-Ih3Pu=KFAl++Q@wHA|IR>^+$5lPvGF0_pzn3H~xKs6~$g_Gmf zMYKX3gj`vbPwE?F!b-G6Cx2L)ujQB~`p38co#(F2=?co(kD$2|zn%-~^xDX;dJ{@0 zi3~rGic@NZj#lACojG$RD(WIk9H_?`tA3lrdwO~pW2eiz)e}ur@!ZWh^1CTSD24|ZNZEZy?omfw3fx#9D@uoqG# z64|xs@+JiE7iek0x{$K@z14E-Q7d?UT*9?0E$mwaNY_ZUz8zX;Jnh`jIw*!+|Zu0icS=%*)HmfAt($G3=N!*pNWz zfwja~&kT=1wWXCidN_`{f26OEdXPA>QApKMr}s>EWMl;%3SX>#4T^@1z!3sZPNa6n zL`9J*+7}d1U9o#>4>IKQpc)xCQiXnL$nJI%6uUwp;3zQ0<>M$o>Hs=ybbuzh2W1iO zyR8|pp_9t)NV!ORtSk}JOYu4tc;WY3y7dvxfF4MkW64Sxti^SpF-PQ)NZbJJRtG$O zJ>B1^sI3@xLv8r8&kc3uD6cpP5}J)OSzwoo1H%{kFKJgaRGsJO_(QTQkE6=_im6>E z-ogpG59%AQ1^hQ;Z7=;C%S0xtm9yKr_P$-2fivKVAA;2|K+j{B0}4=dYTOrjY=%_vzWK?8T|7H%!PnGI`_n2pqTYU^qkX z>We}{q_N@cpMUy$wA7Qv5bCksRq^5%63P7D26JrX+`E`)TmLP7?p^HB!Y+tD2fOO#2CwnfFfOrg(6+LbW_j-MB*SI-2&3P z^kxI3iwKA`l_p(4dYNzSF^R-H@4xT6-XGuNrI%!!x#zym*=O&y*IMV~_oMO$moDa7 z%)!C2lyT@sMf?@Z!7=a6=L_)4Z@;ST#$V#r`_-(Eo9kQKowC&9kUM34&cxi>#PIYt zwtALUhUR8Fgv5nJwtjQg+WMT8gs`yb-)|5yw=@v`R(@VRF0$y{A$2Pbj^(H5&nJ;G z5r!O}aBwhw+^cjxsJF@PnsUvI*vrZb+n%kN|B2+ugZCrPt~_>s0oT^A|08?r`C6lA zj~=a9!X>-&z_Hs0S{0W1{^%K?_T`yH2afb6aXwnHc0d2(PaoK`UIs6E_{Fw21xb4< zHDb$}3_?@#I~%RWqy#cUdy{W%cE^2u_;t_kS>(+nAO0IZy=>Tfa_QVxE-?S2HgE0= zH+Vj)o%`P$n-~2g`i~b5|M0&{{52r|8;ie2f`j9KWAVSSzzF;=UwjCI{~wG+WOd$D zf1rG@gsAlzr%xh+Z8b_tN<>aQ`1PVt_+F=>Vm`x`Jm>iQ*~yDSZUYxwSyL^BFYX#P zJM_dAyhw7HJLM{&&UPs-Y$3|t*pteMa#Hhu!uWUZdbkT@0Z#2ahve_8h61wWBZxHwmRpa>&eZ1C8ed+nO2>> z=`AiEqN-o4-L*CDM0&*r$KGS>^CBzrG1e&(YDY&B@EGbgHOeWK(o=1oV$G4Fwb%HM zi&(#T;xypcA{(O;_cX_T*t+k9X#RMC@KCekfM9ZMbFQPRS54%x+e%7Gx^bzsC3rdb z>c)e$nU!V!J9d~nPc%uqzaT0y^6^ty;b5COEwP2%PUo~;CfXfzA|{+CyCVECu$;}T zvK^IW6_Nu%%hpS56=@sp&mPc!a$~Q$Ls$6mW5=#)+17_w^aS##YS}1fS$8opCwiWu zwQ6q+jGsjwTeI`*Q5vnCrZ0B-`1sW8ddpej%AF6F%fwdMH5=<7RPM+H-m&Ve5FdZ} z%P_v@*;I~#L|0gEW@kv(^Q1GmW3K{xst$6O8%6~CVjM)R`Ij15be5P$r&LG?VBj2c zr?{=kwy)WK`bYWQuYUc4TY67}dHxJj!0eM3X{Lemmhaid7wTjgXL^v^**e5|w0^qJ z+;ys?=Tk}pc_)00-mrh-7=PWL)f3;F)?mgjER2Ef^^Vud*6%3uw{N?8&=M2bPP4p9 zRkP}eUTs$ar%o9rusYjrz#(qic7uwbcRh9VF_SBj{TW-cdy_NYJ(Z<@-ZRnWt$xN+ zbhZ8Ep`oGaSIb=^t5mQG5*!Bbi0=kkWU!1XaJ`_;5J#c>$(jTav&IZf+j_kbi3Kgv zUec3i(_R?G#3q+)5r1>zm*hsv()M7x=HfG6Qog4iUTlB7(aYq`3eJWT=D9vm$0PLL zzPzt~%59a1>5IG5?ZU3bG<7Yk*)ba59_LuTkn`WwX zeAV^+FRz!}rK)YO*Pk<7v1h6`g|4(ZixsP$B7%TfhjBJ+%&>ToYV7yw;ibAaRcSBZH8Tsp`1%jW{!BjnF<2(?=0;h7sDN<(v?V=*Nm{+W$Eru` z#VvJ(@~4OPyuCNif2W}$mzY^lMX;p7%X>df6f3we+kH>OHy2G}xcu>>&{~tky$3FT zopjG7)MsP#K$<8E_3nBHKA{!55E{ ziZ%qIMQD$_eWm2CVfB!Q&h)K93kC7^Ye5!;YuKw8xkn!lT5b^P?2FJ8=?rysiO(H< zae2*-JB8B2+Ee~J3_2*l+WmAwzT9}AE!Q(&ZViGB`t+l*GO2O9Yqq|n)~kQQq|{mKRS1mm?zU(nYVD+5!UQXPYvs* zg&Q6VW%n{eXL=24Op&Vf={i`4LN~V@@dkUA{u3N#b5pU)zCy|Xy+xF=~Gk#sp!=^4VN4Dx_0;*bkya~ z+T>PbqB5umb_Q9=vpHw_ znWE36&(~sRnsJ>!9q{CGZXY}5C#3Z>)flO(TvYT^vil;)VaF>z^!k@o?=qKhI+0S}m{sBIJ*HYb2j zXauK65?teKHY^cw9PK^gFA}cp9Fqf#VLw>FY4!I0!a6m*_O}n0D;!%{v)6sTKQj2m zW$Ob_Y&Vt%h{`sknWkV}8MHmVTH{P9;I2{478;+)(Y`=m!4vv!t5hC$MTCaj+HADW zydp?E^X-ErDOj3!)Kl~cjVu#y`@?B^uqd@K52Yx%VD$+Kb;G7Co5_(LKThrQH&yZx zZ&F&>wsLq#nvF)=2Gi6@8Yr7Q1@pWKeLS6IxoDl!&0+OFD8_o?v(12VQXVW8(0%;J zx1tYr(<0PJHS#g6zEktUu2O2O%(7I_uRg^%vfBoA=OH>2{0GsI9FJ@Y~S&>x(ap}z^uP$+iq}FN( zln06_(1l8T05`25;QOejT$LS4MpFg`7msG~*o z!ADg)v-wmc&l+i`^C7OY6G6x@9-=B*(JH56E5`|VNZ52=t5G@8C+j7d5Y0X9h1$s& z^2?v*B^W?UN8?nguGx)|(jdpyLQl%6dciie#k_u+%wlgoKE}x6Q$Sd;R-yK)d?)^> zlws`wbT~une!548Re~7|-gssAcQcbCr35Z-?z_aU3XAxH{y?>eX@1o?8SCp`7F;RT ze7?0iQG*$am+SQdO)abC#FW#eQ1JbCTby1dGyRrpYB^FdB@WLldDJ2{KDDZW zOimV|wjj4VMngGXIwrKqWvZ_p_#ZG{<@U_)d?AN0Rw=#7rFZOyJGvS(twvj1SzOMT zX`^S>FAO8X(IrrlAL-U2hzIOtRBlxrp-_43;Cdg_ja`Ik35D3TIH%Wc=M2khSvnjO zU^k0`-|_s6m)xJ=ycb1cjf{&^+^k1HQ4B!iDs|J;8d`i#=t?SqWV_e7Vu6YSQ>09G z$3*1h>=8)Fv2;Op5guCe@|K6tVT`l!WoMw02;5ZB21gT{hTi&=>MWZcu9&fMM>z$_ zzPsyy%8}gG#)Rl}goeiC9WO1z0Gm?=l+ks6ZM;cT6(i%1~W+&_Nh3;;kRCz&o-+RjN<=Sq%)X#zenaCC;n0Fi@@=45mZ3Ik-8w|OINB_#;&}3WiO*7j zz&e1Q4l{$!zy6(8-^JzJAwpVNTN7s^LR}=NN#hcYp3pI+(W)ej)g6-{G{kizaF^h! zRjYsk@RO+s(haq@9*qfGuB9U%*PPV>m`#3qSfFOji@-W+C$OUQk(EnmMS9Ci@B?HY z4sqy?;+oN~3<>T_Z;@UvVf`4eHL$-ayOSVFR78lz?UrvZtz))ckreu~@l0-UnQF&S zC}5os=Aci2TB&_BZFH}Q#3>}C)uF_*8$4-CF@2Uu31yw7^NPWd8UpCrc zu5*S}#O=C=nvGE`Nlojj{o+XK!(;g@DYPf?JA&-SUtQ`XP|6<>lzgwI2n*4hT3lM( zz`ob0J4~*nVEk&+k;j;CON2{Xh+}{8t_G9i3By4WHbyRk9y)D!$k=79t?|Eq#ix&6 z!X9Op29TGLTj=_(>R`&Ni)&qOZs)wSWwXkv&tn$9>dzf(=H-r|rJaJZqz`t2Ix=Qi zZYQ%1qh1O4JnMGF5d+zYoK1a^%6hKA9R~8!Gw-Uq^ZDcgu7$FuJK5EaH9KX{{Va-i z_wCH0*-F>hg~Xz- zHt9b2b&)-5W{jOBE~#}do?p>}OkIw`sMX`L7LaA~U6(gH_Z=8eC!xIY*VAtf%9PMu zB?K&3%5@DetOFpYf_fNKO5Hb4Zi-I!W)I{Mn=t$f_w+aUvm*)BNP9qD3ED+EK@;cp zUfDDBnEElweOAv;U80rz7|`n2Z3X9!L7q&ca?8ixUeX1=E~R)4k#f`rW<}P`Io^8S zs-x_E8#vt|R*%591fHOv%Y6Csl4K+{;k(U(Y0tmI+oERI@fKqMp5hPJs|;Fi>=pk7IAr!3wFdMkh=Ih4v=-pp4&QG#X~c zT0--jr)Z|d`|eg-_9hi|pqrPg43RRyaUP?44or2ZY0fxqun(IAdjaFSXF{(({xdrj2dpkk}gwbzX_defFEgIq1*qosf zaRBb&?3S7M+Ps9Ep~It%R+Yrrtl4FvR;%MWb0kz+s=*{h4t?HLsNMIYkX$KVg=sN9jn{>}YH>>vd%1aMSBD;ihc6$}onvk=ro= z0f7{>i|urk(=`hzxhATSwgsxeVv|CMCJ#T=6o(ON0Ni;B&||6Ho-vS_QokB@{^d^# zqZ=e_d)0XOH(-r+r=j;-$#Cx|59Bn@9Z|8Fq9XAfV-SRvO1f!E`L6w_H}0r0v00{o za|NjUDzQn0Ts5Xd(#1J*-T{0on4KEXiJ6Jd?%%>(IfgO&VGJ!zE85c6oCS27uM5|j zZ$&$STXPugRY&)1d~+(quGxM+^EFane5xth1EF0_*41wPtlR2iU) z;%f75AM_Gpu5_Jl<&J5H&ztQ2j*$es%}Kq4=PdDsSHr#rei;Oxwn`v2@rNWtFDJM2 zyJNW(F`M0AQv}6lf*UBf_tU(dTvnyaXv!@RUp2$u5V=T-oW4fhT`K+mY8tnoMuWBR6yZz{3+|fzCrjFG8*5+N$ zPyTW%IoUF)B1#Bk#TEk4^Ud;6`7tvfOWki>cWfE!u8BFM2paL`b1 z{(I~#kKtD%fMAWOs;0&Mq%x6GL?XT?eY{j>R_qnOP-M0XO};vg^-?T_w#`4`R%@oD^a52tuEX4y;w!L;@HalgXn)#4=HM%f8{1R zNyvZ(&(n3dVobTtV%AkF%bFj!)^mSICz6N`4}U*u9ao?7fO<`C@;Adex1%fjrdw>X z`?HLzK*My`DM(M~;x2&2jNdh&!WyqnF$(!w($?hIN)Iva+VJl4=mkuPs_Py-b4C0P z#lgtfA?MCPfVkDDnZlG8PNT5Z%*1EBcK?o16{DKaod%|h)00bd3$laq?K`Ey`8kIa zr-wsXnkSo53@U}TY}w*Y>uCS6vRpARjwgsh+>c20K&D@f<&ZzqXYPRyuOf6bk~geV z9c}P)w6R12&wh059(6jdxmkVn2BCt?mB0vtpY{m2XXVS{<(XT~Q=^y- zILYHpX;|+oaXZn&Au$f2cqxTzV1Lc^_V#vBX)l|aV}6qrJ~qw~BX}M)P{zi0m8D!O zSbOYVxp8740d_@Y^`Y&B{$tEQ}?1aLLu~)te?<`v@qqk3k7;^Y%%(R z`3?T#cX${Y>1J99vzQD!W*cxtT-9uK=ba@wER+uo-v!J~Z7WFfZ4431w#P>TcJfm(^mIXFS8NJ2 zQEVDOpjHwsc>kA$5xAs7FV>q-Pi#uXcRe%7oyYyA2J&Tg7*v#@0&1PvpZm))iOoQ) zHDku;OwmcFHv-~|s+3G2x`ME@2ro{Y*7C4lZ~Oezhy<-(AtWJoGX?T+7vPW?HTQUH zlEeIX`;j+Q8&QiATyjgc#OM-Z_uH2niA&&N40n{rq*sC&CBO%~975T+Hrh$KF5HMW zb3zIboN6ky8FZ7STT_nYRtPYzptZTYPF#c<4BW$|vJfdrJb-4K3jw5yOV|2>vQMa} zJmTDh43dfTN|dmHkFrXX_KfiEV=_=p6WXagySZ;fV)px0lA#PAjsJ^ z+jsI5A6Jm_0e!(k1gvR#lP~^NYT=={x zRi}=Y=}=p-A)0?rS7KK^rGq7GN*-U`xC3(x%pgU$afkg-8}VakPp5yV3^yu>oft@` zFk)B&W!0)NV)K?sy?erykPfhLBxrZi)uQiGkUf; z&@@eb7Vw>iLH!fHY&!u>G<|QXZhMz$*~A&w#n%kH;!Hxc=tJ*{amj_62c7KJqnU;EQ2JYkQFFqQV9au!RJ(>EEPzy3f_T-w z?M&DfrB!QY-td;;i%*$Ktg}wg(&AHQK>Ervb7m)d_}#xFtw&yZqD{h>#+0-thZ?B_ zU?>A5_S6?LNSNJy9zEC3Tq&lmjDj8YsOX0x`Q&?poT8&dkU`VpB_cP=wl9FXKonU! zW*J^5-G7)0>BQRA9TH(Y}T{@me5@=qcW)*};+N=>D70V=hp` zB5OKJXMf9;Mhb`ysKsD%^#M|pqtDi8b!oz)w?caEAi@v{E`i!s_EG|(S7IH?&#Svt zq9`lq#iGw*xxzuHX*(z2kDP`C1O;3 z0b2?0B4^kVaLK_s<#)f8H83sHuh21j&vh>~?Z5%wV&O%u>QKLyTBj|E)oIJr4Rs!6 zpzTbqmX~bTVNI%f-YA&{g&L|0{8@V8lX0nk(+Op{cDMLj5>l z@@7}o%pO2_!tG`tjdvP8<$;*g^r_cao3{c9sR(v%P~9X*lXS%lLuUi&(iYsPMy|r8 zW$T9^kv)rP-mm`5Ny0o7p&-1CM|`01gpFxOZeejFM*~*7ciGZzsA~49SBQx)0b^T} z0FH0UByXal9vrY`g?MM%^l+t4n|Qfv{z!L?0OcBxrfgQkZlJlVCEt}M&%teXCu>>! z;$UJ@p{_>G%7?O2WLS1iTC44(r@s^3G!k0G`ELC)Gz=Oo+!Q;sj_14BM(43s>v{;L zJtk@b0M`KeDak0*5qCJYOleblLM6a!&(I-slE{KvOQ!2K`8zsC$ICejzTw)$07)(a zIO_zB6-~c_h+fGx$ig-^i>M!_bWNNrRP4BA3_b?vQO=k&d#q{Ha+W`ms3=tzZ4K{3e;<**7yWX@c06Jb_H zJRbm(gp|X(L*S{A?2|#2DTS8itX_S69U5a2`q=asTnwwHf;vPYgRMjFZVw8Y^Xrw= z1X`kYbRgfveR+5qA-4m*h8#2+RPJ*3ZC6$~VVX+PeI^nfCzqNrUdGp~mi$`L&#Wn{ z=5vv9^S;!&3V=Kc=!n{!ZTT%|{6#-K39zSm_NP}9TiX)r3s2JfPPByzgUy? zakw@D=DQT~$p@*O$%9fQM$cOJW(O;16kLK+)c2)1$SK|svK`7h0fn826~Vf$`P`x` zNIa#`UTMBSCre(_C@MU<KKEJ4b}YI_>*jgBv#9yLMO+7${LtWKUM)zzgBqN-axelMCkqp38&5^(EQP#!Tl~=Q-S1C27(Q@7`PnGCyIG zw4a%|H;r^xJXOW`U~5c6e&ofBT_YBx7h)w;pbZY)QL%d?WO2sr1D=+kn(8K+6m^ya z3KW6LGEJ-BLt9% zn%aJ5K(xv}!>3`zKm4+RG4s$$nfeiWJ|bqm+yo+V)@QY#hEhUf>h3%8!2w{SwpxbY zMtsXKh&z1txn&MI{0Yzmfxj%5K7aL0s$un}slId_RQG&JwDpL6E2R3G&tpl-BXP*n ziXzXO2dL+;h**XdsI3Tt%8(^9r^bw8;(U1^l+66l;t@TTXH*kxpF30{AgZkzcVh?c zldEqy;Ys@L&XmLeZTof~E`eUezz)hTHqyT=SXx4!93-G=@ci;oxVbfZ&Y$!238uD2qb^sP_$$(biOV4pzZZ4M!};Yj*<*YrI2u|@J<)47|U|wsBqsBdK6FzLZgG6bwpN^ zM2ae+MUD}-QTTM7ld-~7^W)7opBe3~&qT_7dTYN2*A&}hpbq+F2)SL4Aj`$8%*>xU z?OPs_Q-FxWDuZU9R=Q)doKej?qB*i9qTDVjNg4Gsl&?K_*fr(Z6t znIWEm&4*6`5BQLXN176Oqwbp256RYv1Ab`;q?IRZ*_?i3$`q1L`JF(s0JIl7#2U}- zb9r~0yB)My_34E^ovI_j(g-as0qvs-Bn?tKBfdD*A%Z1l}?UsDZGEu*|lOqqDlw%aj9hY{FA+Z`D*5yil z1&N%5#<6vX_2?zAMKLmxPfUDP^noc9HD9%JUBphp%`gPYF0Qb|YPt0I$y~20qN1_H zQ&m2T>B%B!Z{wnx2M{dU|F9s_FE4s58)Zegn_X9irKc*hO`9v(TJHBzg0n2C&9u5W zbe53hOhtMfa^(bgLSs{ID1=npcxq(cPkqfkfx=aUl zYGB1LUinPyS27?tVfkg^7GfGU9l8Do<#Gr_$I({CG$1P+-*{mVjkz8ejwWXoR3aaUvo+l(Mzjf0t3y|@ zLUOPDJc+t8;k@zFp3X=e*9$$@)@j(hA$a^UQ@GoyI~hRX4k||Gn_C`6v4^bSH_-!X zPwlUx-e`w@*<&oFKmc1YtkmeF+`PKi9_Inmdo)EmAuq@?H3Y7DZjh2tkQz}M=|LUg zbDinY!MY3WuF-)!s@Zd$$O=x4;&}9H#1RqtMBm#96x;CK7=iD&G~7GeMG+uo9s)~2 zMAi&-$TV=yS^)PwZ25Q=I_l52W{jveAd8U&sY8I=jq@uCa_ACXB?0g%BhDWHq0J|BTK3R zXyFetDHWp@7mJi`^(JP~BQ`USo*MmEPT|zM5`ly^^332dRGQKdR1v`Vnb1v*dR?(_ z!uNZu7VY>MI9iYt5_kfYIy;>#RMH(V?8E3qwu+P`!74@7C*CICnJvV}EUpvWVy(`t zEO5O-W{jMTBQNY2<(Gz|2lnJa)Cz|dp9%hS2TDY@3;DX@I)B;w%Yt^yayirxb>kc$ z$xdJat+R4436xUrK*WQiKwiEq(5-Fb8*ch^7$n!pO6AFY8iNN+Na5n6s| zJHoS2a1F@U_FE!FvRfD$U?OseLvI9zsRvcy z&C@c~JSKAf?Mu>?so+xYN&;dgKznt*D~qh5yfHn*TkdLc8n%YtgBS$zmQa#dBjsQ{ zJJrJKKt7NY87oR9(kB6Hf}CL>-$*d8hR`A5C5f~`?iRM9_u4~PA9G(0t{?yw2Lra) zqvCe68TZ9g3_9SCrMsSgLp0Ke`)*=AFrsYXks7IyJn-RWWU8cfg@!ZAOlBZwI2q%H zbtp}Ct09^sB$Hwi5;{v(4Qm(#+dPHqkgO>f`Qon>_FTm@@RIAo(}f}cyhI?bXW%Wj zs~CcfHwaV#(oJkQ!b#i(EG12{6Kjk5zNg6O#JaH^Td)9<*ogj|!a5mP!vJGn2EXc6 z@Vf12l`GJh`9OS9xl~+%l1Sbm(mcsA-~_})5*JDBwQ8rxc0j%uQpm3P_~>EyOMzF_ z4emwdp-rai{rZ(Wbyz77fDA~?g4oZb{sr-C5%;p};##xo2skO~6447>#n9yzjuvFD z_UIubjy5J4BR#m@Mv3zLJig)Phvd5;SqC^8D_w#n zn`pqhrVXP@YM!GqYT7l~P=`kX9!RL{!ovrKB{5WlZ%jZ;s3WR|?=R$L3zEohLGWRb zZW!I~_+SxV8?Zz9pB60gaf+%`MG*jvfHY25^&Fj$i6Na$1vsh~Yfc?sMA_?t;70B7i zwy*4|Z7cS+!n#70e{}v&M=~w+zmqk9MDGJ>Sei%=w#&^+YV=TNMer~fo({P3Dq%#u4wsQ?!bcxMpD$SapT_=~ICqJV`<>K5=CK{{5aluk#||5g?tO z%@LRO6sxT=M$&b$(H)~XwT{sCReR0#Q9*j>B2KGGfz#C>4}8S5v%G4 zwE7vAPZ@lA^w0z}PEo9;Xva-hLKpOP8vR+apAEK?7 z2f>a4qC@YVt-7^YSXh{Jz1PTDTl6b})H}tOZrGDiYhjMWdK%Lqj0ERFtl>DE&FaRi ztm$sm{4FH7bCSEM3NZdOR<;l=8m_5j`0UyGWWDWphFhpFz$=38uyw2?X_LrcjiMaX zad*rEC){b@K=C!qC(z=0@R^nw_9{(hx|w` z+7E&`2?Eec#qGWltseF0y9`djo-dNM4;ObP+dDYos1 z`D|hh37;N31#H#f^ z`5$aG!4_sg%Bq%IzR(VvT(xB>P)x z#&N0c7`WAhLM>n$T1kXQ$^dLdO28Y5Y9K*9aLJVb9t8*1nkogsqm|n1HPIsozXjms zaWbVBpg^i8Sh-+(4FCdNs4#mSs zfG~b`RtC;_OdIfYNU#8FG(CS6S|IHzkZrfXj}{Q(ea(X1!l=n_$z(_*jQ4iUnFz1_NkD|F z;mcQW2l7v^aO_7x6(;$Gp8O4}7*s9=rzpoR5OhLm@DtHjE^$Ip)EfIN%ozTC}?L&3}06Sq6dv;WA z)KoH5aWmTQKfc{}IN9OLC3b)bZOE4uXwlQ2)D-5$P@j(s7J(*#ltpY|Iu=t2kEj(^ zs2=u%EIs;4tQ3|tZxSj&7n?tk2!ZpLO}JvJhHi7uMD8{IiMQ|Or41!Se2^f_-Uw%z zXG^pb8_`fKo^eYB@$n5gMMG!b@d?CYyEzAktkUg^Y^yqcKHNnz6cj6_%uJWUQmvA@ zr?hhno3uKRp~|_8W3Liri5jj(L>G}K)a((FBYbIt!&!RqqR^P}017|}c@Ria)TzIY zHa(%n7Lw8(Frviy^+74l8%If4F?lvXgpBG3da)**iu{N>9{35pO@j#WDK3qh| zbr|)Z?XM*93Ox=VUQ?j017y$AF%-O4N1k=gj0b(t?T1&XviG?=sV;gyP}p8_6xGEF zQUmv47h}0IsmG?SV4oiyBEZ4%2W`E- zDD>$C^EANct@IG@8!#-rz~l1bypp|%`oh8YG1OB|B<(1Rtyy1y*l6=@Z_W{18mnQIa{%ly3|>ui3%9kiY)$@uU{ zd0#@whFgp_41^(imufA$XZQ0xE)L#7i1FbtjIeizVEhS|gCDaQ!v4fC06E%_X#8#f zbA<@|h22~q#iozcT%(4c$s0l&qYM?;+m+Z~cfp*nFrPr&UBt;JZ-c){T<|?37AP94 z-9QAa19B9bk0#G7uS5sg?$ITsO^%?4q6gLm?!)GYkrnIL{(b)!67qp7X*;9KiN$YM ztx%hHg9mnl04!%I##5w;J*dkhWs=#E*mCl{C6&Cm#O-pO_+$oqdmnLLo~_Pk9%>p^ zliM5wL+XnL2rusL?!u09BEadPDB2?iHs;pe^Wku^+ey0_$87_km{3bhi4Hhv0>p)O zZXda00oa2vg**&I#8pHCS?Xf?$?HI-b!<%4$|PApCvm_h|8XjnA4m%rZr9n!NFllB%J0$D8rlOMX)jc6>0 zTG7aL#ba23Bj|uhL)L496m9mU1xiiIzTg#ezUJ-xYbc;Qt-s_ry7>5=BMf{6Y4;e| zi$WhB{KA5cv2YHZT*Tj;&*I0HWuF@VjPh5DhFH@z>wyrH z91Cp0U8*(bkfM9(p-6#mI0U9(_KJ1s>OUx=PhYUaLuO8wMQj`7-zseawAaBpd6CJ1TlX=xj&>zs9N z^<1jC?!!&qv%9dW7D&Zs-tr^1joY#HNY_W1d2^q)emPgxmJjc{FtC7NcnjC@h)rPR z8|@7&M{0`o19_dNHZ9-tOU%*Nw7A!~W>lcid0 z>)QLP-%D>{O7G}_{mobX z&MBigcRwC)`0V{?C!S(WIrcu!e3iON<&cD(e28QGtM2zl9b|rC2=KeVXt*Ok)7|rr;bjj$ zsrD23ELjPg=v>owpZp=V4eHM;xGWni|0A#>CAT&zISev76m*-SptN zpm<$G?=VhLA)_gP*iLKl54W>(TfXr8@$UD_0rn$oBxWkB%3YD#1t`oi?Wwx`U*~g~ z5;I#@L;zr}uw| z?<1kw!y~`*ZITPD)ij)w1}^XxEjH%3mwv!cs5p15)&%}8X71?W!pnx&13o+qN1kj6 zwkL39ea^Aj4aVOv$UsmI<-uF*^V>HMlyASQuU}pBXn%IGyMRmT$cK+EEgg{kB+o&X zx+{)wMb)Em?um1t*F9$4|0k!Zou(#;vAM?EjBKv&_T?F3_9 zN*nu>L*}mAx4fuDZ^Na?4=SR22*M-V2LU(6VQ7^@An{bjXy2)2(R#Z!>R7QgEeOoC zunf*!f&U%ub&{1}yBP-M-a)dQXq~1vD1h*9ZPxM9hF&qM82=b?v^|2@8zR(^PGcMUe~ z_Se_!p!D>)pTa8s6JyaFlu zhVGT!uYaRsG$x8GnXAdB+)$QY3cvGjA5sqmw^eJ-Dw+1CWsN*yb=Z;Kr{{J zU@6e|#pvqCy3=4$<|Xq5_1`=&1s)`G%Vq41%Blp7MJFo3DAO4u{^j`3%}zV)U%6e3 zpHuB_yh5_MAZ!Zdlih;DvLZ@M961Si(1;&_bV*_{#H4R%`#YI6(b}%Vj#>IH+QrAi zwsS*#(S^;lf;Nj1OPP4vk7m$Ufh;xiC#()0z_;TEM#a@GC zbYd{s8`1Nj&7@$GH! z9!BVs+jEBHvDX${WpSlzNY0Ecyuig?F4xwb`+lG52eN_VN#c8?(87~F5Mtfq8+%s~ zm;rQ=h^4NG#p$OBIv>3~7LX|MYp@ri&657xH)3meXPT(l9=p`kpXU7&X9&F2puGpwtnbs%*8$xD-`>ma4dULEm z%&{Ds?ivnZGc-%oP%&WpoX`7yhJ1CQ;B7G6{Gu|Psd{^=1 zHQ{S+=j@rNNnmz{$Szne51=^F$}MYU^5GH}c#8ptI6Q6|RJQeJ8>jWaT*36s;=h5)%fR;m{~kWI@OW?Hg$Ar!1Q!Sfqhl7v zQ7cx;nWDC|2Ig9G&Ezw4PMvjH^Vc#N?eo^MW%OrkgeSZBa#g04<;$OnwHkUCt5pK+#+)jks($z)ilW)u~slOLuK8WtP7vuMsYj&Gq})oE{_ zsV@qC@zem{FAMpwMKz!md_QY8WZNmT^Kr={T}_hqQ+-b_&-v0s9{S|gi?<@1YLoTQ z-4$>2x~6|QrE<2V1;j5FC9D;2Ug5N`kHYQO@IO;XKFhh-YrFa8{uY*eh{G1c#@*th z&td)Xn;!o#wH!Lv*h}MVwf_BvTFo8s@d(Ql8iZmch>2yWsO%DpRSA#nE#>C^@R^eg zYBtoqfbaF^bh%)SfbDk_6m09;T?OK1#`22~3-AK3V{eH57au-zftL;vKDj+M-{omk zHS2-yaKP@Qoo8|;^qczR0;q%iZ;NhU(G@&5k3j5E$A%ICVgKz#UJBrdyPooPHQE=? z9j~K{{|bpyyZMSvEJ)korZD;c+*?7KU$&Ggm1lg z>f!1fND#6+3R5aK^)*z^IKFwJ+xD1ZH|dNw&Kdx2+}Zl!nK-Tmpv&jT!)ax#mX_)} zz69dp?JT~Wd}j|yS_ju=+G*7aN3-vt20`1sWF~Xp@x6(3|p_$eAJaq zbYcj31>&zm{ecyUPcCrN!;8GCS~dTC_j7h$;1Crx4S>xY`%8g+#y*2skSY_@1$j@3}%S{w78 zx`BW6qbDuY(8;_s)__eY3tAp7fB3-d59X(xAxp5gTmRxm?+hDH zru(#Bp^>tHYgbvePA->9{1-%Z2nlNIYSST38@SIEn(Kj5{j zy0XMaHt36wpTJGlC};UM3})E#!0q!SY<{-wJ6ZMJ^=+S2NUtsNb#q(t72m#(zC-cV z>t8WGVcyT${T_zmkpyN9WDKiKZtGpeR0pvZINhn^Rr|Lq%AzkhpE*52}fKfLG7Tex8&)?wsP z@yAo7l%b~P&BKsbw3K^UdNbFi5TDIgGyeOKU*kzmNm0+X736jq{}%hQ+HqP#1a|)G z>gqn4I?G%f_0IZy`>OGm^EPX&G^+VozkG%8zsHpVX+8eqzKgwAdpI|eo%zw-q) zzbuBCgtxxFUMWF4SZAhNl?>R|wkU6DF7i=26ss}6PG`AZ=>x}s`+9!&ukpXV&3*rY z+tDb+)4GQ~n!YWXiWGNGUE{H#2sSfMm~xU#>P4`v{&}VqAI?(PPWKNp;98hvIR#W= zZ@?k5RVQyx^Fd#wLyfj;b0)XsjI7QX8ieT_wG*E>K?uSdnZLwZe~ z=W`$S*^=?EHB{7z1#x%h_ny42h6xdD-5}aVNZTLS?vj znf+QG_{!g4v5@LV^ZRsAUCld;rz&1c5*tAT;ca^i#()mkD#4$_=n(0wqgLlV47ait z*0S<|y(8V*&VA!FW!ju_ap_llx6O6Fmf2uw`M^0!@$I|no3Sd%x;y_B^zxcj=efUU zuJnX`uVCtP=8DIzEHUf8#WGWCjs$!rsBw?Ud-Ltc%xU#gm1;R(hn)XZBXw~>>R{`s z*FQc_S=dqGx#{EZfBJh0vK)i#hlA2Thkcgp?08*%!8YU7m)CMG)+_bfclhvYU5;M%5SC|eJ}FZ(VX+vQ;UGX?%VKLYJzn_(Dyxt9e=3~&*$(~hZy$J)B^s-_rhInph3V&6DdY}* z##Xh&3!vbvO86rZ&6*^z^gnr?YRpMzlMHv^;xZG9wyImUz35ASDD3iyE6eeRn&HHiT>;>c+=v{nRHBX-+cbpx)_Rkq&z6)hf z^=mj8PQ%zu$Ebycg`u$wO@yvw1`-EW_6-BAj4is|wQfc=--wMpH;}a3B){O-%lo_= z^QKqSCS0M zV;*$ZoU2tYl8(_7w-C`U|5@7khLinu@vmGaN{=cvB+uWZaM7&kR`J!3Zr}PedoT|% z7_Y?gCzisAztG>`ANg1;2*wvWmCSvtif7di>m@HL)(w{E_BHUy7mg3U?axj$=GG!DhpxD!T~{pNlJ zeqP~&2syk0jxt>^nMJV!4e4IwDt`4NQ|YbZ@l(V4YPnpay+6s!@WMW}5f^RaAbDW=BRzD7GYc2L{TTnF(Pp@=9Ut>9;ArEL@&^e(>}brO5vj zeRWvz{+b;NxWtwfq#A^}W%vKAAhR^Vu=-Z<^Q=`OB18S1m6tr%ek;b={p_bx5C1*% z%%AAS&*3Z`US3|eYu8p9J&Uv`8^#Z)QL8R_GZgO|*DbT{FUvN6v2vHmg%Y1{qvOq5 z@}A^~f5ElV-6H(({c44MhZkSly5?U?@7h+H)}K$L8m-FA%-m_-95jE)>Z9?RnGY+U zR#!)9q;7#JN@bnGW=7KSu+N_*aQB6Ha6bG>;rkesZ7xpd<$^wU9KF|=S*8Dvbc8kZ zdj&8MZw6{QMD-}YY+QfEcSjiQg{1baE<8(9LUcIi-<_E8_?#!DTB&5SZMzDnU?L# z6tum=bgPp#n$%lP8}v0C^sb8h@~+7u&$X}&7fA!;{bRVV6%jt*5yI|9JnbW#h4uh@ zqwYh1Sa=Tyn^dBpu4fOy^_^XpLgj+FRn^P?&#Uq93b3F0JXqMx z8h1CtcuxmHZuZ%t{OL=0$8N$QXdxZUQ=omVJ=v({hQr9YnRdU#)AOCCE&g5KiXX7= z4N>%f_8OE;{<`R^*XOtx?w)^5^qLg?y`j=a^Y)YlEyAI>LC(8O>Ob}7+m~)u)%GzO zEWSsvc@tpFcV=pgJ4pq*;&ulgU@n$#$~rZvZq}bvY1@B!)0ye>=f||3r!L_VyQr$; z<{|WR*^!K)!|YyT?#MkIV}p+*gUSuqi71aFL5eC`*--84OUDs1(Hg5%5-wsV*rB24#{PLfX;54JL#REgIt#t9IIlTH_a|K$5x z)MzCZU#A-^2kUTZrpfNUP~u0Iw&r0^ zirU??U*Zz9V};zE^uGrPf-zkwPQwRu;uDze&vPGRekW%5bh%8RglNF!?PqRSv>w%T z997Tg5duDY8?Z|-;A2S0ua3?kjAsxwQN>&b8;-+%Lgm=tu;?C4btUBWKEDA_UnAQ# z(aNCNx%y7xsd-BAU#!~ls7Gn;aN=jhSJ$`(<{e_*I)>|{nKdo`9(xV_#mqPSXH$ZY z;E7#V_`Ji~67`l$53n8>C)|&U&`|j}nyR6yO4v9 zK0^7p6sn9_Ap5m%4G(`PaQ^dTuJ7yO6OC2Nyf%Kc##R~0W5hp8b7i^M4m8V647Fpc z>LytDWx@~pxQ`_vKpN9s4u(xFjZ&TcRXPEMUTUqwy9>`;XR|H^1tsF3o~s)^x+dfB zd4hXCiJCnB1BWmj&v$XQw6X$uCWD5o#6o5ah~@HN7Z1NUR+{tV_xUd8I{Z)J6u4Ip zmb{9Jcn z?{<}4{c(zd=h&A#{QQ4{w$paYKDu50FWLPsbJi`(=B9?|<8rCd`ikHrY0r{ueS|!-Z_J{NtzqD{z>N*m2x!DE4T$<5Z09 zw=Wa@LaF&``XzjAyS;Ow~TBm;g|`btdvzsWN#&8l#Hy5%#s<&CWMT# z%E|}{k-b0o{O;#NC!Fu=|Np*@PT$ixyg$$TdG6=Fulu^LTkXfEXd1|GpINcK(M9(R+Pyd)aig-EHj%nq-g<$^nQi z#L~Xx1~Bsk2?%>-cb-rNIc9tmjmR9p*U1!w>3xSrU8R0L4X~En@rhHi ztk5msgTxYPx<#Sjd-VuXDlkqUqU6?V0CNDO=s+c7OW6rg8Zg6jA=vRPM*BAE79icr zf#b>J%aBrXBj5UC$_#QV7aE=&m>;XGI#QW$IsKpQGdGD#ZQ`|FUz1!K2R8bnemMp? z?BgA!?ts-!ux?b+2ra?8fhs@H!U{s7`2)1d;sc!^RFj5=?VvTUTniT5qj_A#zJGs! zQBC?b`aBM^!mCi@G1B}*d+xPpW%=u_YY$hCx?;71Lc1! znv~pL*+y}f*x@yWwDa;+&UY7BN<{5=M{{@BaG`r<7|lQ`!}Lso_(>8EJs7;9@4y&G z!fK=z@aic8*^=h5Qq&&44A&RZ{DOw) zQ7!d4l(~1|&!DO?2@DFV@Z&DEe?QyZtpn=TauA-^g2(hZ@EKYD3iPn|!_KhVfLr0w zlBA&a_SR)k?pHwsKO8)(pekl$gkYo=1)l5s`ETw`*RHI4IxP{g^Ak>@LJmK}dbGtE z{553L1kBp}!}J_|K#1l6)cgE*jaiTi0I$6gxcvc`RZ!D^1AWgm_$O$i1u-hL6xx`T z0@_g-+nOLtdiqXp!^}`U4*K{lVBRfR#89Oj#+7BoY-oJ0F{UZyFvpd>gp}?cyo}p- zVTkF*>07g{v&0!z~t_!^X`|M<~p;JptrdA0ue z8Q&cWpoLntE%b%ZB+0S+`YF`+B% zD{&IU1Kq=M_|n}~LQg@NB7ERv&!oL1pKiB?P zId;d1!U|)RQy8jZaL)?Nia5YxB$tX|VffA3IJ}reT0obx|4fxHMSxbO*6V?O_`eI( z6m0530}h~6W7W*)?r#*1LG&C@m>kb31z1r&ge9;qA@zx<;J_NC+bu8Z*R?`dFbh-7 zvPn>~ddoju$g;P5@4XFrO$_b;{==ZlUOACbbOSM_0pA7Lk5(>1J(7HPs1B5bzOZX+ zrVV{P@oDU23j5v>D)UwPha0Y9Z*}dxQyOvZSC>ket9jz^aw(~VySUSHjr84q+kZ;_ z*AgeY;()*9n2$puo4DHsa1FdEV8~d3P=!_xDc4i9^CtQmJHdbA5Vjj`nI#bRkYZnV0q}DkZZcjYWSQd0nZ&;qGJcc zbapRx?~(F87b_7+i|_7D=jBOwf^pv`@gvEKJ`&qe41ztl_n?QnEalCqsj1P3qcm9Y zsV-2SpbbDwb9(P{m{R2o>Mi79<}GP5RQD7R543mqf0%?OYOM_Tsc7s0C|}P_oV>j) z-J>=*p;jyB#|O-pRD&UTt=A`Yc4B82E#V$u?E`B*8eTp=(!M)Ag-_upK@lvB>J2mo z1QImlajz&NvmZo!$iD@tx$&SOD2eKrZs2JtMvFWJ*2-g0YI}m)BTDz#l_%wcuF;Fq zkkCFjDgC$LkyjeTV?%YG7#786`>lSwTU(T>4T4mM$I&`G3ljbR0FGqnvLnBT9Na&r zY6d{4+Qnnq^fw5CYru*Xk4ns*2Y{VHFaV%H^8i5CNURXb5obR=;JpX|oCHo&BAmE4 zX6=umY6^iHxoT+bA5{*UNfKxS7&8HM39pCF881MB;7h7(@Tu$jt$*(Q)dfhMLD1Ju z6RFPZEDMAG{a8VDU^P$!@?b>(R|E37Oko`u7xn#j$;-(ZKz#M56<;|Z`2x~gkc0E> zvlNqK`#e!f!=walfN!&n{qJ^x;gX8svs3hiQT5#33f!KD=&w%bgbN*FQ-5fV9bEpl z2QX8ZIn!}$d%RvxtGzD_~D|MK$Tx~G1| zSb(GZy`lWyPwEvQao;TC9&4z)hb*bdg zxW1BDF#K4kFqdakN<;u&Re3=|M6=&!tiq~MI=DHV=5q$INi4Cr)48>kEN9{IZhTa2 zzc&6DxR|O5mq^w%>HHnZOiOJ!+-y+l9D77ZI$26sWP5h7dg^DYMYHsi2z*UF>N?}) zTqi*tCr(3?%Ig81xhJJ7-sHkinKH?DZ*K6IkaW9AhH&7&h$Tso^~N{5sN^de^ZU%- zuitj&p%sOLnW5`;G4P~H?#+CzfAKQ#mLuS@XDqSeQOZm>vPW41h;GmL~xw>sdEzJ5%Z9u(4wOEB6WG>vw>V zEp7-@f-a-%_3AT&)p7X?<2e@Xx*)gi7r1@JS{S=@AcXx0f!Sm!jj!c!EUDz$4Svj> zuarzNNiQnDl|%T2%cUvd0K%ZF|HmQ@)%^#&NnDk~L6iIF0ekC}eZmm!4$Z=0<)r~a z)8^fJE&?Ih>kEP-ZJd$*ee0Bw{PzGQcoFNRk$!blHGk!1I6rHozj@Dd+uK5zTO*x^ zf3|xM)mJ*1^+`kmqwTrcizxx^HwT#%4>~{}tHQH+C&tL?hRgWb1O)}r4_`arOO9`p z$pSW7;jb1x3aaEVt@dHh%It^MjncCU80*p4Q9_eZLYbY@|GK8)O5jO@t}E>s%YE1< z|2<=FJI>Wm0J;KCq81f^G$PI}kWW=DpASl`c&>E{TnVPw^d&;R{H^mM32k|8mdc^; z@1CEPl}ObSkM!R>XX51j5-SHY3id;hsaF!e6RlERPsCLUBkCDC5uZPQu55=2!DrS7 zK*6)GuT9O=L+=Gz!j0lb+BDM%)SBXXoQ2@D0Z`EGY{l@8ODFIu`rC75xQcSn*FB!_DW!8tB}b#YHy zW@AyTsGuUF=)C$rk47}Lz%2T;8AR6?v9uYO#YQ)oJac$w-~U}WEN0KG9s+7=hsD|9 z%IZge_SyV=mu2uD$cm92hdQYSrK;@e%JtAIdG3NyQ3ZE7Q@;Jg1WsY>VR@m^$9?az zvNjtM;rh18uUy=Mv_n=FJb!$w#KmHLAZlqLM$#1m?bJYXH9ZQ#Tu(VRI(%rC%vun(h~{JUQNZn5mWG!KKva`jQ<*M`Zln$1=IYqflE>(wUf0 zK6}TG2S5au0QmMoy4vyfwsvvA2{|?f`OpVn%{}t^yoO`FkBh=^rstN|!v)wQ{eOIf z9(lMRsd@e`+sVh~*DL_;--;bn$|Q80Px;fB>}XN33E>LrV?q}0VBil*PioK% z%jDktf*RGRV~mo};s9oN)T06fhu=%AkwXI?ApWmK)}t!!V@xYZgGW!u|&}-}cR@TE0j~REP8Z+*4}-7tUbqd&5_pr+b%F?Q-)cD zsGwCnc09A;QZ~cf&wZ(yvcc2y?GEjldiV;V@8(9E$6Y=M_MO2jSajXsF(v}5SA%Ti zBco(B*0Rrb;z2`$%E;Jfd`iyy`v@!A*$L`H%e{y#zFdlP7BoL5wl1U?NiHE1x22tG zaI>4Q@xf#XwdAcYuT&nU=^Y0;o8^v2_YI3j1PIonV46+@0vZmW^otkUf_sNb!CfPm zviJ!=(4#f2azO((s(kM^zJz1&mN+Sc|N$03)v3xN@B%Hf!+g87^DY5Z(smIXG6C;?-gxrPb2gM&0;)u zfk{q;8j^{;p8a>qh=D&KYI|mFCGI`tm$@u3F6R z!tt-4sRrfupEPIE5mDgGJI-qYpI=d|mG*^2 zP}eEo*^GGwKo_*algG}SpIW!@OfXZH2`O}_q0jL#(o?lNN;rm#cXPO;a^I&wcML;kmXK_pSG-j;t31-m9P-V)_|VPI4~^X(DzS3W(~puk|&4EPaOY9o~IQu z@T1W?N#O)1F;|&bp-WljMS9atvg{nFMz6<41P=d9m1;Jd)Pp%IDsw-(sm)6SR&PBz z7C}V~onN1ux((ZrZVsT5V8=`Ub$ffw=Yta!}H?c_ngpbb-B*<3Y+tc4zon_z?vE!$Q;$9Zin9zb> z{Wc2|^4?GJ2PN7U+*y14&mxU#rgI`@NERAFW{e8wGWGf?a__TMeT*g5)TPL<8+VTp z2gLWKCMmSf*)Gk*?ovr@3tS8kr%bMvW~ByP`W&_8q@HQhw+|*gMa;@Fudngga@^`7 zcTFy|#+STsv8zy^I7|wb`4LTJ>_-usc-G*mliRCd^(?P=DZg!3LrlhF6A{SfZPfUCu$?Oxfefik5a6ylV zisxJY80hVOKy$~;{{p?7>KrcQB`8?xJ0D`wSo_ZGz?ibS;>tSlNbqWp2w)%N*2bUw znG&o_Ognf&!bm!E2_gBdie>TDs|~vNhULv2x9?q=`q0t&;WnvdtONNq>*3`5lBwN_ z5oD#VdA$U1eB<^1n`P{=8vfyi)iT-+`x9v3k_k2~xK519@9&$IfO{RkzfIc!?zfV= zj8S&9MAW+>nyqytU-$$jEs@~qpg)SeQPAZs>j=#41_1j)ftG(_AOkBt(9ED5%8-3@A4&!CH@Rk&F60Zp z2!qQ-YzSH_B-Vw}CcE~f99rXkLu2l45_7NI!(h%-$eyQ=S0~#D0|$QEiH z1NlxV((+$128TbwWOyN{ixBkDWz6n;Vc!$SA^IL}$%S9_MZHCWidFYj_KVo_ zMEa9T2+gefai60eW|q9i;vElbSs&^-3iD+4cXl@aidLj9vXdJA)bIHIVE#gvMNfeh zO9Z2Q_3Cn#HEUv0`B0c_&Tw;~^*tt~tBtf!LB%+cL<)GsNf~^KD)zDW6yN#CxMMTo zJaXqOR_*3ZHn8jv0c0WxbZOJVK*_;Rwi)9g9*c2$Zd9-GWVwp_YAqQH-Nj5WW4gM! z#8Z2FNFItg@&Sn@Jh)BTFUrZ?gTxbfZA#C+L5r(_FX5{sHSb-k*^Flbn0^ya@dVS0 zrhT+;Z{3qzJ&|@PWjEj`Ig;-de(^Zl3(a~S@ycyUGMF(0T?DY{x?Wvs%G75`c}%qn zrU>l79IUeuF<0&y4BSOwS`L4*{6POF(KuB`7EKa~Rh|&dD{mf}XSZtK@ytF7Jg_Wi z=6!!_!@jyY>=2wkGw3aEnIPk-8b$5Era^SL2^a&QOKg)|BlkVhh+ThOXOHk`AEw+roHmJ(GuNZ|7Ok_216@qVK9CD@9yy={e06JFIr#f;?VB2 z=#_o+!O6CAk7mR7@#J9~)Cys=4%n(A2EJb}xbB`E9!5tPEJj(hL75jg)+k9AQ!*{H z&X=t0`SBUtNH;%!z`9P?jdYw+R0o=C_lp;mg*N>noatA$qvE56LAWNwx)r*Q!JbV# z4!VgVmW=Xe03ijxL+HJ?r5n7%Tl(>gNABjQV;??xYUE7VJ#C)n$bVnw`R z2sdetiIlQ(?-gZ|hA3*?(YtMFPl5vv`Pu;{0y)~reyc{{{^4#Vf2v*LP89$SbVFt3 z$NiWIm*y<0mWYVWtCt%(Pn``{)_ig;aN7mII!%ECMX&`x?>SZq4nX|4Mo(`Erau?# zB~^=TZ-c5-1{AV`mgj-Q9fG#Ru8D#J>+5zx7I?4JeQblidXQE`+j*%1LO9j$ZLu~F zv^8t=S^f!?v*a@Y$haw!F3Q1}WdY!J=rmaGOPL<*@3E_V|F-&3{};{vc)!2-_!RBH zTKN?3-Md$7gQiR@YssLvGHu-tNqnRLKxGeR!*H8fo`q#Mzv*ADj}zvmDzhw3BqqKq z81^cDx#QYZy}NM+fo%YI?tJ(>xLH30RK5l{Vhp4DtQU9RZ08Cuxqi`CA=F<=ts7j( z?)ZM?c0J^)c=3WS`v-p@bWpH=^W(ny_3c#d%9K?8T#L$8zjObhZsfHjMlWRe%fuyh3*=z^Fh@&RK7K#pDDve3N&RI^sfE@i4>rvA;X1}WJ8Lh=(}AB~{N8?lsz^;LLFKFRl&w6c zu_CwK>uz(RT(hz-`-uNohR3h3{d29rk&JV3f|@u^`MCXbl_@VG$Nf;z>B>2#@KkNurZo2vy8S4knPq=NM@P-o96e;4z99e%3v-){!r+Y!2Kx~I`N~` z{`jp26aMnsP9dqS#j$+ARPP;fmjVOY3e>R1gTZ~or~~-=YWaPkSmhvw6bpISBzH>o zQLsH2yO?lwd4`Ac)|du(hgR-=y?PYKwM)vzaqS3<(i$ZQhmu*|pQNYX130@BFFzow_rQce^?*Ek}xrhSXC6&41@(;u^=$ zY@P-OS)~1*fdp;01?oWpA5s(oWa41{mQ0fMEs2t)!e@UM%uL4O1Q34?4yC2>IAy6-aZ)vgZ(AQ`meaU7Q<;<<)am!XjFcdQ`+HjMo*ftQ-xilnKkjHh^gDLof{ z1?XUYvI7pdI_dJ^i>6crZ=Bdt1tE`-eE40v-DiDf+`6z>x+|ZoQh+jTR>Xt!i0KzPvK3>g|yEDM?K#GX_7l_l&e?EcZ1`}6_8<< zgGK6xPoMT*g&A+(7uFYj}^ zNgjjrvmW%ZHj^>fHM>cYr1#^u?8jecX(av={-7<_Y<)4{5-zKESXxg`K8K^X1H{Ux zAhVsP$hv^boM6Hw2ej3Pz{_wDif{2^SC}x*1CG){F^XvQp*##?95hskKOgRU>95i? z%G0+AlCncWRP;m`7#S^jZ701xMOV&0JQNqQ&*0ZYQ9^CFBwZ+%=Nta~)fv-;)lvY% zjg>ys-n+Laz*(cjalnw{KY(E}z&u-!EqO&;wShc=`OUkR!sk)57{=7pte=5@O`Z+2 zaQ20=gIbfsIu+<rS5=T|zkNry{#P5^JQ*Vb&O+UeW-N10 zSq^=Snf~C;T-ix=-fO{edb%u5A<-)$$GAwm;t|xNm|H%JaI^asK8j*?Gb1 z(2FMv4<48q_8@&+78AOkkdi@Jrpni#4pGvrjC)sJ#L8%f;P_=LD)~UUPJ;oi`7x*V z-277KCgUgqGSH<+sC5^_3f(n3__DLFE{s3nyDk$h-(D!-e+>WQGug0h^d8o?8+9Q} zzoFu}iU+pWz-3}ZbQ!Qv^g@1ofKS!{Q5sf871A9U8CYRm$JO~DMuYWLQ(nMHCyEbe z6wiE^hh(4;IA?B{%qtWGmj6pfv%Ua(Cyc8p5r4dM52OjeK0nvp6bCUwF*XM4KUn)a z=5$JGY^!Tj4W4fB_rM+YFEpncQPNR^6=r=XPWG07kss+MB^@+X=lU^8Z;p#TcX47- z+b?44&owxf8Nivk3)+)Kwg4hP0uJ=cyx#lmdLDd@L~1?eh4?EAA5l_^!%4iICOsGX zNH>5IfLZuA1t*Ztcv%jT32rM9*cxyP_?yR$1V-3JH*5`>>xt?bKuHcyCZG-ISq+u7 zF6nWCTr+R{Ye3y{-_^V$x59_nYS=Z9E!Rkrv>(wYdM4;S_RRdgU2iRVIMdGuue^Ow z{RoGgNE`eM-d1A36mmmc^P!F{e{CzFBN3~RdIqPnZ{hsG8lN*0}0K9(vGM)$y!6OoD z{E9tbatv%qP)I=yl143t6;prt1S>umc*^D`i%k-a!aGT7a8zgZM?dCSB*7ofH~>sY zzataiwdcS$sFFIecC`N1597|~cy~cC>c@gM2C;Bpyq4fA5I%wNJ94x}KBt2&=V4u0 zCpMx6BZ?Xtq71M4W-UGmBjWG$zFS`@;fUe~x5x7s?1sW7>$)t<`?W^wgZV}Y4_a|D z<~_h@C?11ajN9!VA(y)!m#;Cl%CFyt`S~SwDoKgrP4Kq+{cmbLsgh;=M^6zOckL<6E)1 z+&?(_!aKfes((r!OOgB(jx>xWTN+S#fLXfUk-hI`yH^dZ2j3_6L-aFcZfU%LMgRI* zIbZD1luQix9YADhFXtwXGsX0{x!Dub>?f5t_$3ue^uE%#gJHpkC*YZ#eni}VHn9Im zH1Xxj`|{v>Y>9_K{lIxX_xma;1+#d0-S7wqUGU(`rukQhz);T*01X1 zpgp*lE{)x|k#+Sfze&NOUjENl7UI^XEKW+Xv~|*Z_P?v}T>gq(ER(do{rv+EJ~3RN ztH0<{{eh85Kv6OO03Yy2m0ULGWk&wn1THJzCF$7-jT^lntv|H~qI+9 zsKoBG-vbPelhRk4{tg!gn{2xYB4Qe^`=3t~*?P8U%FFW`WosEidmQAIX2H*B>Tco< z)g50JXb9laeJ&TQGX4|61zd`SG$Q0b8t|nn&O@xPN|!o7CB&mRL4$+PjqkVv<8GCY z$4%l^H&y8oo1f>^nEa%-wSNY2e=L%AO)%r9+cnx2QsV@d3X5 zCi~*GizpEtZE|l3)^(FrrB_163=5sCcn@0TYTba6s&&V&=;6)%qG8G#`pxaDG=wDa zFf%IL%9n?|V}TvY7QQWGJzk4TSW!a=I3O_JT6+!A*FW%l`@ zd7qZavB-hY4dDeO?oMm-xDH}c(YDr$mLYKO*H%1JT<}Imv&?n#l~IRoqh++N!be`-L?-k1~9%buJ4 z0qc2-8g8<>0BG0bTgHv&*>Q`l+w)Tcw2t2L@4py%p;-`e9{9oR&>#E+1M>E~1AC(y zV!>{tAQ`~r+;aPpG}7s~6e)iAe~OMXs8m@HHf+z}Y35{PEUf4vP0=I;UB@ZjYp1Hx z6^@v;>;(-~jJQ;P>SaDEbnxztwjQQW&^kpWijqa>xV%fdyxg~lxpvWaD=XIN3WouMxwq>KFa?QO`L21 zPx`NCcg|h@MTd2#X+64XvT=@VPg{>zjORe@I`}g+PBDV#*SP1At92nfA1B+i zLd5V^3&5V`R;_GdLb6?CdrHkwu zW3sN-OxGKN^`{Uu4lq1Z^BCHV;S&P3x2&r&Gt0cLZh$3XMO;AUTM$sa@;kgx=p z2T|t(sLI1bfGF(?-kLj;??Zc?0Y;BC!dQ?G4GrH-K)8$N2C*YT%z+n11JKb3nv1O# zpplQokCC2<@#&UrrhhW!68j2oYg1fJhWG;2+~^)S!UKi?QWs4^Sw1x^33NV(#!^8; zQGGUb_qVsDI#XSo{!w~l3=yRvmQ*3kE3*^u>@vys$>e!vn&AVsA3|8OY;WhAHz7gN zpH0TrjmuF-)*H(x1Yw@ZpoD^)+zCO!FyNG26X1*?^% zQ7Q&BG(b?>Br!w~sIeF+76SouBD-GdI@@;L1A@I&J$v)CjL12U)>(L?+^cJ^dNBCNuJWK#!KoENcF=kOEVqN^2q|B>@aR;2XvKYjaezQcg} zElmf+S>IR!0AY|Q{4^&#O?VeruJ_Ncj7@zH2GW~Ko_RQvZI6vlpk*Vx35!?6(}ex-`Y}<=1o#;c4yP5e*)OwG6j> zFF^-~h_3J_o^^AhQ|}K?74@AhaWvqh-=mQ(p9bi_x0SII0u+$QlzUXJL${@#Tk2O= zv(Ya8o8)>BICC1k;x)-pl2JUb!IHWQNb2=r-sig8(x+x)W!1EvNz!E$vgu5AAPn{A zmwt?-8Krtk5Z3=%)z$G1p5qxN7e-nb$2tGJ*`SbvI3og8ez4<5R)5Y*u2bL2LpZfV zLCXm?PIT4NhUlY^(+PItHHsZ3(1R`}+`B(7$M5Xzw6sZ)dfqU)2nrS3bVIq3FYF;u z_+IZTEgl0h#AV`hKJcAQc_&H)CmlzcRH&RMOL&y-ymL22fo}TR(yEnK_TEwqmtvRG zVvp$p_C#mrlc$%mwia@%%Sa! zQYD|%&DJ&a_zMLTV?+(tojdYR6=#arKxJim_whpc0iYv%eszgwtH?ibX(N!zO0FCd zwVOx;M+OeBEZU;`Kl!z+ zr6*U|$)Vd0u0Zuj!vbCpKOh4|KA3ngsXGENma0VybPN))9W6<5X|`|37%SA%SjKya zqeZ=M7{b;4@2;}sg^ujuS_p9=SrdhXuP%A8U8q(D-be@(X#MfZ5|>oYM?y=-8YEfFmvK{#SkU zD4Tz+nc>nDq(X)vu`^Xi^s@l`gDJrLc;E`AR45CSD_lYEDFX=ULkiFC?&2}ykFHB# z3V}NFmN3Ln(X!~rQD8UIIWCfRnJWUfNqysh*3OB+WDcKI?%v$9dC**we%?kVz+uzv6EP&|&y*6(EX zU*US9Tl$8bfeg^L!N*bd@nt%Jo9f4uaW|zr0|mPmz9}$H8X_FVVqFQU{6NJY$7+s8 zFLO+|3Ww*7^hu7-&$q{s3)*rNuksczodkCj4((8^e_kI+>lZ-b*fdFyiZa8b!Tn0^ zOl%-7L9MmmGkzfF3dr+^z@}oLAb4j@&J6XBSz~_j7@YgiopXSa3naH=8XrDWU4-F4u#N*VMwcZ@0q818QO zGo^XFFOM6tA7SDCQuP!bfr$u{=Z3Z0#JVkIHNWre#6dQ-{jb#JqEK7t`M2bJxSJQ} z+d%|clVOK&Ea_leeMhp`O9=YJ4MBGTd)n=(@7 z=|5(AVK+HNyPt$;sODlqJVtK|y&};qc7wtLCqph>F=ER19Ebcy8$}=PfLh;=8VC)K=ctOdB1zNQrSH7!>s!y< zx~I_8O6bHsf!&xPc>8|Y>OQV~JeX4(xmIN35h+k7xS@v028w z5~uz+?{5oV6KVnZe5rCm=R!ChwDWw;#@alyHPz*x-X~+6neZkJFu&styi_9vpfci! z0%w783tV;_z}oQ8ZVOzJGR3-~|9j!Owju@-$Z#?Ht{q9hO3puUh zNJp+LooDL>()KmXu;g#&m2a_FR))LUdFNzgde>H$4O(=~xjv`tsq~6CRKEohQSC>< zO)>}}3Zo#|gt64*Kw~m)Nt8q0P|$n7`|aH>P@Oq|xx|2r;^ylt)u%FR8~2VKl3ko= zN~h_zquUVBh&#Sx&pb2#6R6im_x}3oP1X&mRF_qCwEy$FvqxqqG5Gu zDpKeWRQCl|vt?yl^29_#9E<7v;zZAluWvb#;S8i$Mqn5G85sM$HSn4tD>sn7{Ltet zcNHgUN02dIU0)sf#CP?Ky^)b;MOVrTnLNPeTLxveIwa zTjC`%6Ar)|>I2Oz0-~44^s#j?L$Vy@u`u5U?7>Al06szlZuG1tRgx81@TB`E-AG%% zywdar7F~$iObn2?l=eShwFDgyX2GEA@@H(sHLNQbf<&J7%r=6I8z3Jblpr$q)jD$f z(Ilt%f3;q2e#Cic$;Qd=@pFrw)dN;_QMrI3bJsB0QuFRP(eypu6D*gG2 zKK8g66*4Q?CMP9NuAcvCB$Eb=g9qS{U3B4sHz=CeIk!1GsIG5h)626!qd}HwKie}K zVPDq5G+jP$EP|sf0)nA9uulUkZEs-hS7TmoJlzxM&Omb5o@0z4s5ft&h0%$aUM!QE z>;=jPIJtfR>b96!$6sw}wQe$y0YTC5-FC7>);NqS8oPGFIo*rYyIUuJsv=9HkY6!| zD6`BQ9ygTJaV`B#1<}zP^0~Z=Y^@ z6(*Yv!cbzr&6uca@a9fm;9{khr`)B!*f*j2-pToE<0f%hpW~&&v%$B<8ZHCpnWZrR z5mPf|HZ4>-JDcJvUpF>9y!|h^GdUl~+ z>d6gUBJtb=vn2k|>(jSq+RV$2tEyGju)M$K`!$raDYZRNt=spAcf53ec71%rz-b#K zs=9Fl*Q@5EcLS@kMsr8UrKJz_{loSH=U-mV*~Wpw(ju_q8#JWhlDXZHZA1vY^Z-_z z;oHW^ASi(L?)u8}vrXyRfD48XER5+NStS>Azxk!YklP(NP4i;_*z}L*e0*qAjcfR4 z*}l8z-0n`;el*-kIHycAJN7d zdA9w_9K0|xyZ*)P^hIhtk``2)2zC15=H?Ei3G2iYR<*0YfI|5EY&m%w#}=G(^FffE0sdhoBrXQG4B|7R zElJ3701O1;fhM+z*vW}>q9?#njih{1EU(+oU&i_7gvL zZYyzGUpxSGw2+!YTjIq(I50i!R?yIw0{1fD&C8+YXIC!!VJ5}rK<~ms0&-Ny0hi>I z%ZfE{2)cw~H$?E!ia|!c+MwXNf8BRM|_xyV4t z1XU>5$XL`e8U%HedlHV+2k|t1-)Y!7mu07o9!5+YBBPG*}K>*OVD2RXrqeR$RAlxL%M#ufy%U7DC9Np*`REL9%V7DQ-ePaDcfLKAN0)rk{YL1 zqQV9o6fCu{u)$sZjU8{`+xOl_CC3nnfKY^ZA4|nWSFQrn#q_|;d&j?M-qJfyR zFl@5=#m2UiCjlnKr)Y;o5X^ZIs}%66zq|@T1z?iQtkc#kpyX_QQOVZ`7Hyz?UFQSz zp)Lf!^^>0cF6hI2iU#uV{{&ute`s(%ZHbdQ03v$iIgkQ^V>}o->|LVVmzzrUhqh47q|&#OV19~`=h!Mpb(p}bDwZj5ZGvdw@Qm1+$Z4v>XH<- z`2@&@{?}{HfF2L10@vseHjU6tB(uj88uaIbfzX`bb-)J@f#2kbjkYt{=jun2s~qh0 zu=LItPGG6~0w4<|_dn2I1B?|Y%q+nIzjX-@rCEV~h1)Ff>p&IiE8T-~9gSD5!BFA; zXq4e;gIYJv!|rj^xXp)8(Xe^0_eqR=X5}6n41j=Q(9Q+0DAVcb^$DG` zrrTN*8_UUH33(XUBRJbvbRj2)3$8PVTUuH++Y&*E?=R)DrRfoaBhq+I-BT2&4$vKQ z6Xzs`QT37j@zNj;G+kIZ0pI!f1>4i(eWXe9Ra}46Z!NW`eq+p!iaLh8BEYSo6Y7dX zjddZckcD(tKw|&Tqo10J8;bF&v&9hOaehm{_q>!+>%Yo=5iB0>FZOYFL>(1?s$1O` z4D{Bgyt6ct2!QO3Ephu$?7-v5e@qAuAlFJS3Y|YH;Kbl(NQ7VkbArJ(CuRVqq$k*n zwW+6R9dZY=tSAUcr<|88P^S)c=Z1q(Ul$b zGLaBaG4D?edFMLcTXRo;|6x53G&&UBq;f2EOL z2M!S^Nyfcw{7iG*ml~WQ8IRvunG6_qTkPp}VAk12(ZvZ34VwP`N68cCI07<%`OpUJ z`k&o0C$&*Gts5E{S%S87?b|G0qgXJQ`@Nq6dL`OaV6TR}vPfDhAG=;k3m2l+cXPP~ zd6k)FM=pr{HUS#~sH>(2Pw?^SG)B`XRX0QCH{0k9_-o3Gtz-|?bzB%~hZ)SsT@g4; zmJK<_MQuUTXa{qlT07`~(S&}guU5_DETj7A*xWm@Le#i=`AKkaUTE+hK9h2UjM;20bslE9-%0!S+g*dCuVbf5WRR|OhT zEPx9c&37LJTP)vM){SGBxcY3vRV`A8dJFkg3p1w%>CZ^A7i7t*I*uX@;ky*V&W*M4 z+-2auL#rfH^SEE!`q$BkRwNR%hzBFU3?|S$ULj|2+Jy?rF0Tuq6MNpXYx_ z&$RMxIBl1#b}qpuz;fdKUY)TQ>+Fov(%Qx)KK>CAKKDYUh3W=iyF6OFhsxz=K+2&?_ond6Q8y{aG4)L>XPH))(^9+!G5rLUgs*8;vs zf1t!2J^dh*v+_YY$#KXDUS8yf%PPwdKkK~NsA>=HVc`bK7(B|`v#zN`9sr^`A3#Vv@ zmB2N=;_U7S4M(Y5^}VtIdylmd(}8%-lh9xSm^myOMAeTym7YCE-I0A@WM@1n^`9Ek zXTs(E7D<%IAzV!iJ9$E1!Y+HWSw_~pHG^bQTIn}Ydm&>L3eN%5NXzT_NN9HenEd9& zr@C&KQB%kL1s`-%WOtaEo*Ay`dvkMB(}A+jm7b|Ayt>R6Npx^B0Wn1W0ED}EIlDlDk$MlLn4Jz`|c#FvrhjuA1&|ysV zIyJqfC!zvFcWCAQ?B;ge^>0rB=k5zUIqcd-<#()fn+PrXATkcSPjeb%v zvLJ5Hq5A>-6{J{jcV8?>#E@+GX;R{Ylbt76S%GHZzy}E1lSqOb6F}s%Zc>2w0{oQ0 zrXK5`q9z42-_@m>5uGo!xUQV9q zx26Wn%gQRuaS7BNg0336%1)EY4BfcZr!3TKm`YGYwCjW13nYe{aJ;K14vy#4{uZhY zU8y)L6DmOx-y@`r5T*po$mn-`MAmz{FmxMPtb@`427ZgmhjKVH45JxpVrz4)AQ<)l z`_I1Ldx~7DR>v3f&0{e0MH29NUdGB-G2DD9avA6GKYaKHAotd0IsVkXqvuiX!C=`eeFS|~ZKlR@K_M!kT;l7FK4d&@ zuKd(E%WSwHzNh!NiY6v|1#DGXMjwQ#@%N+SkN<0L8#od65&}jhix_5ny8Qt0ee<5J zlJ9NXHgUIlS_i2laNoDBLg1hF!M{HNx63_#_gCQO&T;qr_KuX``m=q52x5@qEWQio zR~}BNw(UEh!2!6QabR$8CCp!Zsh&b6xzZef5C}Z*PBH;=KV<&)`&|A0jR^H-(-slX z!F%pIGED=kY>{GDD3SRr2hIcbMN!iloezeDv*6?w3N8Wu@V2I=DoI;`-C|F->FcCQ z_NO6|i23t#HcVfAfUkE7v?wqcupIW4vwaCLQHn7WMV4ZnJ}`1Hwh+q zt~E-6J@0{5g`-!&NP5I$eFLBDJW=|Rn!3AnP?e)=TgT;{ZBj#moR<^$HzQOP{F zMa{s8JS!7lYGVc+8Q$Upd<{I6MCcd)il9KE$#UH9iTL6hHbZ-&Vc^rJ6qEG-w8$|t zg8PhMFSwNr^uArm!#TYG80@)b#wU_asclQvy4%POw=0GFk_VI8oyg(6g=Y{~Za~zPBh0gHhX!7tzKtCu;xkTDT2b59@K?)0ZRbsr84LVV#H z?!k$)5b6QMIP?n8Hx;({h9}r#trm5`$r*B#L2zcg3a`%@ygs2Z5C(5k1^F7&{kEl4 zzPZ!NAb)-3e(lFUGJ^=--!CBJ-8+zUL|^ zD?`{mU@zouE(a?s#ZE2Tv*tW*{yQWd`_JCI$s#R|r2J=UgN#?%eRx?jo-@`nh}64m zzSG3>VViOd030Ur2T9QU3|LfZh{fQ(aSeDx{>OyL;5rR!O_U4%jZngEX1a|^MAG%< zo?YJ~uItm+ShOht86l>*x@r%oRlC)2(ku|^`oPP3M2y0UTU$-Os=X65s zYRQe|w*MN-*MV~B#}3}R6m^D!6zJFFDL& zi^urQz;>k~N|?u`kh(97D0l$mqe-b-<)iVnfZBng`S;23^~ITAK*PoeUTniN!GLH{ zJKEV*s4XppaFk4VFjne@SUM)pB&5JUqF9ae>_XpvRBREn+whfgUuLS}M3v+NRp=B# zA-UA`Yoiw<15|+8Ah=UIv%sc~?UKds{2}o>u2uN{xLr5U;#2KGcu@dX93o(%aV4N! ztAT-vm;rDp+W7pcyV`;R0rq(5oEsqA~vl^!`DxL5~AG z69&3^gRJ>I=Hc(Rz@VP6zZmPfz8E1warzEPYdlViUK(Q+X-%=v~Y@tT*@M(eE z9y|61aW*Ee{O;rvk!fikJ~ta(J6Ry7uNfKSlYe%fl|pivs-9<3TM$3zQ;X(oy}ed+ zHwX^;()?H(ydw1X$c1R99|m5H!$L8G1(Rd6nX{3^={6%#&P6|v>?z0)Jhbb;uJ_j! zTHM%CcMbEExkb0rZmu@M+UA#g^ll3$q$~Q0et(jvoyW^K9#*DU?JozDDwTy!jJ7gU z3%nYo=Dg@@#tt5T%ueHhpWB&#bupS{M1X`xPFEMUvj8=2-dWmF@;1D#hf1eA89@D^ z3{fnn)A-pC_!8BUh>kze$F8(rHcQXF>9R3__sVMtm!L<2T*0BeV?$x7we~Z$h8MXUwo2N)L&eWRxbWFmf&M8x`_I4hlV6T>vj6%pjmwa`i zTmh(P1P*26Ae96v(KSe`=OR#zK$K0ZfTzf@4<`+iLcp|KK^r*cbK)5rO5o7h!H~ z39RD~6FdCVXx@0<`eg`?CqGhC>H>$gWp2zzEt5yN>>Z zBW+jis~$rS@J{wc*8V1$Cn-?qAC19Hb#!ktZ+dRn zmPT=;Z9hpG)AdhLhD%d-#SP9yD7KZk!DM7rP3G%2%2vM2m?g+C#;&Lr(DJ{uF;t8S z?{=|^7tV31(JkI0!4YM$rLeTy9lk-$x?JD=-8!nPPK z!WbZNB}3E}evmXJr+lw-4!JCD9HFTz3^l#|9%eb*9f|W!h9pNW2*!sYgJ=g3Qu7&9 zJV3LC1tdNn>`RBxu#{dmS~*Yk8d=fYrDEi;^ur*_nj zsvju;>tea~LQtto{guhyT9?I3Ic^F@u++z#Eec#mlV)c5ZEn^+<2eEI!$zqmIjhzN z>1ttM?qqAw4NG2MVJmruw73&+jIu%Pdv`Rcb`Pm~Fq)t>D(Fdi!m0zC;<1P8MG?-c|H1k3Ay?l_)G@?`GQ#==2xAPa$ckt zxbmMHGA5DF&&Gzu4s+?gstp&l>%^S89t3%xbTJ(^Khl4(a46L9>+RrQdECLHhUjzQ zvGYMnsF87}uM}48wyLIP7_{b1p51=yun6nodJ0r9Rtq3ixI%Xr{p;?hlS@mx)M zGLs(B>h=nDt5iG9|9q+#qeCDhI;q1%c`3bs56XIpRg0MfI!2g`?)ujF<141Y3vTjl z+L=1B4#se#Y8^-7?s6wIcf5x8(4GH%JSVktwuD5Opz{4C&Cb97SFMxNH?O|b|%PYdHllVOU!&rT(TOm2R- z9FklKJUk^n&bAv*F$T^5n$F*o*Vz zd2d56YKBEwt462@*xVeew;z1O8~7?N?)#fNWw*>J^xR6O9s4U_O72~lb`)`x`TAv= zJKb9OtHYAPHM9y6*k!|nN3S~F5@K* zVXj?WuC-}@nqtzf6AV#udF9d3OTKECM!JOo4^YiNCc-cL&3$G*706t06~T3NL`OyM zh<+4@mWpmcX1ZZR^fMrz*-IKLMo5H*#O4HC(K5r|IEim;aB;FJPAZrIjTeBC3c=tj zz&%@R(ioE~M2Y*n&bK&0fR%3erdrPOf_UyKoXFY}-VAxYSCpzfk1)YZ_MvaP~Gjs%01cadoNN>gh4qXwDu2iWbAPjx}>luoe-#*v5 z&UMa*^WiriqI(PTKJWWJ<-XUw*1EaDr!x$LuR6W?y?RZJkm|0_Q+zft@{F z#)&xrhhA0QG5e~1AV+xH5x={}-x()8yQZZ3KqF>oCN;ue>2zJ}nRl~ZM<7(IVGf%``|#qlqvD zS?2QkO;2-HZleFbq~uk4%S?LwNMG>j6a`7Vvs==v=QpEHWR*#oC0Nx`1qWogeElW} zbf_<^FzW&fcCF5vk8@&_&sxO=1d(|uq^4Rfm*cnksIO|dY{t~`T?Ro0E@qN^Y{wE& zmae(K{I#Q-gKs(Mjqs_6K23YNW-p_ayrXmSy!Uwg=qCS_oA$TE%I$fK+(_)J>gwu= zW4xDmaGAKsz=ec-*OY2n$3&1xoTw|FqEkPePlB#(BvQrl5VDfd@=(;n9w4VeHkr76>*SmQLXi|2C8CmIX(W3PWosv zUv2vE={-akvj+t^PrR3`JSL=UF_EtnD(hCawg2fA)`#v6uhjgpXFbcTM34Weq<2!$ zz-oIlIl6z5#TF{{8yJ3sL!N?tw0dGY?+z&ZO5}7*R5!4y*f-e+pD#tb{^(G8m71E- zxjVUunRj8%=c}5}oo;`$B+!uiOsqKdGQW*$;p^n6bG6APDc#W|ws7UBURtqpb*|Iv zU+c_`C-o%xM8!oC6W|tctI<(AxD^s6Fwg>UnA#6MQ_$jeJB+=jev_D=KgFkewyeEN zUahBxzS}S)-%Dw2D?LEuS(@e`=|6nl-+mLQPOI zJ0HrNz0K@&eft+bN15VGuQtt{=ovCg6Oev)yM3wreW7i!C)FF42JaIGl%;;P!78bf zWpPEp&RVeT#$mcmlj8617sbeTlpk#%Au^hzYa9CaIvWUECCH8ZvC8?3ASSOM## z;{66x4~TcyNk|Wu7DgmQTMisDcu9@Q40wJnX-9NsSbc`u$WrEyq|5T7;-7w9ar^pf zX#N-ifZL5v=H0oo#nI6bYuee;ZrAK|(Tlxw$&#}{UXHXk&upWsdBnx%y+4#vCdEpt z#kj-RepjrGC)BB~4v6yXYYn6J4Hs^xHdPJ{*yyM!(r(~vXkRRXu*v^W^Co8jeOp77 zayQ{;a5)^G(25l`{rgTG0*jpzPPRr&dx%x>A9br%@`Threee&=RDjnBZVzG_nEPDLBU>k9a>GK<%b#Nm^~flqr(wuh2E#a_!Lqsjt_N)M!h4QxhrqgS5F{B?t-Gbq86ZGAIP{az~Siv zwV1ons$qIsOoYsN~cj>N3%7Np)%zuRB|T-KqG)1CE?koZ!XG z;c}&!iQ1I(dtW=3imSB9i$rAx8mpcx-xZZP%~A85?rpEp*VjIx4u0rV7i+vLu=Z4p z;98>~-ccey_9lg+5fczm>oy=#n;_Vh2e`4$sljxzzvP1gZuESoDKDEu&Ug}SI;C6x z+%^Q~sFP?RqCf}7yAuIt8NfA5_d~PujAm99Q@~A)i;ie=sJdF)+sin-DnZC(dZfe; zZ2%CIY^XIpw?)@F&64UPm#utnz0|HcmoDs1+ZlmzuV_?I-lWIg*vtO*6V8&T?aohD zum+0)W3^PEqotZ0!DiyvQ5+>I@>Eps)q#td_r_zMrfD9*-ksBWm0CA2HOzpw#81yt z%Xi4<^u|}YQcqT>x^irCBlaavvCkYnJoHQ~;4VLzYG-*>aENq zwfZ!r8iQ2jXeM)~pfvAI-Qz_h9b#TkmGz}G5>t?&9%fSd)bF#e=7dO|tDr`V5SmsG zRk(hzn3+=9tAe@H*-)ULF%+k6XlVaZfh9L7z-ck=Z^of;eJ{O;U+h_0cD6cAe}BeT zy$ONt9Xrj`8z221HKmbueXq?f0}%CCCcc6ZT=$e0N2>c;a25qTx40r=qmWW-#8Qr4S;A}}X`pBayu*+ja*wX^&W{bJAN&Z;Y4CNlmJ1$< z)rdLCP?3r)yEA(eZ@gBl_bUxaYkge*=y&X=5i>ItofSz!Z5Q@vi`bNihB`miwkn5l zr73rMgK&cE_m=nZ`)%2*5eQf(PlQ{Q~-@Z)#3Q&_iV+1*0=%hn4MFf(nS@L}CQgP>DWL0 zh9zF~Fx>l|-!4vrnDpS&d*&7Gr4o#4+T_?IIb0kslasH04+%e7TW0!O>`};oCvlQ0 zqQ!8pnehW-Z!So~HBrL8{kb0hD&D=G*J@lq4)la1wC<=yP-86>Tuz-5D=<(wqA&U^ zt;RbhTyM~|$cL4XA?3jp05@3qPnqs})MvdtOh$kDPI=oWm;C>suL1mM=;`Vx(~Oec zEUy{!I+hxWjO{ot*ABz!Rj^ZdHdj<%R*>B75tNwgGI2W zl$oOa+lx^KZe3vs+e~Jr+OvBVMFv8w=s9fFFP0xWa$Rxsb!7J>xmEYCdAADkeHm)v zRRvz$T?UyyU|K5ecOJOKP?M7`+T?7I;7c}Bd`kFav;8FFUNW_*jy+dnGDqGW#|6NKB&FDb4yQxF;G3}rMqs~zz^24#ldze1mAsQH(6hQB5-LVP0+63oud zo_e!X{&@j+vcMeqjZ1%8Z3==k)9aEo_eWmv?u~NdB29C`^6HNl=(?#i1Dy@~uy>@Z z6Wo#iUYv-{&i!72yD@+Kv^CPKw4*QO4JnFDObs}*0t*bH69Gk3an$G+mnq%cCE`Wm zcje~&UAKCK{L_9aoaP8$<_|xe`Kt+w9UDXMiVzcwH*`AXT6pJjfO=!=GX;<7lnSZ3 zd)<>6nuWZip`*)r;oOr7n2vK|2@=TB(UH|9AK}e6{DQMD&6d-%s^*9U27kgVn(Qg1 zVKRGBaZ?5RG3+#APR!Eav?SSY@a_r9L2l-XrG-J?Am#&$Ok9wEgwz=cWFQ!nq$ z=W7(&X~Y{9>twUn)WkKi^Ylze0RQmmnheL9*pm^QRJ7y@cg$#_pQ=)fwapueViibJddy3N@m#XDK`RvA9t$tET3`97PN zp=UTo+6Kpk8p6ChMYbpA*x1+TnmyJ#%grLe1I?!2(t|>Y8w5mP2BQaK8;^`8r;@{f zP%o^WjV=!MC)|Ionf9Ik>yYdgRT_{Iw611-wv@z;ZK*TU`f5k$r?br3D&95N3j4|il!oay%>X|DOd(>30|oi5;09~I z_%vGZAGI3?a>T|TucfOq(?Z>}}13(y+4o!mH!IyjI~pn`Zxdg}eq$V9&_lXZ-q zH>x<0y>mQ?yMrM_Hu8bY9-$X`87KChR3554jC502ekR#_)!jANwhoFq-X}O?At?du zOw1%@I2N_nq{ycXl~w&njkyhH`Vt@Wh%on=VL(m--0=w1 zD0*4z#J2)5Z?D#7ELo*&lNDupVCoDx#-}-ZW!r+y=}Nm5_B$%7N(&->Y=7{CzRl;D zk90JEpC~`qMmyGzOZ_KR!{|r(MQb<76zkMCW+{+Xqe_Z)3Mo%kqPU>NAlo_rC0-PW zxCjs@rxX!H47zm6D*(OVw1XjWSP1x7Th-hxH&oLUkyM%hKh|L?qNwpImK7tPH%QyNhTA`~;&pfKju#E4SFiaPyaB;kz9C?$-;3aM*dnVHrW^E4AggT(p0d~!|hpfr%!tNy*<^M zVKTy9w;0&$M4oLrbCCM$AxXE4H1B}zB*lJoxjM0CDdl!I@cIORPA$h^*LfCg<;>y# zM;x(&U}dML%d3V_C^}J^vd^qPyhEKVz!38;*LPFHhv~pIt+abrk^bsjo!cHV!oAt< zF>CG!sVW<}#wrWw1LwZJH;$EwgZG-L(nkAZ!Fn}%)bytGHO2Jy+~M`L5$;sTfkM0w zuTqg}*o7p&{k2J;M>zTQDk9qI`Q)rVJaY ziSg7{?2JA3c+hX2{DPcs35xeY`nGv#HATF~4R0!T8(rQ=hixWe z8hcE4q!OoWwjB{L4;L{xh1wtH_W4!J!wIV>2PI)+eG5*MvM?b2Ex~s#ZLW-#+_Q&0 z6L~c7O9ia~j!|1tpm&_2o#;)Fgi4wtob+DCR;4g%LVB>1{`!QD-l2R4_DORZP1CB2 zQ$45YOZiW4;$z1gzC09H4EyTdct(7_tH_aQ9PexMZ%maGXr^*YgQLC<6!4hv;~3@9 z8*G$pa*r3j1j623BdVME!+3`nI@}dk)P5=KS9&~BuHG|dwBM-gFFYk3K@kGDL`*hz z;oLCqa4wiMOmA{(K#Nba;I5X*W{#x;b@pkVPmjvacj4Fq_fYo`Nfe|?wd5-M)SE{X zrZLo1A?;#ojg>poFRWC|{}5$AGc&E!8Db@5oBP6*ih3ERjUVCq4Tk}PQRLo3(qWFY zaS4R5fMJ?)+&6!MDC14fPgWj83OCu^D2~!L19BrAiIM|MCR0O)>?-)&M^agqGruQj zh1pf9kXo5>zR~5)K!6dmynox_`uGp^r#&t5!BoLmrj;xy+UAVbQ?;-s(#xcjW+n@i z#CxLx=}k_|=m%1Ezu9w|6c7>{xk-8yq~9Vn$?tR@ZbxlyDD`t(Q;C;mppu(7@(phg z7{^>3rbCF>>>dRo0~##)5@Gyzx5?l$A3Ji1xeGeG{T#-x*N_yw%y zU(OO-wG?f5gTZ6!l86{r-tX{Uet2$7yOQwRqBlvAqinAj}G|K zO5LEVoPaV(x$25%P?->FLWJ-#-$RnmYv!!|tK+QH3L#lnzqFW$-L*KJPiJ8X2Kcpx zRQ1(efA;jAUwgEgnAggJv1h(UI+G8S?eIn{JofOR(0y^)+D_mZCFq{Y5I_TT<5ZSV zC{Qk+67UUzU`wA?DE!aaQJ%Ndl6?lCG{6`eO#4Pxi(WFaMm4Vx_u){p3huBMo!i9~ zsT!!!iBxWmd;H?4Fd%*VH@}x8a$DDS)IC{MGWLU1(;jG>L-Q+Ut0@uHPW_6%_jSpO zU3^BcQ0~>BR>TdJ`_p)Zy+SmR93ni&g7GrjlzK~MUQT{jxLYTz1QOq}<{sW{7wmlr zXS;^<>fgQS@OZL9U#(DOuV4;4zptLIj@C|^jO6K2lDYaGP>;51I%hJ}cow*N)GN)4 z+Qx?Y?fwEwH~E!@?KC+>0uN?ftSfL(s|13B0+j|7%9<~5p__NzB%+j)rN?YC?}QN4 zKP2KyQXxik6OOklZ9MTZ5cN9Fzyv4*!BF<4T$)2dDd%gLckYasx=%)^o+G_t+ zT00{m-l+f;oZdtwpM7_gKPb|lT`SSk(zOqEeim?Z>^ z+4RwQ(^vOY=|Zf%950 zwC=kVnnhy`9jmpvYm``xY&Sz|N_+=1*0&G`{Y|Ls5Gm;yTDO3+M(Wpm!J-Oy4MKp%ExhfU4eJATw6)Z^G+o92jQ?4f3kax;K(HsMTtXDBH zT~0`^-vI}Sazqn>OZnY@Kr?acg`Ax7aLfGmcX^|_qSbJhHbdN64``Uk(Xyc9LKm?^ z24_opWP5W%HV}9kDUR73&aYCj=t}{Q`V1u~o7ket}t087VC2 zIO;pDOX)f<1kEQr$=9YY;Ie~cSN8Ew4S zQlSK?9~NTPK;|~kTT(S2 zw>cH~CNw>)nMvV-8Ap?En3g6*0TGdQZS>R%n{ZOKGxC%UmX_e#|8VeZ_DD@Ts)nqf zJt81dnX>arKIAOhz1^iMnn#8z{7ze-W!eTPX_m2b6EL0PTI(>!II`D|W$u3kAbw97 z{N3o_*b_d40^Pu4q#9U-Bh|J!*8zp6!kD>mgF^dG%RebV$7>n^MSYsGt$yppwe)Hl zBl#hMmXQHNp$HG|BrCZ{2$fI-*wtzV4fZAKec98a_i)Bv6Ga*uYr{o$d~;h#ZHJdv zlaOJKP@qu=x^g;7V~&W}EwG*(TiExMFNad2lwT3h|L!$#UDnwWzXyQXjNHY?r@EBBVSa^)lElrlW7U~!r*^p^2^tzIA*XF` zDtCsJ^siFCLYBsCw7m@CJFyw*(ywba@$4@yQ-P=;&g^>m5V)WC5 zEpzG8C6uf?aG1ot=$>0~z*fSIeQykYcEoqLSuE%KI=RH}!m^YC)gQ&h{gHO&*OV&N z4%f=t>f3u?iC_mYtx_Q{foO)o;6#a-aKw~p6(>SMm`^&zy)3;hI^MBA#K85`(4!@& z!zg@AS2->(@0k|+wLf0Q_@1R?$3w;M7i-_4R`v-Uc#;) zXlz8#(W;u2``<;foh@4isyD_Lg7@{P_(wA5daIe*3BL1}tlWBO`OU#!wG6N7n}>DO z8o3DSy7l)^3DBt1+nlE?Gl^RF#5FM?aE4Tf%z6g)`yk3Cn=tq!6hBKUW}Q(&lB?dQ zpL=ecJt7pkM@@)istEq)FF~I6>@mNEQXp; z#H0Z*MctMY7vQD~uzL(U(@}kX4!*cq5%A944)ynk!lCPbx zg~ab95}E%zT55^ixY>il@v2#ageGso5&_skAvZdcSIGTlTQj$Al_v_IjP|m<46sl~ z^sM|-^whu7Ny?+C9P#j=o;i_HzW=%vW)`r%SJhe!af@F_GW`j5lz59TW;DSTXT`{AB`#fI=~|fH5HMtj60%E|WTO1I zsV?&|fH{q5a#q>*CPcl_sy0m%1OZQD0L>k7(77iGD_eNX(`^JXjW@cO+Si{#(?@Ld zTUQg3F>GQwp#LZa)BL8kv4><*{)8p84L-zRx-_|H^P(WKvJj-AuXJSHXsO01fNlWr zz@5T5v)&kFTh28xV7EO?Z3AVjcfZm&EGrS`{tRPs9`q(@W3~%DLs&jF&1M-o3)L&yb7?Ig|`qI9KIZE%{>kb96ik_CRX7kkL?X zi^f^@)O`zDmo@l;aZGx|lmP6?;ztI;uv{c33nEIy)O=*6x+|9#q=7a?iiAkQ`tyO* zIP?a9lr%LI$O;mVdhh)BON9tIGpeE0^L28$%J=50o+NZB`6-*^hVm53g9E($28?D? zxIv`z>Ygbe=hwdZY3_AW9-57eltqZUI#3O=QDu>b%CqIa=Q9jLRBEf1eB9&ObHJoOdtT+D|${uSq z{p&wHKw8e5@cxw;-ew<-}RZ zz;nEV?b(Me#d=JWImiopcZ~s@P;u=W%1U7G866HqjcFvt68M5fB0`hk>%Oct8$e$ql!Y z6IJoo`9vCcoEhpC75ONpL4641Q#nj;dZx_qYN*`wbVON1i|bsyYG|ZxHuf6rqrSBP zxw)jPrt*gS;5D*)$Gh@|4Qr*Sfh98`_aH5b`CM&2ex)QlA%Gb-KtI6S#5ci{-MQ=8 zNH^nj?%-A7%DgFi6a-|Xg2lIWzfM-RwpxkQ@QU-uh(eAN3vix1*JXSL;NXE^tQsmX zIE!t>-^gsB1qtAR(bpz?@}BO*mvVjBSMy6p=lB+;Vi~Npv5CBqo9^z27b!)C#EB=Z)$L*f)hD0 zqO{-9(jb&a4!W0^81lG&mI3Px7UDp@43q$C+UCmG2dyHJdtS<$ndluy${Ye6?1P$F z4FDN>$Z?1?^hip>OjP{a+L#SnfOD`a@36}D3g@uHz1Q7D17_(bN9{;Nwh2X3%iFU# zd5w`~*l?0nwekg{cgS27GdKXXo_K`y6vbCwY#!nf}SK*M#`8SULMfVBXr?QOhq z%U+SpP8qfz%Tg)2S z)H#!4EWU)fp0_dIkMxOK09ujb-UaPx;_YR|QGS*B3^f23gKfBrZ)aD{k}m`_VrV&> z*U2nzJ_6!KhvI)tX7L0sByt1ie`*4_z|Ph9a@uJmAp8?1NHlv-9{7?xCGx2Z$Q?W~ zG}acSNz>N8YO3_AIxXCrGx2zBG1C$N9BKpTlLxyiRO$h2YG!cfs$L7~6okp3TwX)X zYbWY-5l9=N-gb$sxA{G^Lp{E}n7g0q~+?9%I z%1XIRgMtt0RB$IA)MVJAmIFR{l>`n}pM6_hP>)NiRn+YVX$+)0%p#cWgd?;Hz7G~c zN4`c3_2zC7MiZ&RtH0pNtU4%K8ZIjX&2n*HUFtBL7O60xE{9ls=(UM&2SS4q^7r5h zcOg6us|4SLCH!!!nyi0mh<|Cgp%Kcm-u%W_)wQPgt1Fq_ayDv1(=!%LUl6V9M9y#W za)ewV9f$!S}Z>ZCC4aJ+C>(lBC2dtbLMzX1^ zJ{@(-U`3EH)QydC*y&{cP?heG6JW0^b6(geZPM-zII zUD;1hwg?+$?~&8rpCt0Vn5vM9;>-*OWIcLDGtR#l@wKSL3ki`*3&30#(rv_mfRK1Y^g!^*OO7ijIA6j(QG%x%?O_BAbFv>y#kw72?4{T zQ#xR~D1~Acpzza=nx&@o5AB;`)LhurJG1)J+lh<=!A%^mZUXJVVYDUoN5NiBZHj%1 z13JQ<%lw|_vUkavAo`#EocIovJ52nSInIu5iD@)KJm^P#%S{*{$(L610)%n<&wzxG z8#|os&`r1X91cyBTiAadDFPg?pZ>U#M4qUGg?#7_RSvrGp%3bdCTHPIGD2?s)gPu2 zqDqtCYYX9D?k<=bDh-j`xk_&A1g51w`k`(Q1>tcN;^%pG6~=kHv!|EYg!R0BnH7qH z`quv58v+RtmCWDzx*5{BwtjcM@(e zU4U$#uy$^(-#o{N!P*rRH$8?lMmD)0TOR-P_PP@n)V)I`nIYc)jo(RY2aShMBE;4U zY%w9;_K6BUiv-#H2voZ(@7SI1grqnSy;5^OhTIg39JIu9nHn^m_;4?A_o}IEIFh%_ml!bS!RIVIhmBIO7xj?;>?MOiRJuq2Vfz6 zgy2!I*XSzTV9Q&ZOu$Zy&QTpRs2Z1V3IWBs%oABkKL~8#5_oV_ROO_>7zV42{`oQ# zVwplqkUU->%m?w)uOEOY5b?Y_CL)}a$MF3Xs6ln3#n1lBo1An$qUMwb#X9P{n8jD4 zGX-&gq0XU1cqLBtBGPT%SV|0pNRU^N2wg{#tf(zVH`@K=v#<3}!jMZsJI2NBFwB2xYa??|}>Mr-5i*6`UsUa(MtMCR{a9_FW=2I4F;K zf@thcC99L(4%Y`Y&;?Qp*IT_BbC_SK6y~KZhvS;qAVea5t15?~B&0&>nr=O@KwyAg zgofL_yTi&T2-~G|15Zsk#7{Yxj>g6a$nCqr_^k^_J#*p?4nlukv}Ec5%x@GXxrJLQfZX*biihGQOl79Ogyf-@_o;nf6}01HL$^z?LMf z&TL(HIdSOqM1uEA8-(^Y`7P(8JM<7uuJ;CQ!*T@l%%tXX^7f{VZo;g-x)PKkf`Iv! z2HnISMq<5JsyryvvM-O*oP`whEL!~vC_ofBRBx4J2G1gLNL#t@U$~rHBDg@;b}M>* zh-?R;A)`eVs1yOpsGbv59kpmyfx?;nbRJt$7Bpm2Us?LZg!&`>Pn0Xjyml_-K^e=$ za(^HVwh1A!9csdhvZ8)0y=0B#LrS={~41^lG8}DG=VEynof)4o(c<-QTPSr4Oup`LC z2ZEA~HwncjO>W_6qI_p*ao?$c#b7|#={1>QS}t8RMA-hv501ZU|P1)F~)iizH z1A*ygqGG*-B@Lg|Z@%JpeXIg`HEriI@3Oo|8$--31O ze^_FwfKGY!7+Gup%)Y#`cyw``#f!o}WT8-HIn@b{&u|#mr4&_VB|$b|R-hKw4h@=B zl4uebfk?6^kg-}8p;RaJ8;p7I(*+S2o&YJ{&ptocIXxw=nB&zz0!xb)hWhAR!)yWJ zFirT>-!SlSOlfi=s8vt>&7&ZK%}g?$tWuL|`~lLwc(~hukO(PE;A31YA%Yw&kP5jA zD^Otthb9asFILczl?*=sm`Lx9R;(yVLZt{vAl1=QlFv3b*;6 z!{RfHK8|uPi8|OV+@{G;(8sw$Vq7|!v{Oe$MtEsGP>HBZ^Mn>GcgeVq`#dPPQ9$gi@RT}bzI-X=VCWr$ z6UAYGcyTIE*Y=^qwyw!_Nvh}67+Pd>H+2D|b{uVXW1lSHzJMYb*^vYq2`a^?h7s|} zZjQK`{Io+pf`Tqs1(&S1ioc1*!ksXZRl2KwRho12_&c7uV&w8aDYseJRG=1Z%IPYY=`Ell zh$ZrIgu>I*3Sre-L6Tq~)eW_PKd=!!HVK6U?x-1!!@$>smn>ab`o)IRH_&SI-|&h* zI4<`<$hT*Bte$EG+9j$}< z2HqVY8zEH0cexna{hXkNCkX%-_`Xe3cETAC?474-z(P78pNBB)|gUDIBp$=iO_&QIVY=gT*$m z9gtcUPBu9yA@7kMdTyFG5+XbdLdBj8(TvmKB=QV@yq;3lgJivwhO~z`U^bkcL+J_u#DlKt&{pJ>P zUeI+g)phXM+Ca#kT{tu7Q8o3*H+ZkDkrCDhiN1B-He&)e=d5lHka8Jnk4OJct&RNi z7e0S-kMlrc@Sg4r|Gtr@tF6pa%Z%_fYYSfQ=FYIal<6D2rfl%fd-+`|DzxoWjxfUg zlT*wqM-g`e)}!cjU1x23{Mi18c-8eOtRd^X8MgN4rD<39gnD6h(29M#UK9UEl%i2#wvsDP0N z&XZchKm_#!3rjZ1@y+dOb*i4p9=Yn7p;B1v|6@bjA;a2o_)@$4Oq={r z>ou`X{3sZMPg=r!MiXp3cFKcK13o}Z`-YaWBLjFljrJv?S@6)e8Vg=H2`fVgYv8}? zw_Z&kDPsEg%GG{P3BWMTQR~Pjet$q`)6jf?^hhlOi>^pOVRE$pEc8p*Ge^lrk`L3k zF8I}yf^#x5u$jPexP@R5jp+7k##6tM)k+W}vRNd^0cDx{=mmp64F(p0hTc^Yw-2PD z;7q~WW&}76Z4*M+649>X9p}*58nqz2qO1_^o)<3!n2ecI9CP=(B*ldU|>6?Xtiep2iim z0?%A8BaxL93oD+^b>Wwg^4OaWU6vOBGY2?fiD|RF5}+N{XoGN9cmp*UvA@hk$&WvN zyLyt%-K|khGtL`_64PkZv-K%33bL7B+7KeP7Kbf@%8x&5T)H*#lg2rQqQdmNeTT9^~93;u-!sk5AwnUUb#Q+(I z=)3PhyF53wet~%=6D*bBR|}17E%90^Y|JWY_T^(7`0xurW0tR%feOCxY!5j{KYsRT z0gZ;oL#;%G03eE@&7r`7hpyZDQT4w=j5^BG{i%D)440C$p-x5t)eLYJA-%V6iPeB; zwj&Qb*kuhy@pTc?Z6l@maClH`MM z10>6@&cS>HuDL`y%5&1FD{2@f6+h{Xs3ii9`1pMfQLPsQ8tw4z#_8mr@%pO!^V)n^ z;ch};7r>cyDs>0s1}_uzQ_lQ4RAMXyME=53Vc}ahK6Ep@*K4V)oR#f<{6f9F?kWZ} zE#aQ~p}kY5;kdfz#Wiirq<)zYTj-;L3if^{WvQlz-IEE=L9}-W^!tw<{lmPUL>cn% zU!r(X^TKx?)Bq`i1-1pM==QY>2sq2GNGlMs(2{`a-37VXd5n7yl)}~vVs79+qjk0U zH0zpgIZc7?Z^2O)$^%LhGRc`bZ1#bV_9-O5h&v8XWn-STjH8LI(deYMKr$?7CT z$C5k7U&X9H8*0fbs-m0AtcMBJ*#-}!Z8}Jtb$FG3?f~g(+OHGtgaAUf~@wCakd-fQ~^Zb$gUmAi6V@2al zdQia^Brg7sTUM)SqTu|%viEO0#^tCIW34!*lIs3gzubX>g8IA`9!hIJnw^ca!|W{c z%=0bKTd^KrWQlnNd-E}tZ@zWw%pp*0sg|0O(tx_a!~8f}3r0_GsCkQSDvrzDwE#o< zo9E1d>WG~Rl3ToEWXgc^|2u?FGb9z}`Om@7Jp`McuHcdfg`O6`)N8Pix?d31s42A6 z;V2*MG-AINhuKp!J3`>^@I5le6V;1}UX=G+U~e%r%@$s|IkRs<%=6|(4+n%T)n=lg zQ1OggzgAiJtRQ24P>>G)D=9ylWTz+?kN>qQjx{zMp(LIt z9h2$!?#A+suRnMMIFSmf&A%uQ^>|F9h2K42bK8wW@Y`qCHq_!=EA)k>FKgCmz|QUM z#r8Xwi+e#g(ohwt2x*88l|q@aGX0ay^ZgcY{)mlkrwPFcM4uXf6tIbDueSoZ6)ILD z21p$rX0mi{{(OzTw?!Em{BbI9mjy)&8O+$%AosS1yV1R-wjn7@fvaB1Kz6m+E)cMu zegie*W&<_mPtVS*Bq4a(CS+njUQR+e^d9vCh7){+zqD0CsX;& z;dfQSV8GqG^Xs^|SV7~|xR#2YmB|0B@>fr2V$c8E(}mkYw5rM+8v#mjB~?59n~-H8 z_eV?%j1*WNL_jR?p264J?6$|li*f=kWs6Rt_gj`P_7NcZUeG{5HYNP1|5>(&evGWg zV&v-!vyJlRFx;KT(+1m=wpjJO=8?qoK?&fUmb>^GPHWU;JE_2X3nV8?%n^SEUdF=b zQPCb(2a$NKD0m8_fl0$-T1jl^ZKSxN!V=F`Xz?#%LjYw0OI5JZ z%gcW3EdieMT>aBx!WB{V&6=5jg@UE)0m^MY)Y?A&V*FDwmA(zN34t-aN{ji}dywGi zh*6!US9u*H1*gHH&~R+Nw-#@{6xwfEYIbwv7-JQbYjzpal<55#P=Lmv_M$@ydW$Y{ z2)Kz<^S)CSIx}@8VkkBRSCF*=ogd4!*Plq1OLi*}$?|{s)xXV{!au%b#l{4*);HJ& zrom$iRc)|R$#&KO7mbhCeDD?Dp6$VsR?jY`{k{Z?-nj{(#n42Td`bQOUQG~C9@txH ze-b8E+QXy(B8xgqi;_u5;JXr1eNxs_;uU*Pms)S>@h{5#$0UV>BO%e%BY#U6v0J#} z?dsRqZC)H@Hn9IM6@~G;m&yIvBKEn33lfBd-=xI1#%;bZ6vigPCHOGreo!Y~4ql!v zqc9#i`59&B+QkR$c`X!{(k6lFEr){u^6*~|p`s&_P!<*niYEZMc!N*hmEGY>fZFD~ zD`C;LOXgI9yr!+HINdzUJIquD4D04y#V7}>fC_y4 z`Cnp*RJ(3!oG4w1rx#*47j9hz-D6Z?cm6(r%4DVcsxICp&MWd5T6QlhA55hN8iSU} zbK4?p7LYjjB!q}$^z9#X`Pno7C`d;iFL*HO?^Acd;i|BaCl*b{w0#r#$qsp2{S#r+ ze>t|_L^7=`bY9%hg66LiNDg2rvw_Ae0%#=oeoT9InN5u$H|-g9gNx`q(N7WFiH6Yh zC+xs~Jk0$xcBWXjyn9hS0KQ&uO**H!*b+RmPZGS_bv(rO>6PoXVtJayR!!N0!0r!B zI{>YHbQyKGMR%UcE3OQeTjkF3kJ`n&cuAFfKA_V#PyQBdRgSLUQ}6dwQakOo;JYJk zGc{(J3I)z5==OP9*C)<4cM}txy8Z#0@F!C@p}aRqc#_?+?^OTd*s8w{4vuieSTs&1 zdW_CTN#DzUMR~9$@8g0-9sdt_l+la<0q4{RBWe%)+hrb}Lb-rQ>LgAIELkc9g(X~b zl_QLLt3G=ar}W80qIHe6G5BYG7;ol~dNm5aL?GO)#$c!=0y6kpEOwA7!E?ug78Lay zs_8LZ{sa4hC1NfuU2uO!~$5H z)mx@Xucq8^AnperKW`$lCkMVuO3HVo+Wl4%{yD|sla2IX=!*n>$TQMje41FC-(j77 zDnSjv3(|{^-$t=;oC`2A8l`8MtRMfevIp7V>D>ACWYb;s_Ye|40X2A7i2FoUiG1&i z$c~C7lr^7*ij=YYvvkA3)ZdrrkM5!zjLg1t>AMvNKR;h%-kj^wjohWPJS;TPFsEjM zI0(#s4lV;J98~U_m=k8L9Hr0n!WSp=l!sqN9<81DC*}MnPuKd;;ltz}KiNqtgiOf7D^Mu@AkoYZlKgr5+fVjJ=pqg+o=TweD1I^= zlv2N3d_LeSb^!Wc{FVPZi20}Szfa=-e@|lUbvRtk3A30~NKjo=gIs4^`N;M4{%DwU zHrO{*&coFS&aCn2^_JZ)B98(_o-xVF6GCmM48R5xHoy26rgF%h*eJP0Q9kANRxBm= z(`o=$2w4OAyaN&^OP~L|*3b{umYwKS;>Ix{hbS;-nLO8#VxmPPKrydu18bPpCSPI? zJCv5>AzFkIEJ1FUZ$7XOjL33`0=#Nls}_pMc(8KO*oiJ((&PWf`Sdn)=CLDoZiLRJ z0A6Nb$5Q#RAC24xpP~FFqF|3!@14jPDnK2$DFEiq;nWj<{io5cXcK#2I7-Yzgp`rc z6GPr%^lw4u`_zxuye&C28ATCI;u&Ey@M>p-j@%4;zzevfVY4UQJNSWIumU8Y)BRa04-;%W$Wl0&ywH1r^~tcd7aL0><`9&zLp0 zJ8;JQ5{xQBC*ik!zSiLVKt1}b1VL>oHf#rO;Y>~iAwv`81UnIR4l(fRmUaSGcL7mI zB2Yi#fQvoN*kiy#@9hxtXKO1VV-B)sWdI1}JjRY1xc4tj7g)xM=2>H&|zsNy5tk;sDX%T;l>y;3J+l2>ZE-d)W*9 zU!kCsk3gI7$tGXirJMcZFNJd_D^y^dF+DND>|Fivm4i?IQx13XGFpDaaRndD0|_S-5*3gPf;IsDBlTfb$d8CuPYE$t5Ch&SIx+rn62L`VqxQkQ z;~-cx{%y`LO0+2V#S4tTsvP=TeFO88kQEny{`cUVe;WTUp9I&-jkEbA$3e`8D|^#z z>(;&hyfPWpvtIRix1GpV=@|N5m2272elK?Cj7+v2kSGXdmG)IMN2Dzsbd~k(x*04R!7)#$&|^E+rH+z8>+bto z1oQpM#i1>Qa^vEXPhYm_+eeF#l>hyT`B%^T^M4=b$H(#i#j|kmDzK$c>=i_Q{^=<> z^5j~+{CENWf2tH{*MD$}ro4*4B#qyL*kj{2-$1J~FzGJ*@#oxc>@v#T2Np*fXHtqH z_w9PR$zw{qK!BL>Ww)TuxD( zc8NN7bqvAX@3;8nW()uPbnNz19lqj!^9F~jx36!kz&b~cjNK5P~4Inf=2^EAF-X5s}zOs~&8S<&XF zp?W1sDUjyK^_8DFryTUd7BX?(fx`en>B_;r)C#F0cz9TF3ScC+UrBoFRw3n}ueTlX z{u6;&k)Bn;cS|=pRDPAzU5Y4p_T<}j{ylMh62A8hw55M@XS=mrBau5eDQhuYSA6qkJ1|Ak zT~&hR)_nuo`sKtkUKp6wNJy2MyevHOR)eu|{r4qHzt4AKZ0(P0wsuA@*g^j!cR?xG zRllo>Q6tpWd{}}Og6I^tTg`2M@y+TTFXfBnhPPIHZ5OZYbqwiO-4>oY)5bkW2Bd~R_N-q#&CcjnmvO)GDz__gg+C!)Sf#!4>4 z8iX8f%lEqtf+UaRO&H&t(z?D?nheSg2?P1(-#|SR2iXbgrM)T5XY1C9I=>J+xZKOh8p--Zoo8Kn$291X;wQ(WId4Jw@{O%h#vF&|^W|`zQCWAXz zMC1G1HSG|wJ4!rp*>mHJdyj}P6giBqnzQB<>_iA04GI3#%dexHlCjtA{m+(Vf zPmJ_U$W5TL3p(d)W3DgRnIKa<@N7;02D1k11Usgd2)7JKINJ0?tlRHkK?W%VWxB^L z(VUaJzilJ^=81T;1`m_m1fl#7B1BC-j8#$?DyxwcvDe}1l& zG8;N}(?zXL{D-^ta=T={p!IB!uIcEHJvptb@Zy$!y6@UxF+an=j8`uVGEU3wdRf$) zd@@n!C{4XRxXU8Pe?YoupT5JYjM9Dj?ZJUft3>ImOUh&Pd+vMrZHm}2GTi7k9+u}! zW1BaBHyoZ)5gEeSS6rH~<9gbNr`m&^Ls~y&H5!>N@4Jc-98h=I6~{5MsJo?ay1Dv0 zw{>P52mQDYIsY8#Yr|i!J!Gaxucd4+=WQ1bpbr)zf?tg;wZ?DVA|_tNv^JreA&7y6 zNO0DHqvhz{qY-g65Z;chjzsBu4hy5TouTG~^}9V0rCxNCqhT+&GcHMTT+%a?MDwJn zT`P73pJlr<|4j&-YxB9tTnfn!^cqG+zd?IB2-ee6mT<&IaGrer{1Y6py(BfIal?>{ zzW}X$9Tdd@_CNj92j3gJYAx+&@w%z4zFt!%r1l{P7V4TEhZRLo$x|zm=Z;sPab|C1 za>ZP(+ji7y`K%*SG1AfzmJ3tcz~L)+hNrfcToPp0v~fwZi4)Lk7N%uaU>D4- z<^=9GTu-QKZE(il?Oqme^&!(Wqlxg_jnMLSUvGSB>x-lO*NO44U5RMMQ*b@4c4#Pt zbv(EHuAAR1!sTX{9sIzHh;9<5o6NQ(6eRElmZ7J|B+W3qyeE2Ry%tPMKEz<)zc_jW z(2yVq$5U9ZKDnBdAbBJ>$bZcy8Io|X+iPQ>Wnv{M-c!fj(UD}ihLr_c>j@(7wnP5- z3<8XGxHogsqD+@9eboEU@=4>mb^e_+(dzoWS3e9=tusRG{en4~mfFbnbhmK42Co@= z6H01&i|6FkTCw_ylFpH>S*Ng{io-P(t&dNWRugvIX-*Ng;0;aLTFq2ZJ)b}8BaTXn zZ|&LZ*Y<~uL8%S(Sy^3HfKvsuOD}s zp_zJL{c|5oCiK!0h^N}cME>LP5rJhnKmX>(()O=E4ClaD!65P&8pQl&#dh%N z+8{NCmRh%!ULfEUd3o5Ex?giF_Q5i1?U7W9BYVEbrnMa^i5uuTekb^<5!K!0>O{h)|!KSYQqHi{-tHKcplV0l)!L2 zyik^XvE0SY?HDzfl0w@2P&$6gyAZy^1PcET{j<)6yEHJ>&O}cz6W3+JP)}YSi|y)r zvo+mpRoRRs7%q_Sa^<);FAuIR4`Rgu5zb$3B6GrypMU;4ahD@0-Zh5VvnzKL4!4#v z<5yV5b_HzFMpc6;f0I5xK_}L=9h1DQV=LX{8e_Z6#}Dq0K<{HP*}-M!hY4@(#hn%; zkajdUI&Tx`qt?u0MyWHo443$p*gTF?LmV5ll(sD{2j82fQNG@RrRi-pYG?cM)Tb@O zOcm_~Y013g+r#HFon}toZmyf5=CO~rXsCcw4Q#Rxg!>$p9sqMwWUHCM#7taH|4nC3 zr493&j|G-f%kd9H%71qML}nNy^zBbDe;5u8EV$^ z`i&UljLSa7Rb!L!EbDw;Son$ghi7zVm`B~azSHc~a2nknE7Z}PF;tx`apC{qGHVz( zF|#l{utz!3+k4?vvv93b=ZLXVL{Vv{hOyQ}h<;ZiwFxV#ZSodK*MeZ|MX>K-4^*t-SDMgYfD7*{p!+)5E>(s%`tnd-&E$<(Yly?&V6i zaDsy$wVjsx8GvI7gyhxR^rc$Gz_6hFN}t% zVv82YXt-HX6D{HYPYPBMZZX1XpLOs_n<+2&l>U*pZ<&2(j-fXs%^@7Kw+|gEUfiCJ zdbNImUFGxh7zdQ5uZ3AlUc z-$Xq$XlY0;ij?+rZzhb7kKD13M2C>`H4|arn~^(1p$d_qgSh`&OU@5UU*3JuRxs0e zvy&MwqpAuK^6s)1_pRKy0tbNM$k3qR3&2CvC)|yJ(1Zjc-^`Ad1g_iq#FjPhR<4SX z+q>TXNahdVH4CzBV!ggm#mTQ+i5`k)<#32gk5jeYNSJ4~#f{fH2yeoyC z7Pt8usq`;mK%XiJ;#I^+3WwP%EDbv1Ju9rEb_F{Qt9#ej%%&{#GOcNldSj}d9t`cj z|B?|l_ej>i664$aKd0}NJSS)sI;|Wdx)In4r`LTmd$KOuvZ-RSWF5g)E@M<@9v7VZf*x!>b@syWw-a*N@iGl+!zkBlW!Y2XDU5R!NQ+IU1*O11?6q#fe>%Jcgc&yzOW^ zU}r1ce{<}`XkP;}ncKH8+#K!HCqUjvc@>g=!4em#s{v-=xCCs|?|5)#w?pGyh9_Ev z>%RNXwGMA)Mm*6KTZ6Wx_1J-~&twFREu8)>uACyC^$xkt2PsXJtR87^PC#-0!lHG& zPSp;(LVwJEJ~h# z^b`Rk#XnM~OFnR;(`dV(`KyLV6Mbg6N4`a5{w2D|owM;6oWqs&rS9I_StU39dp~BC z(n%A2A3?=Vepy9Pz<5~Y&kr<9UL4FB2$a!W@VPk`5g>jgU(e&nBawLgB3NtJ2W!6O zVm4zYxkORF{2SQ4wwPd!;DjAAjr|%d|W~ zPZzn~^$+>y#sI6218CZ6VYg~kOp~nuh>LW1Fl2olo>3k4GfvYS+`6`_YiMLpg(MqK z$~aZGvR?+RZ$3~AHF0t{_eWj>na6`#NR*cs!2oRxU81aeGDq+1HU=MINO-#)Pp}P- zx^3O%msdAPIiou&&jG`m^-L(%OGQ(B5}JIbY8DB=yK^hrz7*xn#8ds|nBd$Fj6Gr5 zRSlHgPICazh#Eatj@~kYDbA_E3)$KEmtX%Rmx0SzrxJ;Nlf5;f;AcSBi>YYRifIc= z{U)v`KcNFLp@Z2YqsRtfv#dCe(a!KJNCz2{iIPo$z_(%H5tR&W(Znvpur3u zpY1 zWo#LK#TG4G1EX49)v9e2CekgIgLWI(+JZe%ly5_GRh!((%jbM!Job*6KB1{+=m2bQ02sy0M?I|hgpB=BWU*jqJ7r4C((Yc@RplZgCID!TA}MtD7=#g?A(`%g3mF~d zPDOA9rhTf3>o~R(U5*70l!XK-j6~v+2Y?jJk}1xJf3d)>yq;md%PjJNdx=@N&5*%a z?^{MK6?p;SCo~IAOa}?f8UWpOQyh<*IS-g?L|N<$QQr5yWuifEena2wz-kqP<|a*p z$=vC<{+)_@n}sL)(k)6?rI**bmWr>8zgltT1{w(lyp6aKMX`&grjSIm9OS3CM(f;s zFgtdC_Gj54^VVR_$WU8C-=Fq`bBlheZt!#Tc12Pt zeX4qOxDZ9wI@GOYDBpo{KyeC9u3wHE zKt**3t&=yXVo6+vB;kfMff4ljrO?Du8$4K|eKjV?*Ab@*N5w!|Fj!1P{7BT>82e5g zn)JKJ*FR_mw}XCe&m^C=BchdK&1GR(%ki7CIS&}}@<^3Rh!$N!2YnbJHU{Px2~GG- ztsfAXvVRpeaUPqIe=yxZg)k7!yAf{sg5Ott7i@M=JzxoKy{w=~N4cBR!P9~2u^hw5 zTz39Rmx7StOF}^hlO-_bGQwhp2uy)gI*h!xiJ_FeA#EFjQ)4b>Ft==TX`TlD1Ulu$3+wT4#d<%-aZ#ih18 zRUJ!D9+LGq$rdC(J21{0m62Vu2g7P!gcYW-0}6D)>MJvHK0kbDep%PRRy|gvjUn}z z6MI=ois4irLsf?7Xg7|;nJx4Qx0w(cl@9-P{qb{`MaaLTk#U9Z2_8H83eR3Iu+QD3 z(vi$oT(#ZLAHS8B9M^PRzivhq)4VY*(VqYNok@S#ne1bh`=DOw6?*;AIqVqm0(!7G zUxTCE$=c$M6Zzx0{k)})nH{Cx-$X9$2g@ztZ3y+z#YBaV?vT}G$|^Zy{yrIv38kSl z^bAC=3?+&mZQi?`5bs?zXS1hZ%?g9w%(!{?pA^^ij%SJDo?I6+XHIY>z$3}MByGjg zFKW;IGIfidAnDf2zXc9Qf3{3%T~xDhdu)oobtymhDG>=SZGNhE9k*I(L>;n75yvzs z2pR|L!~X|)Shhrad$&kwo1L+>1M(>4Dc&OFQF0e4NJEMlS8OwJz&d-K(B_Uo$3mmG z=tK8tousteA|igoEYRB72)}7elQxs5`9htzHcz`qvXW6axws7`^HH%*r_N1Q7*KaUK}ZqDb4# z5+ejth>akyZV>Qz$-Vy3=1=J@sN|4W1rNeReo>2V`HvH;FR$y>fF+d%2EPFK7vqSL zB!!SP*Wdterne=*gC{&oHk7SonWi~_Iw-vG?8i&U!-sIbZUE-<(L9d+E0PewiLtH_ zO{aXi&MqNedX6L~@6s}!x;Bxhd?FRP!AUsc)nPch7ZG9w)Rn#h>fT%&;W{ zqS1RUtsAFl2(z9)eNw2C%;gY`@+q`Zk-uNRV?&kSp%Gmxje<>u zohM!O4_VlzstRru(b&_=34%xYTvj56m*`BwOFGK_PYn5Dh$`LZX9Z6h#66pGO;O4MDB49;WzAvOeYFO6fLq->BhA*M0uTY*W|g&c4yA#9%1wP%+y6W3G;3` zVx@8ykJvPkkk|5Hl?vCYN}s5XYq}w-eayIQ(aULEZcMajpZ2yb z;><8rT^|_*SGa9c)Y98?+aM>tdPMd;?THXB@Pm1jPN|W1l8twbi$&4y0^h-hok3Pn z6{bhH6Fj4^X4i(|etB_{W0HFtZ=F!~tagCVN&MB#;Tk{(mqmlfSJ}5tb~S;`lx{hA ze`Pjhhi0ll--@Z92lJa;(YW|OCXx%rvj=8VY$j&jF%aGrcPyQMD`nhVsbW^Xy|g=a zaqul;-G5$*QbHm0%DMy$6 z0v0AWlaU=%bsT|uYS7FR8BMR@4-0q0SnKYePk;H*R;}U-;rB<*amLn1#i^Vy+jSVj zJgrs1BYp;G8%ahHsLFPq4v#TQkNL3HisyDh#hT}KZM3vriK8~?8i)_OS!%W9(FtQs z`bX)_-4N08sxWbWS!LVh*UyB07+AL7;7|<&b(;x{FO=nsfK_5~d7K3*!r*;BmGWOa zW!<8;8id5W+W)7rTLD~3MZ4@nbl?QIg)TN=jC<`S-Hp)-oXRstx6VgBpfelsnYt3K zCNQkQtoUqGf^>`STenGw6@9Z4T_$o2H8D16HSpTo&```TEOLZ+OkMmaPN_DFXCuhz z>Nbs%5zJM)3~7w1P_h8Ul{sCR+mJ*Fr%jTz)Jb4K6HdHZoE!lF@JqaPtSE#Vxm9%} zML{Fmcc9&@!B4QAy}>-=*bS7T9t!qc8GdoqX*NI#jENlz#}z*<=CZ1C=KXy(q$$@$ z#+7-ATH^!e@3=U0v`u?=2oP=WvkL6;>UnNG4u* zPLGsH9^sH>Lu1=R9r zqwlu1$JEk(bkEBixNzziemXmtiqS^sU$j?u70PSXGmS;%c5l}L-1~O=_2qzk7hRnn z;7pE}GV_4oLmEoK^;h~aHSE>LvTR50?%AMdVC}@8X?BR&s&~>wb7%HkV_%LzbpdO4 zHml0zyz1Sws5`ivn_FC;C~))l)kl82bv`1J$bsbR-YuS93$KiFPTc7#J8$Evo0}2E zx)s@wx|)Sj-?4aK?`SWdTGSToW>X@o zTN6ULS5N=bzmAT#+3^ z<8X^{!Ld=s)X`a8n$5E_FP7~>OW!zDyCB_ew!r|k9TY^O}`0_DVH7T#JTu+DfAqdHEH3hMY>5(_7xnF@eD+*P9#`_3P{W9b zh-;1(IULD3j*Wn5;(R)>ludUT3OX>P8$)VfNT!417~~Kz*J@yHH1hTr8ne3}t#)c; zq(g58OClWzDgB(G9PseC1Ad}$Ukg<)e4rW5SXq9XiQm)ekf@oRUv^)#8=r?refQFl zIOpah&Ai1iEJmM4pK_vtYvq}ilK!Ex=V8ibNF6wgEVFQ)7r!xjojqy+^!8Mj$p|&16p3ma;GJ4N zO^aK>H0RbO=3IcjmxqyJ1X4YBfax(5=#0sBsSS&gM0TFJVw)X@1A8YZ03};cPxSEB59?tOJ;2A{9-=W9jkT1`=7sa0bAiHYNGC*sabBlRabLemus35EI~R z66ZC1_xi?U1)fDgVOol7RE$zok)Y;SHt({%d-%z^1zBHT@n+2n75iBC zOnu>YP4o>0bz25HzM^<|5=pJRQx& z;MNJYh03Ze3fokarAIebQoZa>TUG1XU)O)g&+6Fkl^d^IYgOjs3}_h97P7<-Nym2H z54^b(ZbT=Dgach>ER8rmA?)75>G1O&{)vYuli$n96-z3b>}qB?pOtPqa%r0)bAY3= z?^pifML<#GYtyF1j3S6CrYeVJyC%be1nFr+VAyFyZHO1gY<*98;8*U_%+!mdZ=pd?K)sRkzy7dxLdm!k=CiM=e{J; zu_@B^r|^~)_>y|2m82S?GaJwnSg#v(a$qQHL}ymsH7fs-$)DMaLQafUi%4@JcU*@U zjId7gO7WNGN~zw;l31s}51b+DC!vEAb*HCYwKrdB#AeMOT{g2s|DEkS*opaO6iW(U>K&vwI(7&dIu#Om=)A31g##T6IAa z6Ihdqo3$p=!A_)0DI89pzAP@!(^#PrUR{)2Dj8GZ+K;AD%|586r-R_Qe<7UI+*`r9 zIKO`REc@d0fHT+_V=BS7qaI^!q^JW3Ktai4H~%o)U5`VrRgB_IP743pCcC>RY}`iX z`tVEJ62U;7O%Ew0B%CKE>-b@YWN;fmnS@jYn-L4e*xwN>t1P>AwSPcm3pEI!)Q}=~ z04_+bkQyn2bdsYm!@v4OGrfJmmPgU&c%nw$PnQ=Fy7FGSh1Ixm+ID!ix>}CQIi;Y` zYNW`f6e<|O;y8)D#`{Xn(_5dcRJ@&rgPY&mzVP!GY?-YQUu41y{h5sWNMyW%TnZe4 zM0H814aLcD=cO$Pv)|1F+4 zL+4nges!~1%9_v=ai6jL9v26V zY~CPiOIlo^j)5&zk%GZEm{8mR>+aHveDGvtPm03f_&3(kHZvfjGH~Z*67-x0ba6-C zIu^dK`P}^4PN*Sn=*oz{zgsGa-yA4I|y<6GH-OTztm zRrY@Urha=eAZ%y@;7K#Fv}VQ543qJW_(`cmELv=Z?SEou|q|`N=RlLdk zOx4%vvQB*-t@CH%U!^6?8=nQe`84o}W`2+Ko17yCu8(Y76ZiV3Kd#7}l(}m*`sVvd z8LOrh)3a1DwF?nGE_E6WjTNpf9guU$79B`*RVnCL&9U9b3bQ^M*{I&!S(a&m#4)l+ z|Lv)GPu>ysSvx{=#{XA;H_Y6@1f%K%pYj zF3_ho@H<7petUL@zefwST;htIY%l-xb2`4YziO|T(>mcX_0>BEEmc40^eX1;4J^~| zd&hGQK_>q4URTfiVG`A>f$Cvajm?R1qwsad8)LaC<5Cx$vXvzQ+I6nSypjiaopi!fPm!Jg>kBqWOSyD;d5zLfv;n}GM1U>=&T zNSTJY&nsTASIiWMNv?PRlOKobl?#w1*1D=fuS*qgE6|@3<_j@Wn-UX=LiJ7uP5B|= z3ei}ez-)C;s%#@c8yOKd6*#-Og;N$x)!CqH?@Y5#u{)QStA1_6eVv7Q&VK1J4!DNS zi{FyDb_L$=6=+e*icU&0i6IABeE>Qcd3gt9e2fdf{6JNHl*KjVLd(q(kp4$U0ws0>uX-BmEHD2(cF;70nE%1~?bx;C-Zvqs7jAev8gSCsT323J?ao@g}cvczl$`qI7iPet{m zRCnB+^f+lXph(nJ4q64r9|a@Hx*xuMFYVsF)7P%LlcZ*4m6S;6`IE-jo1PE5HbYN3 zyDFaOI)7~%SL^`neF`38QPMC8rw~ac2HxwMXZ&il8dUSoO*UQh=;{Z2A^>siUWRet zE0RzPrz=6-c%qb0%izndms-8}Vp5X1tYF24aIIs5;o)tfOp~%wl>5IH1p7=6=zO9% z{C8SR+P1o&=1a;jRDiy+()->9|9Xu7QUiO_O;qJZ2~v3q7MYjO`1PDPM&vR3G7|%+(kMydkOA}j;7)%DtY99hxJkQQd3B!iuv#J*b zm%y7U5AZi(bSbfRWRNFW^t_69pC&iyx0rHe-4$F02dD&t-nEaZ_QF8+M{-U#*mik> zzA}JwG1wVuP?Yo($gRGTBLQCxWg#zcJzNZDuR`YPk;TpcCjDov;V}78JJFd_p#Pk`oPz(hM3cbkP0{;PE6jqtH|y~ zy+m^2$eYsk)Af>9Ux_ski{Jj*xO`=Sgs6pJiE6Qg>#5ssBQ!!B*9w}gh0e^dOQ^Lx z#}?{1sDJc}V8K4CYj3#+Mzw+KEm(blHdDWG5fBI)!4GsOTRNr^~ z0Dn2rKQO&T@`~SnIW_XjzgvoaTa16VSN*mUYu5aGE&jb0Sb_gT3*u79K;Msl6@Bv2 RYg(}*`o|CDe|hS={{~bb-P-^F literal 0 HcmV?d00001 diff --git a/docs/source/contributor-guide/benchmark-results/0.5.0/comet-tpcds.json b/docs/source/contributor-guide/benchmark-results/0.5.0/comet-tpcds.json new file mode 100644 index 0000000000..8d249e3c7d --- /dev/null +++ b/docs/source/contributor-guide/benchmark-results/0.5.0/comet-tpcds.json @@ -0,0 +1,744 @@ +{ + "engine": "datafusion-comet", + "benchmark": "tpcds", + "data_path": "/mnt/bigdata/tpcds/sf100/", + "query_path": "../../tpcds/", + "spark_conf": { + "spark.executor.memoryOverhead": "4914M", + "spark.eventLog.enabled": "true", + "spark.executor.extraClassPath": "/home/andy/git/apache/datafusion-comet/spark/target/comet-spark-spark3.4_2.12-0.5.0-SNAPSHOT.jar", + "spark.comet.cast.allowIncompatible": "true", + "spark.app.submitTime": "1736811668530", + "spark.comet.explainFallback.enabled": "false", + "spark.cores.max": "16", + "spark.comet.exec.shuffle.enabled": "true", + "spark.app.startTime": "1736811668800", + "spark.memory.offHeap.enabled": "true", + "spark.driver.port": "42815", + "spark.executor.instances": "2", + "spark.executor.memory": "16G", + "spark.comet.exec.shuffle.fallbackToColumnar": "true", + "spark.serializer.objectStreamReset": "100", + "spark.driver.host": "10.0.0.118", + "spark.submit.deployMode": "client", + "spark.executor.cores": "8", + "spark.driver.extraClassPath": "/home/andy/git/apache/datafusion-comet/spark/target/comet-spark-spark3.4_2.12-0.5.0-SNAPSHOT.jar", + "spark.app.id": "app-20250113164109-0060", + "spark.comet.exec.replaceSortMergeJoin": "false", + "spark.driver.extraJavaOptions": "-Djava.net.preferIPv6Addresses=false -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false", + "spark.driver.maxResultSize": "2G", + "spark.shuffle.manager": "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager", + "spark.sql.warehouse.dir": "file:/home/andy/git/personal/research/benchmarks/spark-standalone/spark-warehouse", + "spark.executor.id": "driver", + "spark.master": "spark://woody:7077", + "spark.comet.exec.shuffle.mode": "auto", + "spark.sql.extensions": "org.apache.comet.CometSparkSessionExtensions", + "spark.driver.memory": "8G", + "spark.plugins": "org.apache.spark.CometPlugin", + "spark.repl.local.jars": "file:///home/andy/git/apache/datafusion-comet/spark/target/comet-spark-spark3.4_2.12-0.5.0-SNAPSHOT.jar", + "spark.memory.offHeap.size": "24g", + "spark.rdd.compress": "True", + "spark.executor.extraJavaOptions": "-Djava.net.preferIPv6Addresses=false -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false", + "spark.comet.enabled": "true", + "spark.submit.pyFiles": "", + "spark.app.name": "comet benchmark derived from tpcds", + "spark.comet.exec.sortMergeJoinWithJoinFilter.enabled": "false", + "spark.app.initial.jar.urls": "spark://10.0.0.118:42815/jars/comet-spark-spark3.4_2.12-0.5.0-SNAPSHOT.jar", + "spark.comet.exec.shuffle.compression.codec": "lz4", + "spark.jars": "file:///home/andy/git/apache/datafusion-comet/spark/target/comet-spark-spark3.4_2.12-0.5.0-SNAPSHOT.jar", + "spark.comet.explain.native.enabled": "false" + }, + "1": [ + 4.536968469619751, + 1.3565826416015625, + 1.338052749633789, + 1.4378666877746582, + 1.3548147678375244 + ], + "2": [ + 3.3205230236053467, + 2.462737798690796, + 2.488901376724243, + 2.5463438034057617, + 2.527088165283203 + ], + "3": [ + 1.6579983234405518, + 1.47391939163208, + 1.4881572723388672, + 1.484151840209961, + 1.5221748352050781 + ], + "4": [ + 34.67596936225891, + 33.768837451934814, + 34.09213376045227, + 34.43401646614075, + 34.43135046958923 + ], + "5": [ + 6.600037097930908, + 6.366944074630737, + 6.282367467880249, + 6.359925985336304, + 6.503366231918335 + ], + "6": [ + 1.7636768817901611, + 1.6569476127624512, + 1.564842939376831, + 1.6130192279815674, + 1.562074899673462 + ], + "7": [ + 3.080796480178833, + 3.0056214332580566, + 3.1245322227478027, + 3.1279425621032715, + 3.1556527614593506 + ], + "8": [ + 1.871955394744873, + 1.6913323402404785, + 1.6697134971618652, + 1.7257518768310547, + 1.782395839691162 + ], + "9": [ + 6.713606357574463, + 6.480487108230591, + 6.67946982383728, + 6.535262823104858, + 6.520689249038696 + ], + "10": [ + 2.2367494106292725, + 2.069857120513916, + 2.1344666481018066, + 2.0363876819610596, + 2.0629072189331055 + ], + "11": [ + 13.441156148910522, + 13.661643505096436, + 13.749130487442017, + 13.663432121276855, + 13.914729118347168 + ], + "12": [ + 1.4952607154846191, + 1.327760934829712, + 1.3590035438537598, + 1.3424949645996094, + 1.3338284492492676 + ], + "13": [ + 4.1215620040893555, + 4.143518447875977, + 4.115273952484131, + 4.1204657554626465, + 4.203814268112183 + ], + "14": [ + 27.271831274032593, + 27.740113973617554, + 27.78661036491394, + 28.459989070892334, + 28.01462411880493 + ], + "15": [ + 3.972994327545166, + 4.010797739028931, + 4.036948204040527, + 4.097492933273315, + 4.126002788543701 + ], + "16": [ + 9.47628664970398, + 9.299378871917725, + 9.3838791847229, + 9.37733244895935, + 9.319334745407104 + ], + "17": [ + 3.5423672199249268, + 3.4606869220733643, + 3.5566649436950684, + 3.574188709259033, + 3.5854296684265137 + ], + "18": [ + 3.558619499206543, + 3.7194430828094482, + 3.720961332321167, + 3.7957425117492676, + 3.7437338829040527 + ], + "19": [ + 4.280632495880127, + 4.2238616943359375, + 4.188013792037964, + 4.338703155517578, + 4.348641633987427 + ], + "20": [ + 2.1416103839874268, + 2.134965419769287, + 2.132283926010132, + 2.2123031616210938, + 2.1274831295013428 + ], + "21": [ + 1.6397242546081543, + 1.6214661598205566, + 1.5790197849273682, + 1.5165207386016846, + 1.5434236526489258 + ], + "22": [ + 7.052053689956665, + 7.3520801067352295, + 7.85315465927124, + 7.75153112411499, + 7.7829179763793945 + ], + "23": [ + 43.88178491592407, + 44.70656991004944, + 44.685741901397705, + 45.05996298789978, + 44.751373291015625 + ], + "24": [ + 16.058145999908447, + 16.151873111724854, + 16.42806100845337, + 16.49088191986084, + 16.125349283218384 + ], + "25": [ + 3.971809148788452, + 4.022783279418945, + 3.9840853214263916, + 4.052400588989258, + 4.11824893951416 + ], + "26": [ + 1.5638508796691895, + 1.5986883640289307, + 1.596513271331787, + 1.7379155158996582, + 1.6375956535339355 + ], + "27": [ + 2.977374315261841, + 3.2149150371551514, + 3.2501866817474365, + 3.2618322372436523, + 3.1989095211029053 + ], + "28": [ + 6.967320203781128, + 6.830939769744873, + 6.937035799026489, + 7.009445428848267, + 7.003203392028809 + ], + "29": [ + 5.793797969818115, + 5.6882500648498535, + 5.628785848617554, + 5.72254753112793, + 5.698575019836426 + ], + "30": [ + 1.488220453262329, + 1.3394947052001953, + 1.3159022331237793, + 1.3776261806488037, + 1.3321137428283691 + ], + "31": [ + 9.204715251922607, + 8.69102168083191, + 8.614688873291016, + 8.787571668624878, + 8.771551370620728 + ], + "32": [ + 2.203498125076294, + 2.1785271167755127, + 2.0557124614715576, + 2.211980104446411, + 2.1879453659057617 + ], + "33": [ + 3.9984161853790283, + 4.1804728507995605, + 4.131994247436523, + 4.213898658752441, + 4.188544034957886 + ], + "34": [ + 1.875131368637085, + 1.9190173149108887, + 1.9300975799560547, + 1.9599437713623047, + 1.869981288909912 + ], + "35": [ + 3.308222532272339, + 3.061551332473755, + 3.043506145477295, + 3.083115577697754, + 3.160632371902466 + ], + "36": [ + 3.769775629043579, + 3.782244920730591, + 3.828948736190796, + 3.8956804275512695, + 3.8769822120666504 + ], + "37": [ + 2.8596391677856445, + 2.8715765476226807, + 2.8092164993286133, + 2.891303777694702, + 2.855314254760742 + ], + "38": [ + 3.4315402507781982, + 3.4382240772247314, + 3.4430289268493652, + 3.531895637512207, + 3.455247163772583 + ], + "39": [ + 19.4264817237854, + 20.363921403884888, + 19.874735593795776, + 19.83957862854004, + 19.403523683547974 + ], + "40": [ + 5.129716634750366, + 5.0438196659088135, + 5.126132249832153, + 5.1094677448272705, + 5.055907487869263 + ], + "41": [ + 0.14982271194458008, + 0.1292414665222168, + 0.16390562057495117, + 0.134552001953125, + 0.12277865409851074 + ], + "42": [ + 1.2786576747894287, + 1.2783203125, + 1.3130738735198975, + 1.3072741031646729, + 1.3132829666137695 + ], + "43": [ + 1.6863470077514648, + 1.6458985805511475, + 1.7347028255462646, + 1.765425682067871, + 1.7646584510803223 + ], + "44": [ + 2.0259382724761963, + 2.0535051822662354, + 1.9940221309661865, + 2.0074307918548584, + 1.9784820079803467 + ], + "45": [ + 2.4755594730377197, + 2.326653003692627, + 2.3411612510681152, + 2.347200632095337, + 2.477112054824829 + ], + "46": [ + 3.9604787826538086, + 3.8620941638946533, + 3.7819931507110596, + 3.939593553543091, + 3.838043451309204 + ], + "47": [ + 7.505825996398926, + 7.342543125152588, + 7.576608657836914, + 7.725332736968994, + 7.735831260681152 + ], + "48": [ + 2.8078317642211914, + 2.809493064880371, + 2.826028347015381, + 2.9383201599121094, + 2.9243483543395996 + ], + "49": [ + 7.083904266357422, + 7.132755279541016, + 6.944513320922852, + 6.931026935577393, + 6.920399188995361 + ], + "50": [ + 6.608541250228882, + 7.139689922332764, + 6.71970796585083, + 6.847019195556641, + 6.901025772094727 + ], + "51": [ + 7.687297344207764, + 7.605288982391357, + 7.882007360458374, + 7.688341856002808, + 7.858721017837524 + ], + "52": [ + 1.515904188156128, + 1.5196130275726318, + 1.5246155261993408, + 1.539452314376831, + 1.529005765914917 + ], + "53": [ + 1.6835079193115234, + 1.7078402042388916, + 1.7528693675994873, + 1.754622459411621, + 1.801300287246704 + ], + "54": [ + 4.626687288284302, + 4.8692405223846436, + 4.670963764190674, + 4.76296854019165, + 4.630118370056152 + ], + "55": [ + 1.5379326343536377, + 1.559366226196289, + 1.585902452468872, + 1.5425207614898682, + 1.5508875846862793 + ], + "56": [ + 3.372633457183838, + 3.4592878818511963, + 3.4758856296539307, + 3.563056707382202, + 3.5439987182617188 + ], + "57": [ + 3.630242109298706, + 3.5384538173675537, + 3.5876047611236572, + 3.6837234497070312, + 3.6626791954040527 + ], + "58": [ + 6.306454181671143, + 6.243661642074585, + 6.297546148300171, + 6.354742050170898, + 6.31113076210022 + ], + "59": [ + 3.9262125492095947, + 3.9229207038879395, + 3.985262632369995, + 4.089562892913818, + 4.107280969619751 + ], + "60": [ + 3.506413698196411, + 3.5923097133636475, + 3.5940756797790527, + 3.6000287532806396, + 3.6011013984680176 + ], + "61": [ + 7.301086664199829, + 7.541475057601929, + 7.927215576171875, + 7.743303298950195, + 7.7511231899261475 + ], + "62": [ + 1.0287232398986816, + 1.033036708831787, + 1.0503861904144287, + 1.053584337234497, + 1.036078929901123 + ], + "63": [ + 1.6723690032958984, + 1.7417714595794678, + 1.97025728225708, + 1.807016372680664, + 1.8281068801879883 + ], + "64": [ + 15.89651346206665, + 15.665365934371948, + 16.01818823814392, + 15.994118690490723, + 16.00064516067505 + ], + "65": [ + 7.785307884216309, + 7.80276894569397, + 7.865602016448975, + 7.857702016830444, + 7.766430616378784 + ], + "66": [ + 3.1051344871520996, + 3.1696081161499023, + 3.300004243850708, + 3.5985312461853027, + 3.43051815032959 + ], + "67": [ + 30.156902551651, + 30.163670778274536, + 30.0355327129364, + 30.676511764526367, + 30.507113456726074 + ], + "68": [ + 4.870973825454712, + 4.989592552185059, + 4.550822973251343, + 4.479772329330444, + 5.1386706829071045 + ], + "69": [ + 2.1566920280456543, + 2.2332279682159424, + 2.0824127197265625, + 2.211796998977661, + 2.105759620666504 + ], + "70": [ + 3.654249668121338, + 3.706080198287964, + 3.664548873901367, + 3.912018299102783, + 3.7903084754943848 + ], + "71": [ + 3.5541186332702637, + 3.5453219413757324, + 3.483576536178589, + 3.603425979614258, + 3.5967133045196533 + ], + "72": [ + 85.7537453174591, + 84.34932899475098, + 85.4409408569336, + 82.9489848613739, + 86.19663500785828 + ], + "73": [ + 1.5869219303131104, + 1.5673043727874756, + 1.5760784149169922, + 1.6549866199493408, + 1.6651747226715088 + ], + "74": [ + 9.591845989227295, + 9.609638690948486, + 9.660242319107056, + 9.578797101974487, + 9.830672264099121 + ], + "75": [ + 12.06259274482727, + 12.131521224975586, + 12.339975833892822, + 12.526771783828735, + 12.704570055007935 + ], + "76": [ + 3.1800994873046875, + 3.1969289779663086, + 3.2074594497680664, + 3.249382495880127, + 3.2397830486297607 + ], + "77": [ + 3.9328701496124268, + 3.8905186653137207, + 4.043585777282715, + 4.000932216644287, + 4.100231409072876 + ], + "78": [ + 19.32929301261902, + 18.75995111465454, + 19.079716444015503, + 18.925426721572876, + 19.023631811141968 + ], + "79": [ + 2.897693157196045, + 3.135042667388916, + 3.0352957248687744, + 3.0536019802093506, + 3.0584769248962402 + ], + "80": [ + 15.195313692092896, + 15.409253120422363, + 15.783752918243408, + 15.655308961868286, + 15.55367660522461 + ], + "81": [ + 1.6697850227355957, + 1.6849262714385986, + 1.7256522178649902, + 1.6104223728179932, + 1.5729634761810303 + ], + "82": [ + 4.702307462692261, + 4.762118101119995, + 4.631009101867676, + 4.632325172424316, + 4.699774742126465 + ], + "83": [ + 0.9655017852783203, + 0.964923620223999, + 1.0950005054473877, + 0.9366519451141357, + 0.9813294410705566 + ], + "84": [ + 0.8160817623138428, + 0.7901313304901123, + 0.9309515953063965, + 0.7813336849212646, + 0.8049192428588867 + ], + "85": [ + 2.1436941623687744, + 2.0501821041107178, + 2.3379878997802734, + 2.2173709869384766, + 2.0497560501098633 + ], + "86": [ + 0.8833699226379395, + 0.8855693340301514, + 1.003610610961914, + 0.9266695976257324, + 0.8708605766296387 + ], + "87": [ + 3.5058393478393555, + 3.5163745880126953, + 3.8271989822387695, + 3.621147632598877, + 3.5709948539733887 + ], + "88": [ + 6.482669115066528, + 6.477633714675903, + 6.942981958389282, + 6.804642677307129, + 6.639838457107544 + ], + "89": [ + 2.1237101554870605, + 2.126112461090088, + 2.1807539463043213, + 2.172977924346924, + 2.20328426361084 + ], + "90": [ + 0.5848908424377441, + 0.5502984523773193, + 0.6235551834106445, + 0.569770336151123, + 0.5719239711761475 + ], + "91": [ + 0.6266171932220459, + 0.5699453353881836, + 0.6588234901428223, + 0.6302447319030762, + 0.6418545246124268 + ], + "92": [ + 0.8711974620819092, + 0.8485770225524902, + 0.8334972858428955, + 0.8226969242095947, + 0.8113462924957275 + ], + "93": [ + 9.335367202758789, + 9.483666181564331, + 9.641432046890259, + 9.4573495388031, + 9.48092794418335 + ], + "94": [ + 5.219858884811401, + 5.30799126625061, + 5.2076966762542725, + 5.314690589904785, + 5.152010202407837 + ], + "95": [ + 20.359986066818237, + 19.688702821731567, + 20.081215381622314, + 19.575989961624146, + 21.01106309890747 + ], + "96": [ + 0.8545236587524414, + 0.8409044742584229, + 0.8572230339050293, + 0.8502027988433838, + 0.9349768161773682 + ], + "97": [ + 5.144511938095093, + 5.298552751541138, + 5.229865789413452, + 5.0392138957977295, + 5.193090915679932 + ], + "98": [ + 4.391782283782959, + 4.436994552612305, + 4.410702705383301, + 4.459229946136475, + 4.5571510791778564 + ], + "99": [ + 1.6422052383422852, + 1.5400826930999756, + 1.5657870769500732, + 1.5555531978607178, + 1.582702398300171 + ] +} \ No newline at end of file diff --git a/docs/source/contributor-guide/benchmark-results/0.5.0/spark-tpcds.json b/docs/source/contributor-guide/benchmark-results/0.5.0/spark-tpcds.json new file mode 100644 index 0000000000..f3e3b6c323 --- /dev/null +++ b/docs/source/contributor-guide/benchmark-results/0.5.0/spark-tpcds.json @@ -0,0 +1,722 @@ +{ + "engine": "datafusion-comet", + "benchmark": "tpcds", + "data_path": "/mnt/bigdata/tpcds/sf100/", + "query_path": "../../tpcds/", + "spark_conf": { + "spark.eventLog.enabled": "true", + "spark.driver.extraJavaOptions": "-Djava.net.preferIPv6Addresses=false -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false", + "spark.sql.warehouse.dir": "file:/home/andy/git/personal/research/benchmarks/spark-standalone/spark-warehouse", + "spark.cores.max": "16", + "spark.executor.id": "driver", + "spark.master": "spark://woody:7077", + "spark.app.submitTime": "1736815365737", + "spark.driver.memory": "8G", + "spark.executor.memory": "32G", + "spark.app.startTime": "1736815366015", + "spark.rdd.compress": "True", + "spark.executor.instances": "2", + "spark.app.id": "app-20250113174246-0062", + "spark.executor.extraJavaOptions": "-Djava.net.preferIPv6Addresses=false -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false", + "spark.serializer.objectStreamReset": "100", + "spark.driver.host": "10.0.0.118", + "spark.driver.port": "43797", + "spark.submit.pyFiles": "", + "spark.submit.deployMode": "client", + "spark.app.name": "spark benchmark derived from tpcds", + "spark.executor.cores": "8" + }, + "1": [ + 5.2619593143463135, + 1.960148811340332, + 2.1967520713806152, + 2.1494410037994385, + 1.9935636520385742 + ], + "2": [ + 3.4253926277160645, + 2.6879563331604004, + 2.561516046524048, + 2.5716545581817627, + 2.563164710998535 + ], + "3": [ + 1.5403835773468018, + 1.4260897636413574, + 1.2098140716552734, + 1.2134122848510742, + 1.208524227142334 + ], + "4": [ + 56.53691649436951, + 56.77414393424988, + 56.02755045890808, + 56.025816440582275, + 56.88295769691467 + ], + "5": [ + 10.489055871963501, + 9.67568039894104, + 9.388097763061523, + 9.27160382270813, + 9.536203384399414 + ], + "6": [ + 2.1579461097717285, + 1.9873638153076172, + 1.9221436977386475, + 1.9461965560913086, + 2.0963733196258545 + ], + "7": [ + 3.032843828201294, + 2.7287216186523438, + 2.9177908897399902, + 2.675243616104126, + 2.7577996253967285 + ], + "8": [ + 2.2934205532073975, + 2.0310394763946533, + 1.994276762008667, + 1.9520981311798096, + 2.4081742763519287 + ], + "9": [ + 5.7878358364105225, + 5.4902870655059814, + 5.461946725845337, + 5.400648593902588, + 5.5186686515808105 + ], + "10": [ + 2.3901453018188477, + 2.212461233139038, + 2.298922538757324, + 2.1111397743225098, + 2.4135324954986572 + ], + "11": [ + 17.00469183921814, + 17.168133020401, + 15.51388955116272, + 14.93636679649353, + 15.758571863174438 + ], + "12": [ + 1.1017069816589355, + 0.9583470821380615, + 0.9638540744781494, + 0.9336667060852051, + 1.1460154056549072 + ], + "13": [ + 4.34652304649353, + 4.188030481338501, + 3.886648416519165, + 3.856201171875, + 4.094677209854126 + ], + "14": [ + 44.88281464576721, + 44.40218901634216, + 43.64497995376587, + 43.865254640579224, + 44.34578776359558 + ], + "15": [ + 3.323901414871216, + 3.366013765335083, + 3.1583144664764404, + 3.360898971557617, + 3.554365396499634 + ], + "16": [ + 12.555872917175293, + 13.052572965621948, + 12.834516048431396, + 12.673231363296509, + 12.716522216796875 + ], + "17": [ + 4.7320332527160645, + 4.594316720962524, + 4.3970723152160645, + 4.33561897277832, + 4.495761871337891 + ], + "18": [ + 3.5807242393493652, + 3.535151481628418, + 3.6561989784240723, + 3.8135147094726562, + 3.7007369995117188 + ], + "19": [ + 2.4516489505767822, + 2.5885403156280518, + 2.571218729019165, + 2.6798818111419678, + 2.587368965148926 + ], + "20": [ + 1.1769890785217285, + 1.084374189376831, + 1.1451663970947266, + 1.0745131969451904, + 1.0696895122528076 + ], + "21": [ + 2.2724456787109375, + 1.6616766452789307, + 2.0807278156280518, + 1.9577069282531738, + 1.7267327308654785 + ], + "22": [ + 21.127835512161255, + 21.504271745681763, + 20.774054765701294, + 20.130521297454834, + 21.08124542236328 + ], + "23": [ + 104.4822952747345, + 102.66089034080505, + 101.6080904006958, + 101.47286200523376, + 102.44011735916138 + ], + "24": [ + 15.568699359893799, + 15.30924916267395, + 14.502421855926514, + 14.607846736907959, + 15.179453611373901 + ], + "25": [ + 4.512059450149536, + 4.125646352767944, + 4.210487365722656, + 4.141698598861694, + 4.136176586151123 + ], + "26": [ + 2.0296761989593506, + 1.7046644687652588, + 1.8830039501190186, + 1.9740185737609863, + 1.7992558479309082 + ], + "27": [ + 2.637375831604004, + 2.3286960124969482, + 2.4201674461364746, + 2.5083210468292236, + 2.554165840148926 + ], + "28": [ + 8.454154253005981, + 8.10464596748352, + 8.339635848999023, + 8.20319676399231, + 8.14726972579956 + ], + "29": [ + 5.196781158447266, + 4.76829719543457, + 4.92963433265686, + 4.789939880371094, + 4.786316633224487 + ], + "30": [ + 2.0309393405914307, + 1.8231163024902344, + 2.189394235610962, + 1.9110958576202393, + 1.9947915077209473 + ], + "31": [ + 5.668262720108032, + 5.5171403884887695, + 5.473530292510986, + 5.337523460388184, + 5.736505746841431 + ], + "32": [ + 1.852400779724121, + 1.6409556865692139, + 1.7917094230651855, + 1.6856303215026855, + 1.6647214889526367 + ], + "33": [ + 3.3466131687164307, + 2.9173898696899414, + 2.859670639038086, + 2.8988308906555176, + 2.7332468032836914 + ], + "34": [ + 1.7460660934448242, + 1.8845741748809814, + 1.580057144165039, + 1.7185635566711426, + 1.5153119564056396 + ], + "35": [ + 3.7872586250305176, + 3.661235809326172, + 3.492677927017212, + 3.553354501724243, + 3.6253058910369873 + ], + "36": [ + 2.4619619846343994, + 2.133469343185425, + 2.03787899017334, + 2.0881924629211426, + 2.1240320205688477 + ], + "37": [ + 4.0605480670928955, + 3.6801133155822754, + 3.9225871562957764, + 3.9400532245635986, + 3.707016944885254 + ], + "38": [ + 6.2957329750061035, + 5.905467987060547, + 5.979095220565796, + 7.394719839096069, + 5.790559530258179 + ], + "39": [ + 8.440297603607178, + 8.22698163986206, + 8.441386938095093, + 8.33523440361023, + 8.563800573348999 + ], + "40": [ + 7.396990537643433, + 7.279749631881714, + 7.595478773117065, + 7.514429807662964, + 7.445756912231445 + ], + "41": [ + 0.23249292373657227, + 0.20892643928527832, + 0.23296880722045898, + 0.2194194793701172, + 0.21021556854248047 + ], + "42": [ + 1.1865589618682861, + 1.1890931129455566, + 1.2970597743988037, + 1.2622613906860352, + 1.2915732860565186 + ], + "43": [ + 1.297257900238037, + 1.3796677589416504, + 1.23530912399292, + 1.2672350406646729, + 1.2549505233764648 + ], + "44": [ + 0.941680908203125, + 0.8873088359832764, + 0.9511497020721436, + 0.949620246887207, + 0.9585254192352295 + ], + "45": [ + 2.0431082248687744, + 1.9389359951019287, + 1.8346209526062012, + 1.8326334953308105, + 1.9307496547698975 + ], + "46": [ + 2.762038230895996, + 2.790123224258423, + 2.7187981605529785, + 2.74556303024292, + 2.5963001251220703 + ], + "47": [ + 4.917082786560059, + 4.889256000518799, + 5.025712728500366, + 4.852914333343506, + 4.931396484375 + ], + "48": [ + 8.931923627853394, + 8.777977705001831, + 8.91053819656372, + 8.892403841018677, + 8.816314935684204 + ], + "49": [ + 7.473886251449585, + 7.176199197769165, + 7.320720434188843, + 7.655316352844238, + 7.141919136047363 + ], + "50": [ + 10.274503707885742, + 10.186567068099976, + 10.226274490356445, + 10.341506004333496, + 10.175342321395874 + ], + "51": [ + 13.496302127838135, + 13.125258922576904, + 13.10242748260498, + 13.413541078567505, + 13.212175369262695 + ], + "52": [ + 1.303222894668579, + 1.4358396530151367, + 1.4652376174926758, + 1.3406119346618652, + 1.5421879291534424 + ], + "53": [ + 1.8200466632843018, + 1.7202496528625488, + 1.773848533630371, + 1.7078053951263428, + 1.649254322052002 + ], + "54": [ + 3.2271203994750977, + 3.08542799949646, + 3.0598156452178955, + 3.265996217727661, + 2.9859659671783447 + ], + "55": [ + 1.4394917488098145, + 1.4950504302978516, + 1.499817132949829, + 1.3825452327728271, + 1.3539316654205322 + ], + "56": [ + 2.7544610500335693, + 2.6781766414642334, + 2.6310033798217773, + 2.745976209640503, + 2.5964338779449463 + ], + "57": [ + 3.1029934883117676, + 3.2437119483947754, + 3.088554859161377, + 2.882737636566162, + 3.0023646354675293 + ], + "58": [ + 3.291809558868408, + 3.0720701217651367, + 3.1128735542297363, + 2.799436569213867, + 3.03535532951355 + ], + "59": [ + 3.6170177459716797, + 3.502054214477539, + 3.6569032669067383, + 3.454932689666748, + 3.7024710178375244 + ], + "60": [ + 2.3729071617126465, + 2.374262809753418, + 2.363983392715454, + 2.4041998386383057, + 2.2664554119110107 + ], + "61": [ + 3.4571456909179688, + 3.543290138244629, + 3.4700188636779785, + 3.515277147293091, + 3.5213046073913574 + ], + "62": [ + 1.141568899154663, + 1.1349658966064453, + 1.154313087463379, + 1.1754529476165771, + 1.1065878868103027 + ], + "63": [ + 1.536681890487671, + 1.5299901962280273, + 1.5405244827270508, + 1.6240346431732178, + 1.5590183734893799 + ], + "64": [ + 22.64835238456726, + 21.65506911277771, + 22.328132152557373, + 22.563674449920654, + 22.57821035385132 + ], + "65": [ + 10.918822765350342, + 10.771467447280884, + 10.830023527145386, + 10.823500871658325, + 10.827515602111816 + ], + "66": [ + 3.189873695373535, + 3.2264277935028076, + 3.0621445178985596, + 3.1594581604003906, + 3.149355173110962 + ], + "67": [ + 80.8468451499939, + 79.63782620429993, + 79.21770143508911, + 80.91981196403503, + 79.78550696372986 + ], + "68": [ + 2.941169500350952, + 2.8041577339172363, + 2.956421375274658, + 2.7988245487213135, + 2.732013463973999 + ], + "69": [ + 2.204758405685425, + 2.115067720413208, + 2.287461280822754, + 2.1623363494873047, + 2.3425257205963135 + ], + "70": [ + 2.833672523498535, + 2.9473845958709717, + 2.8611886501312256, + 2.8069379329681396, + 2.874908685684204 + ], + "71": [ + 2.9525787830352783, + 2.8629682064056396, + 2.8824713230133057, + 2.9291234016418457, + 2.9217307567596436 + ], + "72": [ + 92.81920456886292, + 90.03170561790466, + 88.50960922241211, + 88.13007974624634, + 89.58278608322144 + ], + "73": [ + 1.395726203918457, + 1.3789515495300293, + 1.3403804302215576, + 1.4275565147399902, + 1.6597483158111572 + ], + "74": [ + 20.643322706222534, + 18.648181676864624, + 20.02110981941223, + 20.94976043701172, + 18.866977214813232 + ], + "75": [ + 12.679317951202393, + 12.062625408172607, + 12.371514558792114, + 11.771496057510376, + 11.608371257781982 + ], + "76": [ + 3.0363106727600098, + 3.0028562545776367, + 3.063755750656128, + 2.8598508834838867, + 3.0945818424224854 + ], + "77": [ + 3.665741443634033, + 3.4514997005462646, + 3.597266435623169, + 3.332911968231201, + 3.495286464691162 + ], + "78": [ + 31.306552410125732, + 31.237448930740356, + 32.20013928413391, + 30.75121283531189, + 30.8237726688385 + ], + "79": [ + 2.084214925765991, + 2.194493532180786, + 2.1290676593780518, + 2.12060284614563, + 2.1929309368133545 + ], + "80": [ + 25.5107364654541, + 26.140331983566284, + 25.496628046035767, + 26.28699231147766, + 25.533350229263306 + ], + "81": [ + 2.231703996658325, + 2.351255416870117, + 2.2447509765625, + 2.3641414642333984, + 2.3690547943115234 + ], + "82": [ + 7.771524667739868, + 7.738788366317749, + 7.6906514167785645, + 7.706395387649536, + 7.73229455947876 + ], + "83": [ + 1.6065571308135986, + 1.6151466369628906, + 1.8333814144134521, + 1.5380127429962158, + 1.8119380474090576 + ], + "84": [ + 1.2147200107574463, + 1.3057491779327393, + 1.379417896270752, + 1.3735690116882324, + 1.3126640319824219 + ], + "85": [ + 4.671465635299683, + 5.442366600036621, + 4.581593990325928, + 4.7632105350494385, + 4.413646936416626 + ], + "86": [ + 1.0525786876678467, + 0.9334254264831543, + 0.8857343196868896, + 0.9687070846557617, + 0.943767786026001 + ], + "87": [ + 6.769503355026245, + 6.3070619106292725, + 6.363903284072876, + 6.062292098999023, + 6.507101058959961 + ], + "88": [ + 5.166539907455444, + 5.316728830337524, + 5.028919696807861, + 4.994849681854248, + 5.05009126663208 + ], + "89": [ + 1.8866407871246338, + 1.8803751468658447, + 1.8070712089538574, + 1.9447128772735596, + 1.8955137729644775 + ], + "90": [ + 0.7426741123199463, + 0.7263903617858887, + 0.8066353797912598, + 0.857090950012207, + 0.7823832035064697 + ], + "91": [ + 0.8765714168548584, + 1.0066447257995605, + 0.9977982044219971, + 0.9710574150085449, + 1.0417554378509521 + ], + "92": [ + 1.2541251182556152, + 1.2637217044830322, + 1.277681827545166, + 1.272780418395996, + 1.307555913925171 + ], + "93": [ + 22.15599775314331, + 21.807148694992065, + 21.700384855270386, + 21.663914918899536, + 21.595760583877563 + ], + "94": [ + 6.799180030822754, + 6.733048439025879, + 6.928405523300171, + 6.735382318496704, + 6.847370147705078 + ], + "95": [ + 27.122989416122437, + 27.059486389160156, + 26.826467514038086, + 26.69485092163086, + 27.168644428253174 + ], + "96": [ + 0.7550704479217529, + 0.7472677230834961, + 0.6998603343963623, + 0.7055039405822754, + 0.7615151405334473 + ], + "97": [ + 9.790446996688843, + 9.716257095336914, + 9.671125650405884, + 9.580909967422485, + 9.678182125091553 + ], + "98": [ + 2.175020933151245, + 2.2659518718719482, + 2.1904609203338623, + 2.0544354915618896, + 2.0489299297332764 + ], + "99": [ + 1.5075435638427734, + 1.5963904857635498, + 1.3782060146331787, + 1.4856946468353271, + 1.3421602249145508 + ] +} \ No newline at end of file diff --git a/docs/source/contributor-guide/benchmark-results/tpc-ds.md b/docs/source/contributor-guide/benchmark-results/tpc-ds.md index 3fa70eb862..a6650f7e74 100644 --- a/docs/source/contributor-guide/benchmark-results/tpc-ds.md +++ b/docs/source/contributor-guide/benchmark-results/tpc-ds.md @@ -25,21 +25,21 @@ and we encourage you to run these benchmarks in your own environments. The tracking issue for improving TPC-DS performance is [#858](https://github.com/apache/datafusion-comet/issues/858). -![](../../_static/images/benchmark-results/0.4.0/tpcds_allqueries.png) +![](../../_static/images/benchmark-results/0.5.0/tpcds_allqueries.png) Here is a breakdown showing relative performance of Spark and Comet for each query. -![](../../_static/images/benchmark-results/0.4.0/tpcds_queries_compare.png) +![](../../_static/images/benchmark-results/0.5.0/tpcds_queries_compare.png) The following chart shows how much Comet currently accelerates each query from the benchmark in relative terms. -![](../../_static/images/benchmark-results/0.4.0/tpcds_queries_speedup_rel.png) +![](../../_static/images/benchmark-results/0.5.0/tpcds_queries_speedup_rel.png) The following chart shows how much Comet currently accelerates each query from the benchmark in absolute terms. -![](../../_static/images/benchmark-results/0.4.0/tpcds_queries_speedup_abs.png) +![](../../_static/images/benchmark-results/0.5.0/tpcds_queries_speedup_abs.png) The raw results of these benchmarks in JSON format is available here: -- [Spark](0.4.0/spark-tpcds.json) -- [Comet](0.4.0/comet-tpcds.json) +- [Spark](0.5.0/spark-tpcds.json) +- [Comet](0.5.0/comet-tpcds.json) From 9c1f0ee1934e030ad73cd04971f8a3d8755dce58 Mon Sep 17 00:00:00 2001 From: Zhen Wang <643348094@qq.com> Date: Wed, 15 Jan 2025 00:57:52 +0800 Subject: [PATCH 77/83] fix: cast timestamp to decimal is unsupported (#1281) * fix: cast timestamp to decimal is unsupported * fix style * revert test name and mark as ignore * add comment --- .../main/scala/org/apache/comet/expressions/CometCast.scala | 1 - spark/src/test/scala/org/apache/comet/CometCastSuite.scala | 6 ++++-- 2 files changed, 4 insertions(+), 3 deletions(-) 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 859cb13bea..6b0b10d808 100644 --- a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala +++ b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala @@ -204,7 +204,6 @@ object CometCast { Compatible() case DataTypes.StringType => Compatible() case DataTypes.DateType => Compatible() - case _: DecimalType => Compatible() case _ => Unsupported } } diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index f8c1a8b095..27d8e2357f 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -838,8 +838,10 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { castTest(generateTimestamps(), DataTypes.DoubleType) } - test("cast TimestampType to DecimalType(10,2)") { - castTest(generateTimestamps(), DataTypes.TimestampType) + ignore("cast TimestampType to DecimalType(10,2)") { + // https://github.com/apache/datafusion-comet/issues/1280 + // Native cast invoked for unsupported cast from Timestamp(Microsecond, Some("Etc/UTC")) to Decimal128(10, 2) + castTest(generateTimestamps(), DataTypes.createDecimalType(10, 2)) } test("cast TimestampType to StringType") { From d36e8d703e64ba107f754776b336477f4d131ca3 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 14 Jan 2025 14:38:43 -0700 Subject: [PATCH 78/83] chore: Start 0.6.0 development (#1286) * start 0.6.0 development * update some docs * Revert a change * update CI --- .github/workflows/spark_sql_test.yml | 2 +- .github/workflows/spark_sql_test_ansi.yml | 2 +- common/pom.xml | 2 +- docs/source/contributor-guide/debugging.md | 2 +- docs/source/user-guide/installation.md | 4 ++-- fuzz-testing/pom.xml | 2 +- native/Cargo.lock | 6 +++--- native/Cargo.toml | 6 +++--- pom.xml | 2 +- spark-integration/pom.xml | 2 +- spark/pom.xml | 2 +- 11 files changed, 16 insertions(+), 16 deletions(-) diff --git a/.github/workflows/spark_sql_test.yml b/.github/workflows/spark_sql_test.yml index 477e3a1ab9..238fbb2715 100644 --- a/.github/workflows/spark_sql_test.yml +++ b/.github/workflows/spark_sql_test.yml @@ -71,7 +71,7 @@ jobs: with: spark-version: ${{ matrix.spark-version.full }} spark-short-version: ${{ matrix.spark-version.short }} - comet-version: '0.5.0-SNAPSHOT' # TODO: get this from pom.xml + comet-version: '0.6.0-SNAPSHOT' # TODO: get this from pom.xml - name: Run Spark tests run: | cd apache-spark diff --git a/.github/workflows/spark_sql_test_ansi.yml b/.github/workflows/spark_sql_test_ansi.yml index e1d8388fb1..14ec6366f4 100644 --- a/.github/workflows/spark_sql_test_ansi.yml +++ b/.github/workflows/spark_sql_test_ansi.yml @@ -69,7 +69,7 @@ jobs: with: spark-version: ${{ matrix.spark-version.full }} spark-short-version: ${{ matrix.spark-version.short }} - comet-version: '0.5.0-SNAPSHOT' # TODO: get this from pom.xml + comet-version: '0.6.0-SNAPSHOT' # TODO: get this from pom.xml - name: Run Spark tests run: | cd apache-spark diff --git a/common/pom.xml b/common/pom.xml index 91109edf5d..b6cd75a32d 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.datafusion comet-parent-spark${spark.version.short}_${scala.binary.version} - 0.5.0-SNAPSHOT + 0.6.0-SNAPSHOT ../pom.xml diff --git a/docs/source/contributor-guide/debugging.md b/docs/source/contributor-guide/debugging.md index 47d1f04c87..8a368cca26 100644 --- a/docs/source/contributor-guide/debugging.md +++ b/docs/source/contributor-guide/debugging.md @@ -130,7 +130,7 @@ Then build the Comet as [described](https://github.com/apache/arrow-datafusion-c Start Comet with `RUST_BACKTRACE=1` ```console -RUST_BACKTRACE=1 $SPARK_HOME/spark-shell --jars spark/target/comet-spark-spark3.4_2.12-0.5.0-SNAPSHOT.jar --conf spark.plugins=org.apache.spark.CometPlugin --conf spark.comet.enabled=true --conf spark.comet.exec.enabled=true +RUST_BACKTRACE=1 $SPARK_HOME/spark-shell --jars spark/target/comet-spark-spark3.4_2.12-0.6.0-SNAPSHOT.jar --conf spark.plugins=org.apache.spark.CometPlugin --conf spark.comet.enabled=true --conf spark.comet.exec.enabled=true ``` Get the expanded exception details diff --git a/docs/source/user-guide/installation.md b/docs/source/user-guide/installation.md index 22d482e475..390c926387 100644 --- a/docs/source/user-guide/installation.md +++ b/docs/source/user-guide/installation.md @@ -74,7 +74,7 @@ See the [Comet Kubernetes Guide](kubernetes.md) guide. Make sure `SPARK_HOME` points to the same Spark version as Comet was built for. ```console -export COMET_JAR=spark/target/comet-spark-spark3.4_2.12-0.5.0-SNAPSHOT.jar +export COMET_JAR=spark/target/comet-spark-spark3.4_2.12-0.6.0-SNAPSHOT.jar $SPARK_HOME/bin/spark-shell \ --jars $COMET_JAR \ @@ -130,7 +130,7 @@ explicitly contain Comet otherwise Spark may use a different class-loader for th components which will then fail at runtime. For example: ``` ---driver-class-path spark/target/comet-spark-spark3.4_2.12-0.5.0-SNAPSHOT.jar +--driver-class-path spark/target/comet-spark-spark3.4_2.12-0.6.0-SNAPSHOT.jar ``` Some cluster managers may require additional configuration, see diff --git a/fuzz-testing/pom.xml b/fuzz-testing/pom.xml index 2184e54eea..0b45025c67 100644 --- a/fuzz-testing/pom.xml +++ b/fuzz-testing/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.datafusion comet-parent-spark${spark.version.short}_${scala.binary.version} - 0.5.0-SNAPSHOT + 0.6.0-SNAPSHOT ../pom.xml diff --git a/native/Cargo.lock b/native/Cargo.lock index 5a98d74afc..70c8c4295f 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -878,7 +878,7 @@ dependencies = [ [[package]] name = "datafusion-comet" -version = "0.5.0" +version = "0.6.0" dependencies = [ "arrow", "arrow-array", @@ -926,7 +926,7 @@ dependencies = [ [[package]] name = "datafusion-comet-proto" -version = "0.5.0" +version = "0.6.0" dependencies = [ "prost 0.12.6", "prost-build", @@ -934,7 +934,7 @@ dependencies = [ [[package]] name = "datafusion-comet-spark-expr" -version = "0.5.0" +version = "0.6.0" dependencies = [ "arrow", "arrow-array", diff --git a/native/Cargo.toml b/native/Cargo.toml index 0b39334bd4..ad5e1a141c 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -20,7 +20,7 @@ members = ["core", "spark-expr", "proto"] resolver = "2" [workspace.package] -version = "0.5.0" +version = "0.6.0" homepage = "https://datafusion.apache.org/comet" repository = "https://github.com/apache/datafusion-comet" authors = ["Apache DataFusion "] @@ -48,8 +48,8 @@ datafusion-expr-common = { version = "44.0.0", default-features = false } datafusion-execution = { version = "44.0.0", default-features = false } datafusion-physical-plan = { version = "44.0.0", default-features = false } datafusion-physical-expr = { version = "44.0.0", default-features = false } -datafusion-comet-spark-expr = { path = "spark-expr", version = "0.5.0" } -datafusion-comet-proto = { path = "proto", version = "0.5.0" } +datafusion-comet-spark-expr = { path = "spark-expr", version = "0.6.0" } +datafusion-comet-proto = { path = "proto", version = "0.6.0" } chrono = { version = "0.4", default-features = false, features = ["clock"] } chrono-tz = { version = "0.8" } futures = "0.3.28" diff --git a/pom.xml b/pom.xml index 76e2288ccc..4559d67412 100644 --- a/pom.xml +++ b/pom.xml @@ -30,7 +30,7 @@ under the License. org.apache.datafusion comet-parent-spark${spark.version.short}_${scala.binary.version} - 0.5.0-SNAPSHOT + 0.6.0-SNAPSHOT pom Comet Project Parent POM diff --git a/spark-integration/pom.xml b/spark-integration/pom.xml index 84c09c1c97..24b1f7a002 100644 --- a/spark-integration/pom.xml +++ b/spark-integration/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.datafusion comet-parent-spark${spark.version.short}_${scala.binary.version} - 0.5.0-SNAPSHOT + 0.6.0-SNAPSHOT ../pom.xml diff --git a/spark/pom.xml b/spark/pom.xml index ad7590dbc4..f15b0b2e8f 100644 --- a/spark/pom.xml +++ b/spark/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.datafusion comet-parent-spark${spark.version.short}_${scala.binary.version} - 0.5.0-SNAPSHOT + 0.6.0-SNAPSHOT ../pom.xml From 3eced6733aaf57a17f701659138f093eebb8168e Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 14 Jan 2025 14:38:59 -0700 Subject: [PATCH 79/83] docs: Fix links and provide complete benchmarking scripts (#1284) * fix links and provide complete scripts * fix path * fix incorrect text --- .../benchmark-results/tpc-ds.md | 65 ++++++++++++++++- .../benchmark-results/tpc-h.md | 71 +++++++++++++++++-- docs/source/contributor-guide/benchmarking.md | 60 ---------------- 3 files changed, 130 insertions(+), 66 deletions(-) diff --git a/docs/source/contributor-guide/benchmark-results/tpc-ds.md b/docs/source/contributor-guide/benchmark-results/tpc-ds.md index a6650f7e74..012913189a 100644 --- a/docs/source/contributor-guide/benchmark-results/tpc-ds.md +++ b/docs/source/contributor-guide/benchmark-results/tpc-ds.md @@ -19,8 +19,8 @@ under the License. # Apache DataFusion Comet: Benchmarks Derived From TPC-DS -The following benchmarks were performed on a two node Kubernetes cluster with -data stored locally in Parquet format on NVMe storage. Performance characteristics will vary in different environments +The following benchmarks were performed on a Linux workstation with PCIe 5, AMD 7950X CPU (16 cores), 128 GB RAM, and +data stored locally in Parquet format on NVMe storage. Performance characteristics will vary in different environments and we encourage you to run these benchmarks in your own environments. The tracking issue for improving TPC-DS performance is [#858](https://github.com/apache/datafusion-comet/issues/858). @@ -43,3 +43,64 @@ The raw results of these benchmarks in JSON format is available here: - [Spark](0.5.0/spark-tpcds.json) - [Comet](0.5.0/comet-tpcds.json) + +# Scripts + +Here are the scripts that were used to generate these results. + +## Apache Spark + +```shell +#!/bin/bash +$SPARK_HOME/bin/spark-submit \ + --master $SPARK_MASTER \ + --conf spark.driver.memory=8G \ + --conf spark.executor.memory=32G \ + --conf spark.executor.instances=2 \ + --conf spark.executor.cores=8 \ + --conf spark.cores.max=16 \ + --conf spark.eventLog.enabled=true \ + tpcbench.py \ + --benchmark tpcds \ + --name spark \ + --data /mnt/bigdata/tpcds/sf100/ \ + --queries ../../tpcds/ \ + --output . \ + --iterations 5 +``` + +## Apache Spark + Comet + +```shell +#!/bin/bash +$SPARK_HOME/bin/spark-submit \ + --master $SPARK_MASTER \ + --conf spark.driver.memory=8G \ + --conf spark.executor.instances=2 \ + --conf spark.executor.memory=16G \ + --conf spark.executor.cores=8 \ + --total-executor-cores=16 \ + --conf spark.eventLog.enabled=true \ + --conf spark.driver.maxResultSize=2G \ + --conf spark.memory.offHeap.enabled=true \ + --conf spark.memory.offHeap.size=24g \ + --jars $COMET_JAR \ + --conf spark.driver.extraClassPath=$COMET_JAR \ + --conf spark.executor.extraClassPath=$COMET_JAR \ + --conf spark.plugins=org.apache.spark.CometPlugin \ + --conf spark.comet.enabled=true \ + --conf spark.comet.cast.allowIncompatible=true \ + --conf spark.comet.exec.replaceSortMergeJoin=false \ + --conf spark.comet.exec.shuffle.enabled=true \ + --conf spark.comet.exec.shuffle.mode=auto \ + --conf spark.comet.exec.shuffle.fallbackToColumnar=true \ + --conf spark.comet.exec.shuffle.compression.codec=lz4 \ + --conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager \ + tpcbench.py \ + --name comet \ + --benchmark tpcds \ + --data /mnt/bigdata/tpcds/sf100/ \ + --queries ../../tpcds/ \ + --output . \ + --iterations 5 +``` \ No newline at end of file diff --git a/docs/source/contributor-guide/benchmark-results/tpc-h.md b/docs/source/contributor-guide/benchmark-results/tpc-h.md index 336deb7a7c..d383cae852 100644 --- a/docs/source/contributor-guide/benchmark-results/tpc-h.md +++ b/docs/source/contributor-guide/benchmark-results/tpc-h.md @@ -25,21 +25,84 @@ and we encourage you to run these benchmarks in your own environments. The tracking issue for improving TPC-H performance is [#391](https://github.com/apache/datafusion-comet/issues/391). -![](../../_static/images/benchmark-results/0.5.0-SNAPSHOT-2025-01-09/tpch_allqueries.png) +![](../../_static/images/benchmark-results/0.5.0/tpch_allqueries.png) Here is a breakdown showing relative performance of Spark and Comet for each query. -![](../../_static/images/benchmark-results/0.5.0-SNAPSHOT-2025-01-09/tpch_queries_compare.png) +![](../../_static/images/benchmark-results/0.5.0/tpch_queries_compare.png) The following chart shows how much Comet currently accelerates each query from the benchmark in relative terms. -![](../../_static/images/benchmark-results/0.5.0-SNAPSHOT-2025-01-09/tpch_queries_speedup_rel.png) +![](../../_static/images/benchmark-results/0.5.0/tpch_queries_speedup_rel.png) The following chart shows how much Comet currently accelerates each query from the benchmark in absolute terms. -![](../../_static/images/benchmark-results/0.5.0-SNAPSHOT-2025-01-09/tpch_queries_speedup_abs.png) +![](../../_static/images/benchmark-results/0.5.0/tpch_queries_speedup_abs.png) The raw results of these benchmarks in JSON format is available here: - [Spark](0.5.0/spark-tpch.json) - [Comet](0.5.0/comet-tpch.json) + +# Scripts + +Here are the scripts that were used to generate these results. + +## Apache Spark + +```shell +#!/bin/bash +$SPARK_HOME/bin/spark-submit \ + --master $SPARK_MASTER \ + --conf spark.driver.memory=8G \ + --conf spark.executor.instances=1 \ + --conf spark.executor.cores=8 \ + --conf spark.cores.max=8 \ + --conf spark.executor.memory=16g \ + --conf spark.memory.offHeap.enabled=true \ + --conf spark.memory.offHeap.size=16g \ + --conf spark.eventLog.enabled=true \ + tpcbench.py \ + --name spark \ + --benchmark tpch \ + --data /mnt/bigdata/tpch/sf100/ \ + --queries ../../tpch/queries \ + --output . \ + --iterations 5 + +``` + +## Apache Spark + Comet + +```shell +#!/bin/bash +$SPARK_HOME/bin/spark-submit \ + --master $SPARK_MASTER \ + --conf spark.driver.memory=8G \ + --conf spark.executor.instances=1 \ + --conf spark.executor.cores=8 \ + --conf spark.cores.max=8 \ + --conf spark.executor.memory=16g \ + --conf spark.memory.offHeap.enabled=true \ + --conf spark.memory.offHeap.size=16g \ + --conf spark.comet.exec.replaceSortMergeJoin=true \ + --conf spark.eventLog.enabled=true \ + --jars $COMET_JAR \ + --driver-class-path $COMET_JAR \ + --conf spark.driver.extraClassPath=$COMET_JAR \ + --conf spark.executor.extraClassPath=$COMET_JAR \ + --conf spark.sql.extensions=org.apache.comet.CometSparkSessionExtensions \ + --conf spark.comet.enabled=true \ + --conf spark.comet.exec.shuffle.enabled=true \ + --conf spark.comet.exec.shuffle.mode=auto \ + --conf spark.comet.exec.shuffle.fallbackToColumnar=true \ + --conf spark.comet.exec.shuffle.compression.codec=lz4 \ + --conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager \ + tpcbench.py \ + --name comet \ + --benchmark tpch \ + --data /mnt/bigdata/tpch/sf100/ \ + --queries ../../tpch/queries \ + --output . \ + --iterations 5 +``` \ No newline at end of file diff --git a/docs/source/contributor-guide/benchmarking.md b/docs/source/contributor-guide/benchmarking.md index 173d598ac2..1193ada625 100644 --- a/docs/source/contributor-guide/benchmarking.md +++ b/docs/source/contributor-guide/benchmarking.md @@ -24,66 +24,6 @@ benchmarking documentation and scripts are available in the [DataFusion Benchmar We also have many micro benchmarks that can be run from an IDE located [here](https://github.com/apache/datafusion-comet/tree/main/spark/src/test/scala/org/apache/spark/sql/benchmark). -Here are example commands for running the benchmarks against a Spark cluster. This command will need to be -adapted based on the Spark environment and location of data files. - -These commands are intended to be run from the `runners/datafusion-comet` directory in the `datafusion-benchmarks` -repository. - -## Running Benchmarks Against Apache Spark - -```shell -$SPARK_HOME/bin/spark-submit \ - --master $SPARK_MASTER \ - --conf spark.driver.memory=8G \ - --conf spark.executor.instances=1 \ - --conf spark.executor.memory=32G \ - --conf spark.executor.cores=8 \ - --conf spark.cores.max=8 \ - tpcbench.py \ - --benchmark tpch \ - --data /mnt/bigdata/tpch/sf100/ \ - --queries ../../tpch/queries \ - --iterations 3 -``` - -## Running Benchmarks Against Apache Spark with Apache DataFusion Comet Enabled - -### TPC-H - -```shell -$SPARK_HOME/bin/spark-submit \ - --master $SPARK_MASTER \ - --conf spark.driver.memory=8G \ - --conf spark.executor.instances=1 \ - --conf spark.executor.memory=16G \ - --conf spark.executor.cores=8 \ - --conf spark.cores.max=8 \ - --conf spark.memory.offHeap.enabled=true \ - --conf spark.memory.offHeap.size=16g \ - --jars $COMET_JAR \ - --conf spark.driver.extraClassPath=$COMET_JAR \ - --conf spark.executor.extraClassPath=$COMET_JAR \ - --conf spark.plugins=org.apache.spark.CometPlugin \ - --conf spark.comet.cast.allowIncompatible=true \ - --conf spark.comet.exec.replaceSortMergeJoin=true \ - --conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager \ - --conf spark.comet.exec.shuffle.enabled=true \ - --conf spark.comet.exec.shuffle.mode=auto \ - --conf spark.comet.exec.shuffle.enableFastEncoding=true \ - --conf spark.comet.exec.shuffle.fallbackToColumnar=true \ - --conf spark.comet.exec.shuffle.compression.codec=lz4 \ - tpcbench.py \ - --benchmark tpch \ - --data /mnt/bigdata/tpch/sf100/ \ - --queries ../../tpch/queries \ - --iterations 3 -``` - -### TPC-DS - -For TPC-DS, use `spark.comet.exec.replaceSortMergeJoin=false`. - ## Current Benchmark Results - [Benchmarks derived from TPC-H](benchmark-results/tpc-h) From 82022af6d4d36a90a11239d3e78c5043afcfb201 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 15 Jan 2025 00:12:58 -0800 Subject: [PATCH 80/83] feat: Add HasRowIdMapping interface (#1288) --- .../apache/comet/vector/HasRowIdMapping.java | 39 +++++++++++++++++++ 1 file changed, 39 insertions(+) create mode 100644 common/src/main/java/org/apache/comet/vector/HasRowIdMapping.java diff --git a/common/src/main/java/org/apache/comet/vector/HasRowIdMapping.java b/common/src/main/java/org/apache/comet/vector/HasRowIdMapping.java new file mode 100644 index 0000000000..8794902b48 --- /dev/null +++ b/common/src/main/java/org/apache/comet/vector/HasRowIdMapping.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.comet.vector; + +/** + * An interface could be implemented by vectors that have row id mapping. + * + *

For example, Iceberg's DeleteFile has a row id mapping to map row id to position. This + * interface is used to set and get the row id mapping. The row id mapping is an array of integers, + * where the index is the row id and the value is the position. Here is an example: + * [0,1,2,3,4,5,6,7] -- Original status of the row id mapping array Position delete 2, 6 + * [0,1,3,4,5,7,-,-] -- After applying position deletes [Set Num records to 6] + */ +public interface HasRowIdMapping { + default void setRowIdMapping(int[] rowIdMapping) { + throw new UnsupportedOperationException("setRowIdMapping is not supported"); + } + + default int[] getRowIdMapping() { + throw new UnsupportedOperationException("getRowIdMapping is not supported"); + } +} From 80830867bbf2d8ade50201d9b96ddceea0977da2 Mon Sep 17 00:00:00 2001 From: Parth Chandra Date: Fri, 17 Jan 2025 10:00:18 -0800 Subject: [PATCH 81/83] fix style --- .../scala/org/apache/comet/CometSparkSessionExtensions.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index e5f39b9802..8b29fc7059 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -190,7 +190,7 @@ class CometSparkSessionExtensions // data source V1 case scanExec @ FileSourceScanExec( - HadoopFsRelation(_, partitionSchema, _, _, fileFormat, _), + HadoopFsRelation(_, partitionSchema, _, _, _, _), _: Seq[_], requiredSchema, _, From 5a31ba3a71278ae4e255dca4995a49d4d907100b Mon Sep 17 00:00:00 2001 From: Parth Chandra Date: Fri, 17 Jan 2025 13:54:51 -0800 Subject: [PATCH 82/83] fix --- .../scala/org/apache/comet/CometSparkSessionExtensions.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index 8b29fc7059..f566086a21 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -190,7 +190,7 @@ class CometSparkSessionExtensions // data source V1 case scanExec @ FileSourceScanExec( - HadoopFsRelation(_, partitionSchema, _, _, _, _), + HadoopFsRelation(_, partitionSchema, _, _, fileFormat, _), _: Seq[_], requiredSchema, _, @@ -199,7 +199,8 @@ class CometSparkSessionExtensions _, _, _) - if CometNativeScanExec.isSchemaSupported(requiredSchema) + if CometScanExec.isFileFormatSupported(fileFormat) + && CometNativeScanExec.isSchemaSupported(requiredSchema) && CometNativeScanExec.isSchemaSupported(partitionSchema) // TODO we only enable full native scan if COMET_EXEC_ENABLED is enabled // but this is not really what we want .. we currently insert `CometScanExec` From 8fee4ca7466e28b6b8c4d6b87ed701f3b1d26c8d Mon Sep 17 00:00:00 2001 From: Parth Chandra Date: Fri, 17 Jan 2025 16:02:14 -0800 Subject: [PATCH 83/83] fix for plan serialization --- .../org/apache/comet/CometSparkSessionExtensions.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index f566086a21..addf737069 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -1073,12 +1073,20 @@ class CometSparkSessionExtensions var firstNativeOp = true newPlan.transformDown { case op: CometNativeExec => - if (firstNativeOp) { + val newPlan = if (firstNativeOp) { firstNativeOp = false op.convertBlock() } else { op } + + // If reaching leaf node, reset `firstNativeOp` to true + // because it will start a new block in next iteration. + if (op.children.isEmpty) { + firstNativeOp = true + } + + newPlan case op => firstNativeOp = true op