diff --git a/EXPRESSIONS.md b/EXPRESSIONS.md index 23ecfe5eb..45c36844b 100644 --- a/EXPRESSIONS.md +++ b/EXPRESSIONS.md @@ -30,6 +30,7 @@ The following Spark expressions are currently available: + If + Cast + Coalesce ++ BloomFilterMightContain + Boolean functions + And + Or diff --git a/common/src/main/java/org/apache/comet/CometArrowStreamWriter.java b/common/src/main/java/org/apache/comet/CometArrowStreamWriter.java deleted file mode 100644 index a492ce887..000000000 --- a/common/src/main/java/org/apache/comet/CometArrowStreamWriter.java +++ /dev/null @@ -1,51 +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. - */ - -package org.apache.comet; - -import java.io.IOException; -import java.nio.channels.WritableByteChannel; - -import org.apache.arrow.vector.VectorSchemaRoot; -import org.apache.arrow.vector.VectorUnloader; -import org.apache.arrow.vector.compression.NoCompressionCodec; -import org.apache.arrow.vector.dictionary.DictionaryProvider; -import org.apache.arrow.vector.ipc.ArrowStreamWriter; -import org.apache.arrow.vector.ipc.message.ArrowRecordBatch; - -/** - * A custom `ArrowStreamWriter` that allows writing batches from different root to the same stream. - * Arrow `ArrowStreamWriter` cannot change the root after initialization. - */ -public class CometArrowStreamWriter extends ArrowStreamWriter { - public CometArrowStreamWriter( - VectorSchemaRoot root, DictionaryProvider provider, WritableByteChannel out) { - super(root, provider, out); - } - - public void writeMoreBatch(VectorSchemaRoot root) throws IOException { - VectorUnloader unloader = - new VectorUnloader( - root, /*includeNullCount*/ true, NoCompressionCodec.INSTANCE, /*alignBuffers*/ true); - - try (ArrowRecordBatch batch = unloader.getRecordBatch()) { - writeRecordBatch(batch); - } - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/TypeUtil.java b/common/src/main/java/org/apache/comet/parquet/TypeUtil.java index 340bf9823..b8b7ff525 100644 --- a/common/src/main/java/org/apache/comet/parquet/TypeUtil.java +++ b/common/src/main/java/org/apache/comet/parquet/TypeUtil.java @@ -196,7 +196,9 @@ && isUnsignedIntTypeMatched(logicalTypeAnnotation, 64)) { || canReadAsBinaryDecimal(descriptor, sparkType) || sparkType == DataTypes.BinaryType // for uuid, since iceberg maps uuid to StringType - || sparkType == DataTypes.StringType) { + || sparkType == DataTypes.StringType + && logicalTypeAnnotation + instanceof LogicalTypeAnnotation.UUIDLogicalTypeAnnotation) { return; } break; diff --git a/common/src/main/java/org/apache/comet/vector/CometDecodedVector.java b/common/src/main/java/org/apache/comet/vector/CometDecodedVector.java index fec266bf1..5ebe6923a 100644 --- a/common/src/main/java/org/apache/comet/vector/CometDecodedVector.java +++ b/common/src/main/java/org/apache/comet/vector/CometDecodedVector.java @@ -48,7 +48,7 @@ protected CometDecodedVector(ValueVector vector, Field valueField, boolean useDe } @Override - ValueVector getValueVector() { + public ValueVector getValueVector() { return valueVector; } diff --git a/common/src/main/java/org/apache/comet/vector/CometDelegateVector.java b/common/src/main/java/org/apache/comet/vector/CometDelegateVector.java index 165af5631..2287e2023 100644 --- a/common/src/main/java/org/apache/comet/vector/CometDelegateVector.java +++ b/common/src/main/java/org/apache/comet/vector/CometDelegateVector.java @@ -153,7 +153,7 @@ public ColumnVector getChild(int i) { } @Override - ValueVector getValueVector() { + public ValueVector getValueVector() { return delegate.getValueVector(); } @@ -163,7 +163,7 @@ public CometVector slice(int offset, int length) { } @Override - DictionaryProvider getDictionaryProvider() { + public DictionaryProvider getDictionaryProvider() { return delegate.getDictionaryProvider(); } } diff --git a/common/src/main/java/org/apache/comet/vector/CometDictionaryVector.java b/common/src/main/java/org/apache/comet/vector/CometDictionaryVector.java index 225fcfc43..2cd9c5d18 100644 --- a/common/src/main/java/org/apache/comet/vector/CometDictionaryVector.java +++ b/common/src/main/java/org/apache/comet/vector/CometDictionaryVector.java @@ -58,7 +58,7 @@ public CometDictionaryVector( } @Override - DictionaryProvider getDictionaryProvider() { + public DictionaryProvider getDictionaryProvider() { return this.provider; } 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 e94785f54..521f1047c 100644 --- a/common/src/main/java/org/apache/comet/vector/CometPlainVector.java +++ b/common/src/main/java/org/apache/comet/vector/CometPlainVector.java @@ -138,7 +138,7 @@ public byte[] getBinary(int rowId) { } @Override - CDataDictionaryProvider getDictionaryProvider() { + public CDataDictionaryProvider getDictionaryProvider() { return null; } diff --git a/common/src/main/java/org/apache/comet/vector/CometVector.java b/common/src/main/java/org/apache/comet/vector/CometVector.java index 991431215..27f40bace 100644 --- a/common/src/main/java/org/apache/comet/vector/CometVector.java +++ b/common/src/main/java/org/apache/comet/vector/CometVector.java @@ -189,11 +189,11 @@ public void close() { getValueVector().close(); } - DictionaryProvider getDictionaryProvider() { + public DictionaryProvider getDictionaryProvider() { throw new UnsupportedOperationException("Not implemented"); } - abstract ValueVector getValueVector(); + public abstract ValueVector getValueVector(); /** * Returns a zero-copying new vector that contains the values from [offset, offset + length). diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index de49fdfb0..341ec98df 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -139,12 +139,13 @@ object CometConf { .booleanConf .createWithDefault(false) - val COMET_EXEC_BROADCAST_ENABLED: ConfigEntry[Boolean] = + val COMET_EXEC_BROADCAST_FORCE_ENABLED: ConfigEntry[Boolean] = conf(s"$COMET_EXEC_CONFIG_PREFIX.broadcast.enabled") .doc( - "Whether to enable broadcasting for Comet native operators. By default, " + - "this config is false. Note that this feature is not fully supported yet " + - "and only enabled for test purpose.") + "Whether to force enabling broadcasting for Comet native operators. By default, " + + "this config is false. Comet broadcast feature will be enabled automatically by " + + "Comet extension. But for unit tests, we need this feature to force enabling it " + + "for invalid cases. So this config is only used for unit test.") .booleanConf .createWithDefault(false) @@ -336,6 +337,26 @@ object CometConf { "enabled when reading from Iceberg tables.") .booleanConf .createWithDefault(false) + + val COMET_ROW_TO_COLUMNAR_ENABLED: ConfigEntry[Boolean] = + conf("spark.comet.rowToColumnar.enabled") + .internal() + .doc(""" + |Whether to enable row to columnar conversion in Comet. When this is turned on, Comet will + |convert row-based operators in `spark.comet.rowToColumnar.supportedOperatorList` into + |columnar based before processing.""".stripMargin) + .booleanConf + .createWithDefault(false) + + val COMET_ROW_TO_COLUMNAR_SUPPORTED_OPERATOR_LIST: ConfigEntry[Seq[String]] = + conf("spark.comet.rowToColumnar.supportedOperatorList") + .doc( + "A comma-separated list of row-based operators that will be converted to columnar " + + "format when 'spark.comet.rowToColumnar.enabled' is true") + .stringConf + .toSequence + .createWithDefault(Seq("Range,InMemoryTableScan")) + } object ConfigHelpers { 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 cc726e3e8..763ccff7f 100644 --- a/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala +++ b/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala @@ -19,89 +19,23 @@ package org.apache.comet.vector -import java.io.OutputStream -import java.nio.channels.Channels - -import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.arrow.c.{ArrowArray, ArrowImporter, ArrowSchema, CDataDictionaryProvider, Data} import org.apache.arrow.memory.RootAllocator -import org.apache.arrow.vector._ +import org.apache.arrow.vector.VectorSchemaRoot import org.apache.arrow.vector.dictionary.DictionaryProvider import org.apache.spark.SparkException +import org.apache.spark.sql.comet.util.Utils import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.comet.CometArrowStreamWriter - class NativeUtil { + import Utils._ + private val allocator = new RootAllocator(Long.MaxValue) private val dictionaryProvider: CDataDictionaryProvider = new CDataDictionaryProvider private val importer = new ArrowImporter(allocator) - /** - * Serializes a list of `ColumnarBatch` into an output stream. - * - * @param batches - * the output batches, each batch is a list of Arrow vectors wrapped in `CometVector` - * @param out - * the output stream - */ - def serializeBatches(batches: Iterator[ColumnarBatch], out: OutputStream): Long = { - var writer: Option[CometArrowStreamWriter] = None - var rowCount = 0 - - batches.foreach { batch => - val (fieldVectors, batchProviderOpt) = getBatchFieldVectors(batch) - val root = new VectorSchemaRoot(fieldVectors.asJava) - val provider = batchProviderOpt.getOrElse(dictionaryProvider) - - if (writer.isEmpty) { - writer = Some(new CometArrowStreamWriter(root, provider, Channels.newChannel(out))) - writer.get.start() - writer.get.writeBatch() - } else { - writer.get.writeMoreBatch(root) - } - - root.clear() - rowCount += batch.numRows() - } - - writer.map(_.end()) - - rowCount - } - - def getBatchFieldVectors( - batch: ColumnarBatch): (Seq[FieldVector], Option[DictionaryProvider]) = { - var provider: Option[DictionaryProvider] = None - val fieldVectors = (0 until batch.numCols()).map { index => - batch.column(index) match { - case a: CometVector => - val valueVector = a.getValueVector - if (valueVector.getField.getDictionary != null) { - if (provider.isEmpty) { - provider = Some(a.getDictionaryProvider) - } else { - if (provider.get != a.getDictionaryProvider) { - throw new SparkException( - "Comet execution only takes Arrow Arrays with the same dictionary provider") - } - } - } - - getFieldVector(valueVector) - - case c => - throw new SparkException( - "Comet execution only takes Arrow Arrays, but got " + - s"${c.getClass}") - } - } - (fieldVectors, provider) - } - /** * Exports a Comet `ColumnarBatch` into a list of memory addresses that can be consumed by the * native execution. @@ -199,15 +133,19 @@ class NativeUtil { new ColumnarBatch(arrayVectors.toArray, maxNumRows) } +} + +object NativeUtil { + def rootAsBatch(arrowRoot: VectorSchemaRoot): ColumnarBatch = { + rootAsBatch(arrowRoot, null) + } - private def getFieldVector(valueVector: ValueVector): FieldVector = { - valueVector match { - case v @ (_: BitVector | _: TinyIntVector | _: SmallIntVector | _: IntVector | - _: BigIntVector | _: Float4Vector | _: Float8Vector | _: VarCharVector | - _: DecimalVector | _: DateDayVector | _: TimeStampMicroTZVector | _: VarBinaryVector | - _: FixedSizeBinaryVector | _: TimeStampMicroVector) => - v.asInstanceOf[FieldVector] - case _ => throw new SparkException(s"Unsupported Arrow Vector: ${valueVector.getClass}") + def rootAsBatch(arrowRoot: VectorSchemaRoot, provider: DictionaryProvider): ColumnarBatch = { + val vectors = (0 until arrowRoot.getFieldVectors.size()).map { i => + val vector = arrowRoot.getFieldVectors.get(i) + // Native shuffle always uses decimal128. + CometVector.getVector(vector, true, provider) } + new ColumnarBatch(vectors.toArray, arrowRoot.getRowCount) } } diff --git a/common/src/main/scala/org/apache/comet/vector/StreamReader.scala b/common/src/main/scala/org/apache/comet/vector/StreamReader.scala index 9c4f99602..61d800bfb 100644 --- a/common/src/main/scala/org/apache/comet/vector/StreamReader.scala +++ b/common/src/main/scala/org/apache/comet/vector/StreamReader.scala @@ -21,13 +21,11 @@ package org.apache.comet.vector import java.nio.channels.ReadableByteChannel -import scala.collection.JavaConverters.collectionAsScalaIterableConverter - import org.apache.arrow.memory.RootAllocator import org.apache.arrow.vector.VectorSchemaRoot import org.apache.arrow.vector.ipc.{ArrowStreamReader, ReadChannel} import org.apache.arrow.vector.ipc.message.MessageChannelReader -import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} +import org.apache.spark.sql.vectorized.ColumnarBatch /** * A reader that consumes Arrow data from an input channel, and produces Comet batches. @@ -47,13 +45,7 @@ case class StreamReader(channel: ReadableByteChannel) extends AutoCloseable { } private def rootAsBatch(root: VectorSchemaRoot): ColumnarBatch = { - val columns = root.getFieldVectors.asScala.map { vector => - // Native shuffle always uses decimal128. - CometVector.getVector(vector, true, arrowReader).asInstanceOf[ColumnVector] - }.toArray - val batch = new ColumnarBatch(columns) - batch.setNumRows(root.getRowCount) - batch + NativeUtil.rootAsBatch(root, arrowReader) } override def close(): Unit = { diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/arrow/ArrowWriters.scala b/common/src/main/scala/org/apache/spark/sql/comet/execution/arrow/ArrowWriters.scala new file mode 100644 index 000000000..8d9f373fe --- /dev/null +++ b/common/src/main/scala/org/apache/spark/sql/comet/execution/arrow/ArrowWriters.scala @@ -0,0 +1,472 @@ +/* + * 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.execution.arrow + +import scala.collection.JavaConverters._ + +import org.apache.arrow.vector._ +import org.apache.arrow.vector.complex._ +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.SpecializedGetters +import org.apache.spark.sql.comet.util.Utils +import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.types._ + +/** + * This file is mostly copied from Spark SQL's + * org.apache.spark.sql.execution.arrow.ArrowWriter.scala. Comet shadows Arrow classes to avoid + * potential conflicts with Spark's Arrow dependencies, hence we cannot reuse Spark's ArrowWriter + * directly. + */ +private[arrow] object ArrowWriter { + def create(root: VectorSchemaRoot): ArrowWriter = { + val children = root.getFieldVectors().asScala.map { vector => + vector.allocateNew() + createFieldWriter(vector) + } + new ArrowWriter(root, children.toArray) + } + + private[sql] def createFieldWriter(vector: ValueVector): ArrowFieldWriter = { + val field = vector.getField() + (Utils.fromArrowField(field), vector) match { + case (BooleanType, vector: BitVector) => new BooleanWriter(vector) + case (ByteType, vector: TinyIntVector) => new ByteWriter(vector) + case (ShortType, vector: SmallIntVector) => new ShortWriter(vector) + case (IntegerType, vector: IntVector) => new IntegerWriter(vector) + case (LongType, vector: BigIntVector) => new LongWriter(vector) + case (FloatType, vector: Float4Vector) => new FloatWriter(vector) + case (DoubleType, vector: Float8Vector) => new DoubleWriter(vector) + case (DecimalType.Fixed(precision, scale), vector: DecimalVector) => + new DecimalWriter(vector, precision, scale) + case (StringType, vector: VarCharVector) => new StringWriter(vector) + case (StringType, vector: LargeVarCharVector) => new LargeStringWriter(vector) + case (BinaryType, vector: VarBinaryVector) => new BinaryWriter(vector) + case (BinaryType, vector: LargeVarBinaryVector) => new LargeBinaryWriter(vector) + case (DateType, vector: DateDayVector) => new DateWriter(vector) + case (TimestampType, vector: TimeStampMicroTZVector) => new TimestampWriter(vector) + case (TimestampNTZType, vector: TimeStampMicroVector) => new TimestampNTZWriter(vector) + case (ArrayType(_, _), vector: ListVector) => + val elementVector = createFieldWriter(vector.getDataVector()) + new ArrayWriter(vector, elementVector) + case (MapType(_, _, _), vector: MapVector) => + val structVector = vector.getDataVector.asInstanceOf[StructVector] + val keyWriter = createFieldWriter(structVector.getChild(MapVector.KEY_NAME)) + val valueWriter = createFieldWriter(structVector.getChild(MapVector.VALUE_NAME)) + new MapWriter(vector, structVector, keyWriter, valueWriter) + case (StructType(_), vector: StructVector) => + val children = (0 until vector.size()).map { ordinal => + createFieldWriter(vector.getChildByOrdinal(ordinal)) + } + new StructWriter(vector, children.toArray) + case (NullType, vector: NullVector) => new NullWriter(vector) + case (_: YearMonthIntervalType, vector: IntervalYearVector) => + new IntervalYearWriter(vector) + case (_: DayTimeIntervalType, vector: DurationVector) => new DurationWriter(vector) +// case (CalendarIntervalType, vector: IntervalMonthDayNanoVector) => +// new IntervalMonthDayNanoWriter(vector) + case (dt, _) => + throw QueryExecutionErrors.notSupportTypeError(dt) + } + } +} + +class ArrowWriter(val root: VectorSchemaRoot, fields: Array[ArrowFieldWriter]) { + + def schema: StructType = Utils.fromArrowSchema(root.getSchema()) + + private var count: Int = 0 + + def write(row: InternalRow): Unit = { + var i = 0 + while (i < fields.length) { + fields(i).write(row, i) + i += 1 + } + count += 1 + } + + def finish(): Unit = { + root.setRowCount(count) + fields.foreach(_.finish()) + } + + def reset(): Unit = { + root.setRowCount(0) + count = 0 + fields.foreach(_.reset()) + } +} + +private[arrow] abstract class ArrowFieldWriter { + + def valueVector: ValueVector + + def name: String = valueVector.getField().getName() + def dataType: DataType = Utils.fromArrowField(valueVector.getField()) + def nullable: Boolean = valueVector.getField().isNullable() + + def setNull(): Unit + def setValue(input: SpecializedGetters, ordinal: Int): Unit + + private[arrow] var count: Int = 0 + + def write(input: SpecializedGetters, ordinal: Int): Unit = { + if (input.isNullAt(ordinal)) { + setNull() + } else { + setValue(input, ordinal) + } + count += 1 + } + + def finish(): Unit = { + valueVector.setValueCount(count) + } + + def reset(): Unit = { + valueVector.reset() + count = 0 + } +} + +private[arrow] class BooleanWriter(val valueVector: BitVector) extends ArrowFieldWriter { + + override def setNull(): Unit = { + valueVector.setNull(count) + } + + override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { + valueVector.setSafe(count, if (input.getBoolean(ordinal)) 1 else 0) + } +} + +private[arrow] class ByteWriter(val valueVector: TinyIntVector) extends ArrowFieldWriter { + + override def setNull(): Unit = { + valueVector.setNull(count) + } + + override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { + valueVector.setSafe(count, input.getByte(ordinal)) + } +} + +private[arrow] class ShortWriter(val valueVector: SmallIntVector) extends ArrowFieldWriter { + + override def setNull(): Unit = { + valueVector.setNull(count) + } + + override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { + valueVector.setSafe(count, input.getShort(ordinal)) + } +} + +private[arrow] class IntegerWriter(val valueVector: IntVector) extends ArrowFieldWriter { + + override def setNull(): Unit = { + valueVector.setNull(count) + } + + override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { + valueVector.setSafe(count, input.getInt(ordinal)) + } +} + +private[arrow] class LongWriter(val valueVector: BigIntVector) extends ArrowFieldWriter { + + override def setNull(): Unit = { + valueVector.setNull(count) + } + + override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { + valueVector.setSafe(count, input.getLong(ordinal)) + } +} + +private[arrow] class FloatWriter(val valueVector: Float4Vector) extends ArrowFieldWriter { + + override def setNull(): Unit = { + valueVector.setNull(count) + } + + override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { + valueVector.setSafe(count, input.getFloat(ordinal)) + } +} + +private[arrow] class DoubleWriter(val valueVector: Float8Vector) extends ArrowFieldWriter { + + override def setNull(): Unit = { + valueVector.setNull(count) + } + + override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { + valueVector.setSafe(count, input.getDouble(ordinal)) + } +} + +private[arrow] class DecimalWriter(val valueVector: DecimalVector, precision: Int, scale: Int) + extends ArrowFieldWriter { + + override def setNull(): Unit = { + valueVector.setNull(count) + } + + override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { + val decimal = input.getDecimal(ordinal, precision, scale) + if (decimal.changePrecision(precision, scale)) { + valueVector.setSafe(count, decimal.toJavaBigDecimal) + } else { + setNull() + } + } +} + +private[arrow] class StringWriter(val valueVector: VarCharVector) extends ArrowFieldWriter { + + override def setNull(): Unit = { + valueVector.setNull(count) + } + + override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { + val utf8 = input.getUTF8String(ordinal) + val utf8ByteBuffer = utf8.getByteBuffer + // todo: for off-heap UTF8String, how to pass in to arrow without copy? + valueVector.setSafe(count, utf8ByteBuffer, utf8ByteBuffer.position(), utf8.numBytes()) + } +} + +private[arrow] class LargeStringWriter(val valueVector: LargeVarCharVector) + extends ArrowFieldWriter { + + override def setNull(): Unit = { + valueVector.setNull(count) + } + + override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { + val utf8 = input.getUTF8String(ordinal) + val utf8ByteBuffer = utf8.getByteBuffer + // todo: for off-heap UTF8String, how to pass in to arrow without copy? + valueVector.setSafe(count, utf8ByteBuffer, utf8ByteBuffer.position(), utf8.numBytes()) + } +} + +private[arrow] class BinaryWriter(val valueVector: VarBinaryVector) extends ArrowFieldWriter { + + override def setNull(): Unit = { + valueVector.setNull(count) + } + + override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { + val bytes = input.getBinary(ordinal) + valueVector.setSafe(count, bytes, 0, bytes.length) + } +} + +private[arrow] class LargeBinaryWriter(val valueVector: LargeVarBinaryVector) + extends ArrowFieldWriter { + + override def setNull(): Unit = { + valueVector.setNull(count) + } + + override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { + val bytes = input.getBinary(ordinal) + valueVector.setSafe(count, bytes, 0, bytes.length) + } +} + +private[arrow] class DateWriter(val valueVector: DateDayVector) extends ArrowFieldWriter { + + override def setNull(): Unit = { + valueVector.setNull(count) + } + + override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { + valueVector.setSafe(count, input.getInt(ordinal)) + } +} + +private[arrow] class TimestampWriter(val valueVector: TimeStampMicroTZVector) + extends ArrowFieldWriter { + + override def setNull(): Unit = { + valueVector.setNull(count) + } + + override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { + valueVector.setSafe(count, input.getLong(ordinal)) + } +} + +private[arrow] class TimestampNTZWriter(val valueVector: TimeStampMicroVector) + extends ArrowFieldWriter { + + override def setNull(): Unit = { + valueVector.setNull(count) + } + + override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { + valueVector.setSafe(count, input.getLong(ordinal)) + } +} + +private[arrow] class ArrayWriter(val valueVector: ListVector, val elementWriter: ArrowFieldWriter) + extends ArrowFieldWriter { + + override def setNull(): Unit = {} + + override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { + val array = input.getArray(ordinal) + var i = 0 + valueVector.startNewValue(count) + while (i < array.numElements()) { + elementWriter.write(array, i) + i += 1 + } + valueVector.endValue(count, array.numElements()) + } + + override def finish(): Unit = { + super.finish() + elementWriter.finish() + } + + override def reset(): Unit = { + super.reset() + elementWriter.reset() + } +} + +private[arrow] class StructWriter( + val valueVector: StructVector, + children: Array[ArrowFieldWriter]) + extends ArrowFieldWriter { + + override def setNull(): Unit = { + var i = 0 + while (i < children.length) { + children(i).setNull() + children(i).count += 1 + i += 1 + } + valueVector.setNull(count) + } + + override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { + val struct = input.getStruct(ordinal, children.length) + var i = 0 + valueVector.setIndexDefined(count) + while (i < struct.numFields) { + children(i).write(struct, i) + i += 1 + } + } + + override def finish(): Unit = { + super.finish() + children.foreach(_.finish()) + } + + override def reset(): Unit = { + super.reset() + children.foreach(_.reset()) + } +} + +private[arrow] class MapWriter( + val valueVector: MapVector, + val structVector: StructVector, + val keyWriter: ArrowFieldWriter, + val valueWriter: ArrowFieldWriter) + extends ArrowFieldWriter { + + override def setNull(): Unit = {} + + override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { + val map = input.getMap(ordinal) + valueVector.startNewValue(count) + val keys = map.keyArray() + val values = map.valueArray() + var i = 0 + while (i < map.numElements()) { + structVector.setIndexDefined(keyWriter.count) + keyWriter.write(keys, i) + valueWriter.write(values, i) + i += 1 + } + + valueVector.endValue(count, map.numElements()) + } + + override def finish(): Unit = { + super.finish() + keyWriter.finish() + valueWriter.finish() + } + + override def reset(): Unit = { + super.reset() + keyWriter.reset() + valueWriter.reset() + } +} + +private[arrow] class NullWriter(val valueVector: NullVector) extends ArrowFieldWriter { + + override def setNull(): Unit = {} + + override def setValue(input: SpecializedGetters, ordinal: Int): Unit = {} +} + +private[arrow] class IntervalYearWriter(val valueVector: IntervalYearVector) + extends ArrowFieldWriter { + override def setNull(): Unit = { + valueVector.setNull(count) + } + + override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { + valueVector.setSafe(count, input.getInt(ordinal)); + } +} + +private[arrow] class DurationWriter(val valueVector: DurationVector) extends ArrowFieldWriter { + override def setNull(): Unit = { + valueVector.setNull(count) + } + + override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { + valueVector.setSafe(count, input.getLong(ordinal)) + } +} + +private[arrow] class IntervalMonthDayNanoWriter(val valueVector: IntervalMonthDayNanoVector) + extends ArrowFieldWriter { + override def setNull(): Unit = { + valueVector.setNull(count) + } + + override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { + val ci = input.getInterval(ordinal) + valueVector.setSafe(count, ci.months, ci.days, Math.multiplyExact(ci.microseconds, 1000L)) + } +} diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/arrow/CometArrowConverters.scala b/common/src/main/scala/org/apache/spark/sql/comet/execution/arrow/CometArrowConverters.scala new file mode 100644 index 000000000..9dbd8dcdf --- /dev/null +++ b/common/src/main/scala/org/apache/spark/sql/comet/execution/arrow/CometArrowConverters.scala @@ -0,0 +1,131 @@ +/* + * 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.execution.arrow + +import org.apache.arrow.memory.{BufferAllocator, RootAllocator} +import org.apache.arrow.vector.VectorSchemaRoot +import org.apache.spark.TaskContext +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.comet.util.Utils +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.vectorized.ColumnarBatch + +import org.apache.comet.vector.NativeUtil + +object CometArrowConverters extends Logging { + // TODO: we should reuse the same root allocator in the comet code base? + val rootAllocator: BufferAllocator = new RootAllocator(Long.MaxValue) + + // This is similar how Spark converts internal row to Arrow format except that it is transforming + // the result batch to Comet's ColumnarBatch instead of serialized bytes. + // There's another big difference that Comet may consume the ColumnarBatch by exporting it to + // the native side. Hence, we need to: + // 1. reset the Arrow writer after the ColumnarBatch is consumed + // 2. close the allocator when the task is finished but not when the iterator is all consumed + // The reason for the second point is that when ColumnarBatch is exported to the native side, the + // exported process increases the reference count of the Arrow vectors. The reference count is + // only decreased when the native plan is done with the vectors, which is usually longer than + // all the ColumnarBatches are consumed. + private[sql] class ArrowBatchIterator( + rowIter: Iterator[InternalRow], + schema: StructType, + maxRecordsPerBatch: Long, + timeZoneId: String, + context: TaskContext) + extends Iterator[ColumnarBatch] + with AutoCloseable { + + private val arrowSchema = Utils.toArrowSchema(schema, timeZoneId) + // Reuse the same root allocator here. + private val allocator = + rootAllocator.newChildAllocator(s"to${this.getClass.getSimpleName}", 0, Long.MaxValue) + private val root = VectorSchemaRoot.create(arrowSchema, allocator) + private val arrowWriter = ArrowWriter.create(root) + + private var currentBatch: ColumnarBatch = null + private var closed: Boolean = false + + Option(context).foreach { + _.addTaskCompletionListener[Unit] { _ => + close(true) + } + } + + override def hasNext: Boolean = rowIter.hasNext || { + close(false) + false + } + + override def next(): ColumnarBatch = { + currentBatch = nextBatch() + currentBatch + } + + override def close(): Unit = { + close(false) + } + + private def nextBatch(): ColumnarBatch = { + if (rowIter.hasNext) { + // the arrow writer shall be reset before writing the next batch + arrowWriter.reset() + var rowCount = 0L + while (rowIter.hasNext && (maxRecordsPerBatch <= 0 || rowCount < maxRecordsPerBatch)) { + val row = rowIter.next() + arrowWriter.write(row) + rowCount += 1 + } + arrowWriter.finish() + NativeUtil.rootAsBatch(root) + } else { + null + } + } + + private def close(closeAllocator: Boolean): Unit = { + try { + if (!closed) { + if (currentBatch != null) { + arrowWriter.reset() + currentBatch.close() + currentBatch = null + } + root.close() + closed = true + } + } finally { + // the allocator shall be closed when the task is finished + if (closeAllocator) { + allocator.close() + } + } + } + } + + def toArrowBatchIterator( + rowIter: Iterator[InternalRow], + schema: StructType, + maxRecordsPerBatch: Long, + timeZoneId: String, + context: TaskContext): Iterator[ColumnarBatch] = { + new ArrowBatchIterator(rowIter, schema, maxRecordsPerBatch, timeZoneId, context) + } +} diff --git a/common/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala b/common/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala index 9e0541f44..7d920e1be 100644 --- a/common/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala +++ b/common/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala @@ -19,14 +19,26 @@ package org.apache.spark.sql.comet.util -import java.io.File +import java.io.{DataOutputStream, File} +import java.nio.ByteBuffer +import java.nio.channels.Channels import scala.collection.JavaConverters._ +import org.apache.arrow.c.CDataDictionaryProvider +import org.apache.arrow.vector.{BigIntVector, BitVector, DateDayVector, DecimalVector, FieldVector, FixedSizeBinaryVector, Float4Vector, Float8Vector, IntVector, SmallIntVector, TimeStampMicroTZVector, TimeStampMicroVector, TinyIntVector, ValueVector, VarBinaryVector, VarCharVector, VectorSchemaRoot} import org.apache.arrow.vector.complex.MapVector +import org.apache.arrow.vector.dictionary.DictionaryProvider +import org.apache.arrow.vector.ipc.ArrowStreamWriter import org.apache.arrow.vector.types._ import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, Schema} +import org.apache.spark.{SparkEnv, SparkException} +import org.apache.spark.io.CompressionCodec import org.apache.spark.sql.types._ +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.io.{ChunkedByteBuffer, ChunkedByteBufferOutputStream} + +import org.apache.comet.vector.CometVector object Utils { def getConfPath(confFileName: String): String = { @@ -42,6 +54,11 @@ object Utils { str.split(",").map(_.trim()).filter(_.nonEmpty) } + /** bridges the function call to Spark's Util */ + def getSimpleName(cls: Class[_]): String = { + org.apache.spark.util.Utils.getSimpleName(cls) + } + def fromArrowField(field: Field): DataType = { field.getType match { case _: ArrowType.Map => @@ -78,6 +95,9 @@ object Utils { case _: ArrowType.FixedSizeBinary => BinaryType case d: ArrowType.Decimal => DecimalType(d.getPrecision, d.getScale) case date: ArrowType.Date if date.getUnit == DateUnit.DAY => DateType + case ts: ArrowType.Timestamp + if ts.getUnit == TimeUnit.MICROSECOND && ts.getTimezone == null => + TimestampNTZType case ts: ArrowType.Timestamp if ts.getUnit == TimeUnit.MICROSECOND => TimestampType case ArrowType.Null.INSTANCE => NullType case yi: ArrowType.Interval if yi.getUnit == IntervalUnit.YEAR_MONTH => @@ -86,6 +106,13 @@ object Utils { case _ => throw new UnsupportedOperationException(s"Unsupported data type: ${dt.toString}") } + def fromArrowSchema(schema: Schema): StructType = { + StructType(schema.getFields.asScala.map { field => + val dt = fromArrowField(field) + StructField(field.getName, dt, field.isNullable) + }.toArray) + } + /** Maps data type from Spark to Arrow. NOTE: timeZoneId required for TimestampTypes */ def toArrowType(dt: DataType, timeZoneId: String): ArrowType = dt match { @@ -161,4 +188,79 @@ object Utils { toArrowField(field.name, field.dataType, field.nullable, timeZoneId) }.asJava) } + + /** + * Serializes a list of `ColumnarBatch` into an output stream. This method must be in `spark` + * package because `ChunkedByteBufferOutputStream` is spark private class. As it uses Arrow + * classes, it must be in `common` module. + * + * @param batches + * the output batches, each batch is a list of Arrow vectors wrapped in `CometVector` + * @param out + * the output stream + */ + def serializeBatches(batches: Iterator[ColumnarBatch]): Iterator[(Long, ChunkedByteBuffer)] = { + batches.map { batch => + val dictionaryProvider: CDataDictionaryProvider = new CDataDictionaryProvider + + val codec = CompressionCodec.createCodec(SparkEnv.get.conf) + val cbbos = new ChunkedByteBufferOutputStream(1024 * 1024, ByteBuffer.allocate) + val out = new DataOutputStream(codec.compressedOutputStream(cbbos)) + + val (fieldVectors, batchProviderOpt) = getBatchFieldVectors(batch) + val root = new VectorSchemaRoot(fieldVectors.asJava) + val provider = batchProviderOpt.getOrElse(dictionaryProvider) + + val writer = new ArrowStreamWriter(root, provider, Channels.newChannel(out)) + writer.start() + writer.writeBatch() + + root.clear() + writer.end() + + out.flush() + out.close() + + if (out.size() > 0) { + (batch.numRows(), cbbos.toChunkedByteBuffer) + } else { + (batch.numRows(), new ChunkedByteBuffer(Array.empty[ByteBuffer])) + } + } + } + + def getBatchFieldVectors( + batch: ColumnarBatch): (Seq[FieldVector], Option[DictionaryProvider]) = { + var provider: Option[DictionaryProvider] = None + val fieldVectors = (0 until batch.numCols()).map { index => + batch.column(index) match { + case a: CometVector => + val valueVector = a.getValueVector + if (valueVector.getField.getDictionary != null) { + if (provider.isEmpty) { + provider = Some(a.getDictionaryProvider) + } + } + + getFieldVector(valueVector) + + case c => + throw new SparkException( + "Comet execution only takes Arrow Arrays, but got " + + s"${c.getClass}") + } + } + (fieldVectors, provider) + } + + def getFieldVector(valueVector: ValueVector): FieldVector = { + valueVector match { + case v @ (_: BitVector | _: TinyIntVector | _: SmallIntVector | _: IntVector | + _: BigIntVector | _: Float4Vector | _: Float8Vector | _: VarCharVector | + _: DecimalVector | _: DateDayVector | _: TimeStampMicroTZVector | _: VarBinaryVector | + _: FixedSizeBinaryVector | _: TimeStampMicroVector) => + v.asInstanceOf[FieldVector] + case _ => throw new SparkException(s"Unsupported Arrow Vector: ${valueVector.getClass}") + } + } } diff --git a/core/Cargo.lock b/core/Cargo.lock index 456d96966..e209e4a8d 100644 --- a/core/Cargo.lock +++ b/core/Cargo.lock @@ -19,9 +19,9 @@ checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" [[package]] name = "ahash" -version = "0.8.7" +version = "0.8.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77c3a9648d43b9cd48db467b3f87fdd6e146bcc88ab0180006cef2179fe11d01" +checksum = "e89da841a80418a9b391ebaea17f5c112ffaaa96f621d2c285b5174da76b9011" dependencies = [ "cfg-if", "const-random", @@ -33,9 +33,9 @@ dependencies = [ [[package]] name = "aho-corasick" -version = "1.1.2" +version = "1.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b2969dcb958b36655471fc61f7e416fa76033bdd4bfed0678d8fee1e2d07a1f0" +checksum = "8e60d3430d3a69478ad0993f19238d2df97c507009a52b3c10addcd7f6bcb916" dependencies = [ "memchr", ] @@ -84,21 +84,21 @@ checksum = "4b46cbb362ab8752921c97e041f5e366ee6297bd428a31275b9fcf1e380f7299" [[package]] name = "anstyle" -version = "1.0.4" +version = "1.0.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7079075b41f533b8c61d2a4d073c4676e1f8b249ff94a393b0595db304e0dd87" +checksum = "8901269c6307e8d93993578286ac0edf7f195079ffff5ebdeea6a59ffb7e36bc" [[package]] name = "anyhow" -version = "1.0.79" +version = "1.0.81" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "080e9890a082662b09c1ad45f567faeeb47f22b5fb23895fbe1e651e718e25ca" +checksum = "0952808a6c2afd1aa8947271f3a60f1a6763c7b912d210184c5149b5cf147247" [[package]] name = "arc-swap" -version = "1.6.0" +version = "1.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bddcadddf5e9015d310179a59bb28c4d4b9920ad0f11e8e14dbadf654890c9a6" +checksum = "69f7f8c3906b62b754cd5326047894316021dcfe5a194c8ea52bdd94934a3457" [[package]] name = "arrayref" @@ -114,9 +114,8 @@ checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" [[package]] name = "arrow" -version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aa285343fba4d829d49985bdc541e3789cf6000ed0e84be7c039438df4a4e78c" +version = "51.0.0" +source = "git+https://github.com/viirya/arrow-rs.git?rev=3f1ae0c#3f1ae0c836b0769c88220d2180ef008b7a59158c" dependencies = [ "arrow-arith", "arrow-array", @@ -135,9 +134,8 @@ dependencies = [ [[package]] name = "arrow-arith" -version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "753abd0a5290c1bcade7c6623a556f7d1659c5f4148b140b5b63ce7bd1a45705" +version = "51.0.0" +source = "git+https://github.com/viirya/arrow-rs.git?rev=3f1ae0c#3f1ae0c836b0769c88220d2180ef008b7a59158c" dependencies = [ "arrow-array", "arrow-buffer", @@ -150,9 +148,8 @@ dependencies = [ [[package]] name = "arrow-array" -version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d390feeb7f21b78ec997a4081a025baef1e2e0d6069e181939b61864c9779609" +version = "51.0.0" +source = "git+https://github.com/viirya/arrow-rs.git?rev=3f1ae0c#3f1ae0c836b0769c88220d2180ef008b7a59158c" dependencies = [ "ahash", "arrow-buffer", @@ -161,15 +158,14 @@ dependencies = [ "chrono", "chrono-tz", "half 2.1.0", - "hashbrown 0.14.3", + "hashbrown", "num", ] [[package]] name = "arrow-buffer" -version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "69615b061701bcdffbc62756bc7e85c827d5290b472b580c972ebbbf690f5aa4" +version = "51.0.0" +source = "git+https://github.com/viirya/arrow-rs.git?rev=3f1ae0c#3f1ae0c836b0769c88220d2180ef008b7a59158c" dependencies = [ "bytes", "half 2.1.0", @@ -178,28 +174,28 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e448e5dd2f4113bf5b74a1f26531708f5edcacc77335b7066f9398f4bcf4cdef" +version = "51.0.0" +source = "git+https://github.com/viirya/arrow-rs.git?rev=3f1ae0c#3f1ae0c836b0769c88220d2180ef008b7a59158c" dependencies = [ "arrow-array", "arrow-buffer", "arrow-data", "arrow-schema", "arrow-select", + "atoi", "base64", "chrono", "comfy-table", "half 2.1.0", "lexical-core", "num", + "ryu", ] [[package]] name = "arrow-csv" -version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "46af72211f0712612f5b18325530b9ad1bfbdc87290d5fbfd32a7da128983781" +version = "51.0.0" +source = "git+https://github.com/viirya/arrow-rs.git?rev=3f1ae0c#3f1ae0c836b0769c88220d2180ef008b7a59158c" dependencies = [ "arrow-array", "arrow-buffer", @@ -216,9 +212,8 @@ dependencies = [ [[package]] name = "arrow-data" -version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "67d644b91a162f3ad3135ce1184d0a31c28b816a581e08f29e8e9277a574c64e" +version = "51.0.0" +source = "git+https://github.com/viirya/arrow-rs.git?rev=3f1ae0c#3f1ae0c836b0769c88220d2180ef008b7a59158c" dependencies = [ "arrow-buffer", "arrow-schema", @@ -228,9 +223,8 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03dea5e79b48de6c2e04f03f62b0afea7105be7b77d134f6c5414868feefb80d" +version = "51.0.0" +source = "git+https://github.com/viirya/arrow-rs.git?rev=3f1ae0c#3f1ae0c836b0769c88220d2180ef008b7a59158c" dependencies = [ "arrow-array", "arrow-buffer", @@ -243,9 +237,8 @@ dependencies = [ [[package]] name = "arrow-json" -version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8950719280397a47d37ac01492e3506a8a724b3fb81001900b866637a829ee0f" +version = "51.0.0" +source = "git+https://github.com/viirya/arrow-rs.git?rev=3f1ae0c#3f1ae0c836b0769c88220d2180ef008b7a59158c" dependencies = [ "arrow-array", "arrow-buffer", @@ -254,7 +247,7 @@ dependencies = [ "arrow-schema", "chrono", "half 2.1.0", - "indexmap 2.1.0", + "indexmap", "lexical-core", "num", "serde", @@ -263,9 +256,8 @@ dependencies = [ [[package]] name = "arrow-ord" -version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1ed9630979034077982d8e74a942b7ac228f33dd93a93b615b4d02ad60c260be" +version = "51.0.0" +source = "git+https://github.com/viirya/arrow-rs.git?rev=3f1ae0c#3f1ae0c836b0769c88220d2180ef008b7a59158c" dependencies = [ "arrow-array", "arrow-buffer", @@ -278,9 +270,8 @@ dependencies = [ [[package]] name = "arrow-row" -version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "007035e17ae09c4e8993e4cb8b5b96edf0afb927cd38e2dff27189b274d83dcf" +version = "51.0.0" +source = "git+https://github.com/viirya/arrow-rs.git?rev=3f1ae0c#3f1ae0c836b0769c88220d2180ef008b7a59158c" dependencies = [ "ahash", "arrow-array", @@ -288,23 +279,21 @@ dependencies = [ "arrow-data", "arrow-schema", "half 2.1.0", - "hashbrown 0.14.3", + "hashbrown", ] [[package]] name = "arrow-schema" -version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ff3e9c01f7cd169379d269f926892d0e622a704960350d09d331be3ec9e0029" +version = "51.0.0" +source = "git+https://github.com/viirya/arrow-rs.git?rev=3f1ae0c#3f1ae0c836b0769c88220d2180ef008b7a59158c" dependencies = [ - "bitflags 2.4.1", + "bitflags 2.5.0", ] [[package]] name = "arrow-select" -version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1ce20973c1912de6514348e064829e50947e35977bb9d7fb637dc99ea9ffd78c" +version = "51.0.0" +source = "git+https://github.com/viirya/arrow-rs.git?rev=3f1ae0c#3f1ae0c836b0769c88220d2180ef008b7a59158c" dependencies = [ "ahash", "arrow-array", @@ -316,15 +305,15 @@ dependencies = [ [[package]] name = "arrow-string" -version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "00f3b37f2aeece31a2636d1b037dabb69ef590e03bdc7eb68519b51ec86932a7" +version = "51.0.0" +source = "git+https://github.com/viirya/arrow-rs.git?rev=3f1ae0c#3f1ae0c836b0769c88220d2180ef008b7a59158c" dependencies = [ "arrow-array", "arrow-buffer", "arrow-data", "arrow-schema", "arrow-select", + "memchr", "num", "regex", "regex-syntax", @@ -338,26 +327,35 @@ checksum = "0c24e9d990669fbd16806bff449e4ac644fd9b1fca014760087732fe4102f131" [[package]] name = "async-trait" -version = "0.1.77" +version = "0.1.79" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c980ee35e870bd1a4d2c8294d4c04d0499e67bca1e4b5cefcc693c2fa00caea9" +checksum = "a507401cad91ec6a857ed5513a2073c82a9b9048762b885bb98655b306964681" dependencies = [ "proc-macro2", "quote", - "syn 2.0.48", + "syn 2.0.57", +] + +[[package]] +name = "atoi" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f28d99ec8bfea296261ca1af174f24225171fea9664ba9003cbebee704810528" +dependencies = [ + "num-traits", ] [[package]] name = "autocfg" -version = "1.1.0" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d468802bab17cbc0cc575e9b053f41e72aa36bfa6b7f55e3529ffa43161b97fa" +checksum = "f1fdabc7756949593fe60f30ec81974b613357de856987752631dea1e3394c80" [[package]] name = "backtrace" -version = "0.3.69" +version = "0.3.71" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2089b7e3f35b9dd2d0ed921ead4f6d318c27680d4a5bd167b3ee120edb105837" +checksum = "26b05800d2e817c8b3b4b54abd461726265fa9789ae34330622f2db9ee696f9d" dependencies = [ "addr2line", "cc", @@ -370,9 +368,9 @@ dependencies = [ [[package]] name = "base64" -version = "0.21.5" +version = "0.22.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "35636a1494ede3b646cc98f74f8e62c773a38a659ebc777a2cf26b9b74171df9" +checksum = "9475866fec1451be56a3c2400fd081ff546538961565ccb5b7142cbd22bc7a51" [[package]] name = "bitflags" @@ -382,9 +380,9 @@ checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" [[package]] name = "bitflags" -version = "2.4.1" +version = "2.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "327762f6e5a765692301e5bb513e0d9fef63be86bbc14528052b1cd3e6f03e07" +checksum = "cf4b9d6a944f767f8e5e0db018570623c85f3d925ac718db4e06d0187adb21c1" [[package]] name = "blake2" @@ -397,9 +395,9 @@ dependencies = [ [[package]] name = "blake3" -version = "1.5.0" +version = "1.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0231f06152bf547e9c2b5194f247cd97aacf6dcd8b15d8e5ec0663f64580da87" +checksum = "30cca6d3674597c30ddf2c587bf8d9d65c9a84d2326d941cc79c9842dfe0ef52" dependencies = [ "arrayref", "arrayvec", @@ -419,9 +417,9 @@ dependencies = [ [[package]] name = "brotli" -version = "3.4.0" +version = "3.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "516074a47ef4bce09577a3b379392300159ce5b1ba2e501ff1c819950066100f" +checksum = "d640d25bc63c50fb1f0b545ffd80207d2e10a4c965530809b40ba3386825c391" dependencies = [ "alloc-no-stdlib", "alloc-stdlib", @@ -440,15 +438,15 @@ dependencies = [ [[package]] name = "bumpalo" -version = "3.14.0" +version = "3.15.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f30e7476521f6f8af1a1c4c0b8cc94f0bee37d91763d0ca2665f299b6cd8aec" +checksum = "7ff69b9dd49fd426c69a0db9fc04dd934cdb6645ff000864d98f7e2af8830eaa" [[package]] name = "bytemuck" -version = "1.14.0" +version = "1.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "374d28ec25809ee0e23827c2ab573d729e293f281dfe393500e7ad618baa61c6" +checksum = "5d6d68c57235a3a081186990eca2867354726650f42f7516ca50c28d6281fd15" [[package]] name = "byteorder" @@ -458,9 +456,9 @@ checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" [[package]] name = "bytes" -version = "1.5.0" +version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a2bd12c1caf447e69cd4528f47f94d203fd2582878ecb9e9465484c4148a8223" +checksum = "514de17de45fdb8dc022b1a7975556c53c86f9f0aa5f534b98977b171857c2c9" [[package]] name = "cast" @@ -470,9 +468,9 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "cc" -version = "1.0.83" +version = "1.0.90" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f1174fb0b6ec23863f8b971027804a42614e347eafb0a95bf0b12cdae21fc4d0" +checksum = "8cd6604a82acf3039f1144f54b8eb34e91ffba622051189e71b781822d5ee1f5" dependencies = [ "jobserver", "libc", @@ -492,23 +490,21 @@ checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" [[package]] name = "chrono" -version = "0.4.34" +version = "0.4.37" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5bc015644b92d5890fab7489e49d21f879d5c990186827d42ec511919404f38b" +checksum = "8a0d04d43504c61aa6c7531f1871dd0d418d91130162063b789da00fd7057a5e" dependencies = [ "android-tzdata", "iana-time-zone", - "js-sys", "num-traits", - "wasm-bindgen", - "windows-targets 0.52.0", + "windows-targets 0.52.4", ] [[package]] name = "chrono-tz" -version = "0.8.5" +version = "0.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "91d7b79e99bfaa0d47da0687c43aa3b7381938a62ad3a6498599039321f660b7" +checksum = "d59ae0466b83e838b81a54256c39d5d7c20b9d7daa10510a242d9b75abd5936e" dependencies = [ "chrono", "chrono-tz-build", @@ -539,9 +535,9 @@ dependencies = [ [[package]] name = "ciborium-io" -version = "0.2.1" +version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cdf919175532b369853f5d5e20b26b43112613fd6fe7aee757e35f7a44642656" +checksum = "05afea1e0a06c9be33d539b876f1ce3692f4afea2cb41f740e7743225ed1c757" [[package]] name = "ciborium-ll" @@ -550,23 +546,23 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "defaa24ecc093c77630e6c15e17c51f5e187bf35ee514f4e2d67baaa96dae22b" dependencies = [ "ciborium-io", - "half 1.8.2", + "half 1.8.3", ] [[package]] name = "clap" -version = "4.4.13" +version = "4.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "52bdc885e4cacc7f7c9eedc1ef6da641603180c783c41a15c264944deeaab642" +checksum = "90bc066a67923782aa8515dbaea16946c5bcc5addbd668bb80af688e53e548a0" dependencies = [ "clap_builder", ] [[package]] name = "clap_builder" -version = "4.4.12" +version = "4.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb7fb5e4e979aec3be7791562fcba452f94ad85e954da024396433e0e25a79e9" +checksum = "ae129e2e766ae0ec03484e609954119f123cc1fe650337e155d03b022f24f7b4" dependencies = [ "anstyle", "clap_lex", @@ -574,9 +570,9 @@ dependencies = [ [[package]] name = "clap_lex" -version = "0.6.0" +version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "702fc72eb24e5a1e48ce58027a675bc24edd52096d5397d4aea7c6dd9eca0bd1" +checksum = "98cc8fbded0c607b7ba9dd60cd98df59af97e84d24e49c8557331cfc26d301ce" [[package]] name = "combine" @@ -609,11 +605,12 @@ dependencies = [ "criterion", "datafusion", "datafusion-common", + "datafusion-functions", "datafusion-physical-expr", "flate2", "futures", "half 2.1.0", - "hashbrown 0.14.3", + "hashbrown", "itertools 0.11.0", "jni", "lazy_static", @@ -657,9 +654,9 @@ dependencies = [ [[package]] name = "const-random" -version = "0.1.17" +version = "0.1.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5aaf16c9c2c612020bcfd042e170f6e32de9b9d75adb5277cdbbd2e2c8c8299a" +checksum = "87e00182fe74b066627d63b85fd550ac2998d4b0bd86bfed477a0ae4c7c71359" dependencies = [ "const-random-macro", ] @@ -707,9 +704,9 @@ dependencies = [ [[package]] name = "crc32fast" -version = "1.3.2" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b540bd8bc810d3885c6ea91e2018302f68baba2129ab3e88f32389ee9370880d" +checksum = "b3855a8a784b474f333699ef2bbca9db2c4a1f6d9088a90a2d25b1eb53111eaa" dependencies = [ "cfg-if", ] @@ -752,34 +749,28 @@ dependencies = [ [[package]] name = "crossbeam-deque" -version = "0.8.4" +version = "0.8.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fca89a0e215bab21874660c67903c5f143333cab1da83d041c7ded6053774751" +checksum = "613f8cc01fe9cf1a3eb3d7f488fd2fa8388403e97039e2f73692932e291a770d" dependencies = [ - "cfg-if", "crossbeam-epoch", "crossbeam-utils", ] [[package]] name = "crossbeam-epoch" -version = "0.9.17" +version = "0.9.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0e3681d554572a651dda4186cd47240627c3d0114d45a95f6ad27f2f22e7548d" +checksum = "5b82ac4a3c2ca9c3460964f020e1402edd5753411d7737aa39c3714ad1b5420e" dependencies = [ - "autocfg", - "cfg-if", "crossbeam-utils", ] [[package]] name = "crossbeam-utils" -version = "0.8.18" +version = "0.8.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3a430a770ebd84726f584a90ee7f020d28db52c6d02138900f22341f866d39c" -dependencies = [ - "cfg-if", -] +checksum = "248e3bacc7dc6baa3b21e405ee045c3047101a49145e7e9eca583ab4c2ca5345" [[package]] name = "crunchy" @@ -825,7 +816,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "978747c1d849a7d2ee5e8adc0159961c48fb7e5db2f06af6723b80123bb53856" dependencies = [ "cfg-if", - "hashbrown 0.14.3", + "hashbrown", "lock_api", "once_cell", "parking_lot_core", @@ -834,8 +825,7 @@ dependencies = [ [[package]] name = "datafusion" version = "36.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b2b360b692bf6c6d6e6b6dbaf41a3be0020daeceac0f406aed54c75331e50dbb" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=111a940#111a940b297aa83839e4e2273f0e1a38e108b370" dependencies = [ "ahash", "arrow", @@ -847,6 +837,7 @@ dependencies = [ "chrono", "dashmap", "datafusion-common", + "datafusion-common-runtime", "datafusion-execution", "datafusion-expr", "datafusion-functions", @@ -857,9 +848,9 @@ dependencies = [ "futures", "glob", "half 2.1.0", - "hashbrown 0.14.3", - "indexmap 2.1.0", - "itertools 0.12.0", + "hashbrown", + "indexmap", + "itertools 0.12.1", "log", "num_cpus", "object_store", @@ -876,8 +867,7 @@ dependencies = [ [[package]] name = "datafusion-common" version = "36.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "37f343ccc298f440e25aa38ff82678291a7acc24061c7370ba6c0ff5cc811412" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=111a940#111a940b297aa83839e4e2273f0e1a38e108b370" dependencies = [ "ahash", "arrow", @@ -886,17 +876,25 @@ dependencies = [ "arrow-schema", "chrono", "half 2.1.0", + "instant", "libc", "num_cpus", "object_store", "sqlparser", ] +[[package]] +name = "datafusion-common-runtime" +version = "36.0.0" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=111a940#111a940b297aa83839e4e2273f0e1a38e108b370" +dependencies = [ + "tokio", +] + [[package]] name = "datafusion-execution" version = "36.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3f9c93043081487e335399a21ebf8295626367a647ac5cb87d41d18afad7d0f7" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=111a940#111a940b297aa83839e4e2273f0e1a38e108b370" dependencies = [ "arrow", "chrono", @@ -904,7 +902,7 @@ dependencies = [ "datafusion-common", "datafusion-expr", "futures", - "hashbrown 0.14.3", + "hashbrown", "log", "object_store", "parking_lot", @@ -916,39 +914,47 @@ dependencies = [ [[package]] name = "datafusion-expr" version = "36.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e204d89909e678846b6a95f156aafc1ee5b36cb6c9e37ec2e1449b078a38c818" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=111a940#111a940b297aa83839e4e2273f0e1a38e108b370" dependencies = [ "ahash", "arrow", "arrow-array", + "chrono", "datafusion-common", "paste", "sqlparser", - "strum 0.26.1", - "strum_macros 0.26.1", + "strum 0.26.2", + "strum_macros 0.26.2", ] [[package]] name = "datafusion-functions" version = "36.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "98f1c73f7801b2b8ba2297b3ad78ffcf6c1fc6b8171f502987eb9ad5cb244ee7" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=111a940#111a940b297aa83839e4e2273f0e1a38e108b370" dependencies = [ "arrow", "base64", + "blake2", + "blake3", + "chrono", "datafusion-common", "datafusion-execution", "datafusion-expr", + "datafusion-physical-expr", "hex", + "itertools 0.12.1", "log", + "md-5", + "regex", + "sha2", + "unicode-segmentation", + "uuid", ] [[package]] name = "datafusion-optimizer" version = "36.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5ae27e07bf1f04d327be5c2a293470879801ab5535204dc3b16b062fda195496" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=111a940#111a940b297aa83839e4e2273f0e1a38e108b370" dependencies = [ "arrow", "async-trait", @@ -956,8 +962,8 @@ dependencies = [ "datafusion-common", "datafusion-expr", "datafusion-physical-expr", - "hashbrown 0.14.3", - "itertools 0.12.0", + "hashbrown", + "itertools 0.12.1", "log", "regex-syntax", ] @@ -965,8 +971,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" version = "36.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dde620cd9ef76a3bca9c754fb68854bd2349c49f55baf97e08001f9e967f6d6b" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=111a940#111a940b297aa83839e4e2273f0e1a38e108b370" dependencies = [ "ahash", "arrow", @@ -983,10 +988,10 @@ dependencies = [ "datafusion-execution", "datafusion-expr", "half 2.1.0", - "hashbrown 0.14.3", + "hashbrown", "hex", - "indexmap 2.1.0", - "itertools 0.12.0", + "indexmap", + "itertools 0.12.1", "log", "md-5", "paste", @@ -995,14 +1000,12 @@ dependencies = [ "regex", "sha2", "unicode-segmentation", - "uuid", ] [[package]] name = "datafusion-physical-plan" version = "36.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a4c75fba9ea99d64b2246cbd2fcae2e6fc973e6616b1015237a616036506dd4" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=111a940#111a940b297aa83839e4e2273f0e1a38e108b370" dependencies = [ "ahash", "arrow", @@ -1012,35 +1015,36 @@ dependencies = [ "async-trait", "chrono", "datafusion-common", + "datafusion-common-runtime", "datafusion-execution", "datafusion-expr", "datafusion-physical-expr", "futures", "half 2.1.0", - "hashbrown 0.14.3", - "indexmap 2.1.0", - "itertools 0.12.0", + "hashbrown", + "indexmap", + "itertools 0.12.1", "log", "once_cell", "parking_lot", "pin-project-lite", "rand", "tokio", - "uuid", ] [[package]] name = "datafusion-sql" version = "36.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "21474a95c3a62d113599d21b439fa15091b538bac06bd20be0bb2e7d22903c09" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=111a940#111a940b297aa83839e4e2273f0e1a38e108b370" dependencies = [ "arrow", + "arrow-array", "arrow-schema", "datafusion-common", "datafusion-expr", "log", "sqlparser", + "strum 0.26.2", ] [[package]] @@ -1088,9 +1092,9 @@ checksum = "fea41bba32d969b513997752735605054bc0dfa92b4c56bf1189f2e174be7a10" [[package]] name = "either" -version = "1.9.0" +version = "1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a26ae43d7bcc3b814de94796a5e736d4029efb0ee900c12e2d54c993ad1a1e07" +checksum = "11157ac094ffbdde99aa67b23417ebdd801842852b500e395a45a9c0aac03e4a" [[package]] name = "equivalent" @@ -1110,9 +1114,9 @@ dependencies = [ [[package]] name = "fastrand" -version = "2.0.1" +version = "2.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "25cbce373ec4653f1a01a31e8a5e5ec0c622dc27ff9c4e6606eefef5cbbed4a5" +checksum = "658bd65b1cf4c852a3cc96f18a8ce7b5640f6b703f905c7d74532294c2a63984" [[package]] name = "findshlibs" @@ -1223,7 +1227,7 @@ checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" dependencies = [ "proc-macro2", "quote", - "syn 2.0.48", + "syn 2.0.57", ] [[package]] @@ -1268,9 +1272,9 @@ dependencies = [ [[package]] name = "getrandom" -version = "0.2.11" +version = "0.2.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fe9006bed769170c11f845cf00c7c1e9092aeb3f268e007c3e760ac68008070f" +checksum = "190092ea657667030ac6a35e305e62fc4dd69fd98ac98631e5d3a2b1575a12b5" dependencies = [ "cfg-if", "libc", @@ -1291,9 +1295,9 @@ checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" [[package]] name = "half" -version = "1.8.2" +version = "1.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eabb4a44450da02c90444cf74558da904edde8fb4e9035a9a6a4e15445af0bd7" +checksum = "1b43ede17f21864e81be2fa654110bf1e793774238d86ef8555c37e6519c0403" [[package]] name = "half" @@ -1305,12 +1309,6 @@ dependencies = [ "num-traits", ] -[[package]] -name = "hashbrown" -version = "0.12.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" - [[package]] name = "hashbrown" version = "0.14.3" @@ -1338,9 +1336,9 @@ checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8" [[package]] name = "hermit-abi" -version = "0.3.3" +version = "0.3.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d77f7ec81a6d05a3abb01ab6eb7590f6083d08449fe5a1c8b1e620283546ccb7" +checksum = "d231dfb89cfffdbc30e7fc41579ed6066ad03abda9e567ccafae602b97ec5024" [[package]] name = "hex" @@ -1365,9 +1363,9 @@ checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" [[package]] name = "iana-time-zone" -version = "0.1.59" +version = "0.1.60" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b6a67363e2aa4443928ce15e57ebae94fd8949958fd1223c4cfc0cd473ad7539" +checksum = "e7ffbb5a1b541ea2561f8c41c087286cc091e21e556a4f09a8f6cbf17b69b141" dependencies = [ "android_system_properties", "core-foundation-sys", @@ -1398,22 +1396,12 @@ dependencies = [ [[package]] name = "indexmap" -version = "1.9.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bd070e393353796e801d209ad339e89596eb4c8d430d18ede6a1cced8fafbd99" -dependencies = [ - "autocfg", - "hashbrown 0.12.3", -] - -[[package]] -name = "indexmap" -version = "2.1.0" +version = "2.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d530e1a18b1cb4c484e6e34556a0d948706958449fca0cab753d649f2bce3d1f" +checksum = "168fb715dda47215e360912c096649d23d58bf392ac62f73919e831745e40f26" dependencies = [ "equivalent", - "hashbrown 0.14.3", + "hashbrown", ] [[package]] @@ -1423,7 +1411,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "321f0f839cd44a4686e9504b0a62b4d69a50b62072144c71c68f5873c167b8d9" dependencies = [ "ahash", - "indexmap 2.1.0", + "indexmap", "is-terminal", "itoa", "log", @@ -1434,6 +1422,18 @@ dependencies = [ "str_stack", ] +[[package]] +name = "instant" +version = "0.1.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7a5bbe824c507c5da5956355e86a746d82e0e1464f65d862cc5e71da70e94b2c" +dependencies = [ + "cfg-if", + "js-sys", + "wasm-bindgen", + "web-sys", +] + [[package]] name = "integer-encoding" version = "1.1.7" @@ -1448,12 +1448,12 @@ checksum = "8bb03732005da905c88227371639bf1ad885cc712789c011c31c5fb3ab3ccf02" [[package]] name = "is-terminal" -version = "0.4.10" +version = "0.4.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0bad00257d07be169d870ab665980b06cdb366d792ad690bf2e76876dc503455" +checksum = "f23ff5ef2b80d608d61efee834934d862cd92461afc0560dedf493e4c033738b" dependencies = [ "hermit-abi", - "rustix", + "libc", "windows-sys 0.52.0", ] @@ -1477,18 +1477,18 @@ dependencies = [ [[package]] name = "itertools" -version = "0.12.0" +version = "0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "25db6b064527c5d482d0423354fcd07a89a2dfe07b67892e62411946db7f07b0" +checksum = "ba291022dbbd398a455acf126c1e341954079855bc60dfdda641363bd6922569" dependencies = [ "either", ] [[package]] name = "itoa" -version = "1.0.10" +version = "1.0.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b1a46d1a171d865aa5f83f92695765caa047a9b4cbae2cbf37dbd613a793fd4c" +checksum = "49f1f14873335454500d59611f1cf4a4b0f786f9ac11f4312a78e4cf2566695b" [[package]] name = "java-locator" @@ -1526,18 +1526,18 @@ checksum = "8eaf4bc02d17cbdd7ff4c7438cafcdf7fb9a4613313ad11b4f8fefe7d3fa0130" [[package]] name = "jobserver" -version = "0.1.27" +version = "0.1.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c37f63953c4c63420ed5fd3d6d398c719489b9f872b9fa683262f8edd363c7d" +checksum = "ab46a6e9526ddef3ae7f787c06f0f2600639ba80ea3eade3d8e670a2230f51d6" dependencies = [ "libc", ] [[package]] name = "js-sys" -version = "0.3.66" +version = "0.3.69" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cee9c64da59eae3b50095c18d3e74f8b73c0b86d2792824ff01bbce68ba229ca" +checksum = "29c15563dc2726973df627357ce0c9ddddbea194836909d655df6a75d2cf296d" dependencies = [ "wasm-bindgen", ] @@ -1614,9 +1614,9 @@ dependencies = [ [[package]] name = "libc" -version = "0.2.151" +version = "0.2.153" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "302d7ab3130588088d277783b1e2d2e10c9e9e4a16dd9050e6ec93fb3e7048f4" +checksum = "9c198f91728a82281a64e1f4f9eeb25d82cb32a5de251c6bd1b5154d63a8e7bd" [[package]] name = "libloading" @@ -1644,17 +1644,11 @@ dependencies = [ "libc", ] -[[package]] -name = "linked-hash-map" -version = "0.5.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0717cef1bc8b636c6e1c1bbdefc09e6322da8a9321966e8928ef80d20f7f770f" - [[package]] name = "linux-raw-sys" -version = "0.4.12" +version = "0.4.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c4cd1a83af159aa67994778be9070f0ae1bd732942279cabb14f86f986a21456" +checksum = "01cda141df6706de531b6c46c3a33ecca755538219bd484262fa09410c13539c" [[package]] name = "lock_api" @@ -1668,9 +1662,9 @@ dependencies = [ [[package]] name = "log" -version = "0.4.20" +version = "0.4.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b5e6163cb8c49088c2c36f57875e58ccd8c87c7427f7fbd50ea6710b2f3f2e8f" +checksum = "90ed8c1e510134f979dbc4f070f87d4313098b704861a105fe34231c70a3901c" dependencies = [ "serde", ] @@ -1683,9 +1677,9 @@ checksum = "a94d21414c1f4a51209ad204c1776a3d0765002c76c6abcb602a6f09f1e881c7" [[package]] name = "log4rs" -version = "1.2.0" +version = "1.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d36ca1786d9e79b8193a68d480a0907b612f109537115c6ff655a3a1967533fd" +checksum = "0816135ae15bd0391cf284eab37e6e3ee0a6ee63d2ceeb659862bd8d0a984ca6" dependencies = [ "anyhow", "arc-swap", @@ -1696,7 +1690,9 @@ dependencies = [ "libc", "log", "log-mdc", + "once_cell", "parking_lot", + "rand", "serde", "serde-value", "serde_json", @@ -1729,9 +1725,9 @@ dependencies = [ [[package]] name = "lz4_flex" -version = "0.11.2" +version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "912b45c753ff5f7f5208307e8ace7d2a2e30d024e26d3509f3dce546c044ce15" +checksum = "75761162ae2b0e580d7e7c390558127e5f01b4194debd6221fd8c207fc80e3f5" dependencies = [ "twox-hash", ] @@ -1748,15 +1744,15 @@ dependencies = [ [[package]] name = "memchr" -version = "2.7.1" +version = "2.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "523dc4f511e55ab87b694dc30d0f820d60906ef06413f93d4d7a1385599cc149" +checksum = "6c8640c5d730cb13ebd907d8d04b52f55ac9a2eec55b440c8892f40d56c76c1d" [[package]] name = "memmap2" -version = "0.9.3" +version = "0.9.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "45fd3a57831bf88bc63f8cebc0cf956116276e97fef3966103e96416209f7c92" +checksum = "fe751422e4a8caa417e13c3ea66452215d7d63e19e604f4980461212f3ae1322" dependencies = [ "libc", ] @@ -1772,9 +1768,9 @@ dependencies = [ [[package]] name = "miniz_oxide" -version = "0.7.1" +version = "0.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7810e0be55b428ada41041c41f32c9f1a42817901b4ccf45fa3d4b6561e74c7" +checksum = "9d811f3e15f28568be3407c8e7fdb6514c1cda3cb30683f15b6a1a1dc4ea14a7" dependencies = [ "adler", ] @@ -1823,9 +1819,9 @@ dependencies = [ [[package]] name = "num-complex" -version = "0.4.4" +version = "0.4.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1ba157ca0885411de85d6ca030ba7e2a83a28636056c7c699b07c8b6f7383214" +checksum = "23c6602fda94a57c990fe0df199a035d83576b496aa29f4e634a8ac6004e68a6" dependencies = [ "num-traits", ] @@ -1842,19 +1838,18 @@ dependencies = [ [[package]] name = "num-integer" -version = "0.1.45" +version = "0.1.46" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "225d3389fb3509a24c93f5c29eb6bde2586b98d9f016636dff58d7c6f7569cd9" +checksum = "7969661fd2958a5cb096e56c8e1ad0444ac2bbcd0061bd28660485a44879858f" dependencies = [ - "autocfg", "num-traits", ] [[package]] name = "num-iter" -version = "0.1.43" +version = "0.1.44" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d03e6c028c5dc5cac6e2dec0efda81fc887605bb3d884578bb6d6bf7514e252" +checksum = "d869c01cc0c455284163fd0092f1f93835385ccab5a98a0dcc497b2f8bf055a9" dependencies = [ "autocfg", "num-integer", @@ -1875,9 +1870,9 @@ dependencies = [ [[package]] name = "num-traits" -version = "0.2.17" +version = "0.2.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "39e3200413f237f41ab11ad6d161bc7239c84dcb631773ccd7de3dfe4b5c267c" +checksum = "da0df0e5185db44f69b44f26786fe401b6c293d1907744beaa7fa62b2e5a517a" dependencies = [ "autocfg", "libm", @@ -1904,16 +1899,16 @@ dependencies = [ [[package]] name = "object_store" -version = "0.9.0" +version = "0.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d139f545f64630e2e3688fd9f81c470888ab01edeb72d13b4e86c566f1130000" +checksum = "b8718f8b65fdf67a45108d1548347d4af7d71fb81ce727bbf9e3b2535e079db3" dependencies = [ "async-trait", "bytes", "chrono", "futures", "humantime", - "itertools 0.12.0", + "itertools 0.12.1", "parking_lot", "percent-encoding", "snafu", @@ -1978,15 +1973,14 @@ dependencies = [ [[package]] name = "parquet" -version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "547b92ebf0c1177e3892f44c8f79757ee62e678d564a9834189725f2c5b7a750" +version = "51.0.0" +source = "git+https://github.com/viirya/arrow-rs.git?rev=3f1ae0c#3f1ae0c836b0769c88220d2180ef008b7a59158c" dependencies = [ "ahash", "bytes", "chrono", "half 2.1.0", - "hashbrown 0.14.3", + "hashbrown", "num", "num-bigint", "paste", @@ -2032,7 +2026,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e1d3afd2628e69da2be385eb6f2fd57c8ac7977ceeff6dc166ff1657b0e386a9" dependencies = [ "fixedbitset", - "indexmap 2.1.0", + "indexmap", ] [[package]] @@ -2075,9 +2069,9 @@ dependencies = [ [[package]] name = "pin-project-lite" -version = "0.2.13" +version = "0.2.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8afb450f006bf6385ca15ef45d71d2288452bc3683ce2e2cacc0d18e4be60b58" +checksum = "bda66fc9667c18cb2758a2ac84d1167245054bcf85d5d1aaa6923f45801bdd02" [[package]] name = "pin-utils" @@ -2087,9 +2081,9 @@ checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" [[package]] name = "pkg-config" -version = "0.3.28" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "69d3587f8a9e599cc7ec2c00e331f71c4e69a5f9a4b8a6efd5b07466b9736f9a" +checksum = "d231b230927b5e4ad203db57bbcbee2802f6bce620b1e4a9024a07d94e2907ec" [[package]] name = "plotters" @@ -2148,9 +2142,9 @@ checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de" [[package]] name = "proc-macro2" -version = "1.0.75" +version = "1.0.79" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "907a61bd0f64c2f29cd1cf1dc34d05176426a3f504a78010f08416ddb7b13708" +checksum = "e835ff2298f5721608eb1a980ecaee1aef2c132bf95ecc026a11b7bf3c01c02e" dependencies = [ "unicode-ident", ] @@ -2218,7 +2212,7 @@ dependencies = [ "itertools 0.11.0", "proc-macro2", "quote", - "syn 2.0.48", + "syn 2.0.57", ] [[package]] @@ -2281,9 +2275,9 @@ dependencies = [ [[package]] name = "rayon" -version = "1.8.0" +version = "1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c27db03db7734835b3f53954b534c91069375ce6ccaa2e065441e07d9b6cdb1" +checksum = "b418a60154510ca1a002a752ca9714984e21e4241e804d32555251faf8b78ffa" dependencies = [ "either", "rayon-core", @@ -2291,9 +2285,9 @@ dependencies = [ [[package]] name = "rayon-core" -version = "1.12.0" +version = "1.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5ce3fb6ad83f861aac485e76e1985cd109d9a3713802152be56c3b1f0e0658ed" +checksum = "1465873a3dfdaa8ae7cb14b4383657caab0b3e8a0aa9ae8e04b044854c8dfce2" dependencies = [ "crossbeam-deque", "crossbeam-utils", @@ -2310,9 +2304,9 @@ dependencies = [ [[package]] name = "regex" -version = "1.10.2" +version = "1.10.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "380b951a9c5e80ddfd6136919eef32310721aa4aacd4889a8d39124b026ab343" +checksum = "c117dbdfde9c8308975b6a18d71f3f385c89461f7b3fb054288ecf2a2058ba4c" dependencies = [ "aho-corasick", "memchr", @@ -2322,9 +2316,9 @@ dependencies = [ [[package]] name = "regex-automata" -version = "0.4.3" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f804c7828047e88b2d32e2d7fe5a105da8ee3264f01902f796c8e067dc2483f" +checksum = "86b83b8b9847f9bf95ef68afb0b8e6cdb80f498442f5179a29fad448fcc1eaea" dependencies = [ "aho-corasick", "memchr", @@ -2333,9 +2327,9 @@ dependencies = [ [[package]] name = "regex-syntax" -version = "0.8.2" +version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c08c74e62047bb2de4ff487b251e4a92e24f48745648451635cec7d591162d9f" +checksum = "adad44e29e4c806119491a7f06f03de4d1af22c3a680dd47f1e6e179439d1f56" [[package]] name = "rgb" @@ -2363,11 +2357,11 @@ dependencies = [ [[package]] name = "rustix" -version = "0.38.28" +version = "0.38.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "72e572a5e8ca657d7366229cdde4bd14c4eb5499a9573d4d366fe1b599daa316" +checksum = "65e04861e65f21776e67888bfbea442b3642beaa0138fdb1dd7a84a52dffdb89" dependencies = [ - "bitflags 2.4.1", + "bitflags 2.5.0", "errno", "libc", "linux-raw-sys", @@ -2382,9 +2376,9 @@ checksum = "7ffc183a10b4478d04cbbbfc96d0873219d962dd5accaff2ffbd4ceb7df837f4" [[package]] name = "ryu" -version = "1.0.16" +version = "1.0.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f98d2aa92eebf49b69786be48e4477826b256916e84a57ff2a4f21923b48eb4c" +checksum = "e86697c916019a8588c99b5fac3cead74ec0b4b819707a682fd4d23fa0ce1ba1" [[package]] name = "same-file" @@ -2403,9 +2397,9 @@ checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" [[package]] name = "semver" -version = "1.0.21" +version = "1.0.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b97ed7a9823b74f99c7742f5336af7be5ecd3eeafcb1507d1fa93347b1d589b0" +checksum = "92d43fe69e652f3df9bdc2b85b2854a0825b86e4fb76bc44d945137d053639ca" [[package]] name = "seq-macro" @@ -2415,9 +2409,9 @@ checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" [[package]] name = "serde" -version = "1.0.194" +version = "1.0.197" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0b114498256798c94a0689e1a15fec6005dee8ac1f41de56404b67afc2a4b773" +checksum = "3fb1c873e1b9b056a4dc4c0c198b24c3ffa059243875552b2bd0933b1aee4ce2" dependencies = [ "serde_derive", ] @@ -2434,20 +2428,20 @@ dependencies = [ [[package]] name = "serde_derive" -version = "1.0.194" +version = "1.0.197" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a3385e45322e8f9931410f01b3031ec534c3947d0e94c18049af4d9f9907d4e0" +checksum = "7eb0b34b42edc17f6b7cac84a52a1c5f0e1bb2227e997ca9011ea3dd34e8610b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.48", + "syn 2.0.57", ] [[package]] name = "serde_json" -version = "1.0.111" +version = "1.0.115" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "176e46fa42316f18edd598015a5166857fc835ec732f5215eac6b7bdbf0a84f4" +checksum = "12dc5c46daa8e9fdf4f5e71b6cf9a53f2487da0e86e55808e2d35539666497dd" dependencies = [ "itoa", "ryu", @@ -2456,14 +2450,15 @@ dependencies = [ [[package]] name = "serde_yaml" -version = "0.8.26" +version = "0.9.34+deprecated" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "578a7433b776b56a35785ed5ce9a7e777ac0598aac5a6dd1b4b18a307c7fc71b" +checksum = "6a8b1a1a2ebf674015cc02edccce75287f1a0130d394307b36743c2f5d504b47" dependencies = [ - "indexmap 1.9.3", + "indexmap", + "itoa", "ryu", "serde", - "yaml-rust", + "unsafe-libyaml", ] [[package]] @@ -2500,9 +2495,9 @@ dependencies = [ [[package]] name = "smallvec" -version = "1.11.2" +version = "1.13.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4dccd0940a2dcdf68d092b8cbab7dc0ad8fa938bf95787e1b916b0e3d0e8e970" +checksum = "3c5e1a9a646d36c3599cd173a41282daf47c44583ad367b8e6837255952e5c67" [[package]] name = "snafu" @@ -2534,9 +2529,9 @@ checksum = "1b6b67fb9a61334225b5b790716f609cd58395f895b3fe8b328786812a40bc3b" [[package]] name = "sqlparser" -version = "0.43.1" +version = "0.44.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f95c4bae5aba7cd30bd506f7140026ade63cff5afd778af8854026f9606bf5d4" +checksum = "aaf9c7ff146298ffda83a200f8d5084f08dcee1edfc135fcc1d646a45d50ffd6" dependencies = [ "log", "sqlparser_derive", @@ -2550,7 +2545,7 @@ checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" dependencies = [ "proc-macro2", "quote", - "syn 2.0.48", + "syn 2.0.57", ] [[package]] @@ -2579,11 +2574,11 @@ checksum = "290d54ea6f91c969195bdbcd7442c8c2a2ba87da8bf60a7ee86a235d4bc1e125" [[package]] name = "strum" -version = "0.26.1" +version = "0.26.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "723b93e8addf9aa965ebe2d11da6d7540fa2283fcea14b3371ff055f7ba13f5f" +checksum = "5d8cec3501a5194c432b2b7976db6b7d10ec95c253208b45f83f7136aa985e29" dependencies = [ - "strum_macros 0.26.1", + "strum_macros 0.26.2", ] [[package]] @@ -2596,20 +2591,20 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.48", + "syn 2.0.57", ] [[package]] name = "strum_macros" -version = "0.26.1" +version = "0.26.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a3417fc93d76740d974a01654a09777cb500428cc874ca9f45edfe0c4d4cd18" +checksum = "c6cf59daf282c0a494ba14fd21610a0325f9f90ec9d1231dea26bcb1d696c946" dependencies = [ "heck 0.4.1", "proc-macro2", "quote", "rustversion", - "syn 2.0.48", + "syn 2.0.57", ] [[package]] @@ -2654,9 +2649,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.48" +version = "2.0.57" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0f3531638e407dfc0814761abb7c00a5b54992b849452a0646b7f65c9f770f3f" +checksum = "11a6ae1e52eb25aab8f3fb9fca13be982a373b8f1157ca14b897a825ba4a2d35" dependencies = [ "proc-macro2", "quote", @@ -2665,35 +2660,34 @@ dependencies = [ [[package]] name = "tempfile" -version = "3.9.0" +version = "3.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "01ce4141aa927a6d1bd34a041795abd0db1cccba5d5f24b009f694bdf3a1f3fa" +checksum = "85b77fafb263dd9d05cbeac119526425676db3784113aa9295c88498cbf8bff1" dependencies = [ "cfg-if", "fastrand", - "redox_syscall", "rustix", "windows-sys 0.52.0", ] [[package]] name = "thiserror" -version = "1.0.56" +version = "1.0.58" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d54378c645627613241d077a3a79db965db602882668f9136ac42af9ecb730ad" +checksum = "03468839009160513471e86a034bb2c5c0e4baae3b43f79ffc55c4a5427b3297" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.56" +version = "1.0.58" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fa0faa943b50f3db30a20aa7e265dbc66076993efed8463e8de414e5d06d3471" +checksum = "c61f3ba182994efc43764a46c018c347bc492c79f024e705f46567b418f6d4f7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.48", + "syn 2.0.57", ] [[package]] @@ -2777,9 +2771,9 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.36.0" +version = "1.37.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "61285f6515fa018fb2d1e46eb21223fff441ee8db5d0f1435e8ab4f5cdb80931" +checksum = "1adbebffeca75fcfd058afa480fb6c0b81e165a0323f9c9d39c9697e37c46787" dependencies = [ "backtrace", "bytes", @@ -2796,14 +2790,14 @@ checksum = "5b8a1e28f2deaa14e508979454cb3a223b10b938b45af148bc0986de36f1923b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.48", + "syn 2.0.57", ] [[package]] name = "tokio-stream" -version = "0.1.14" +version = "0.1.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "397c988d37662c7dda6d2208364a706264bf3d6138b11d436cbac0ad38832842" +checksum = "267ac89e0bec6e691e5813911606935d77c476ff49024f98abcea3e7b15e37af" dependencies = [ "futures-core", "pin-project-lite", @@ -2829,7 +2823,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.48", + "syn 2.0.57", ] [[package]] @@ -2868,9 +2862,9 @@ checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" [[package]] name = "unicode-bidi" -version = "0.3.14" +version = "0.3.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6f2528f27a9eb2b21e69c95319b30bd0efd85d09c379741b0f78ea1d86be2416" +checksum = "08f95100a766bf4f8f28f90d77e0a5461bbdb219042e7679bebe79004fed8d75" [[package]] name = "unicode-ident" @@ -2880,18 +2874,18 @@ checksum = "3354b9ac3fae1ff6755cb6db53683adb661634f67557942dea4facebec0fee4b" [[package]] name = "unicode-normalization" -version = "0.1.22" +version = "0.1.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c5713f0fc4b5db668a2ac63cdb7bb4469d8c9fed047b1d0292cc7b0ce2ba921" +checksum = "a56d1686db2308d901306f92a263857ef59ea39678a5458e7cb17f01415101f5" dependencies = [ "tinyvec", ] [[package]] name = "unicode-segmentation" -version = "1.10.1" +version = "1.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1dd624098567895118886609431a7c3b8f516e41d30e0643f03d94592a147e36" +checksum = "d4c87d22b6e3f4a18d4d40ef354e97c90fcb14dd91d7dc0aa9d8a1172ebf7202" [[package]] name = "unicode-width" @@ -2908,6 +2902,12 @@ dependencies = [ "destructure_traitobject", ] +[[package]] +name = "unsafe-libyaml" +version = "0.2.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "673aac59facbab8a9007c7f6108d11f63b603f7cabff99fabf650fea5c32b861" + [[package]] name = "url" version = "2.5.0" @@ -2921,9 +2921,9 @@ dependencies = [ [[package]] name = "uuid" -version = "1.6.1" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5e395fcf16a7a3d8127ec99782007af141946b4795001f876d54fb0d55978560" +checksum = "a183cf7feeba97b4dd1c0d46788634f6221d87fa961b305bed08c851829efcc0" dependencies = [ "getrandom", ] @@ -2936,9 +2936,9 @@ checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" [[package]] name = "walkdir" -version = "2.4.0" +version = "2.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d71d857dc86794ca4c280d616f7da00d2dbfd8cd788846559a6813e6aa4b54ee" +checksum = "29790946404f91d9c5d06f9874efddea1dc06c5efe94541a7d6863108e3a5e4b" dependencies = [ "same-file", "winapi-util", @@ -2952,9 +2952,9 @@ checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" [[package]] name = "wasm-bindgen" -version = "0.2.89" +version = "0.2.92" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ed0d4f68a3015cc185aff4db9506a015f4b96f95303897bfa23f846db54064e" +checksum = "4be2531df63900aeb2bca0daaaddec08491ee64ceecbee5076636a3b026795a8" dependencies = [ "cfg-if", "wasm-bindgen-macro", @@ -2962,24 +2962,24 @@ dependencies = [ [[package]] name = "wasm-bindgen-backend" -version = "0.2.89" +version = "0.2.92" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1b56f625e64f3a1084ded111c4d5f477df9f8c92df113852fa5a374dbda78826" +checksum = "614d787b966d3989fa7bb98a654e369c762374fd3213d212cfc0251257e747da" dependencies = [ "bumpalo", "log", "once_cell", "proc-macro2", "quote", - "syn 2.0.48", + "syn 2.0.57", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-macro" -version = "0.2.89" +version = "0.2.92" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0162dbf37223cd2afce98f3d0785506dcb8d266223983e4b5b525859e6e182b2" +checksum = "a1f8823de937b71b9460c0c34e25f3da88250760bec0ebac694b49997550d726" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -2987,28 +2987,28 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.89" +version = "0.2.92" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f0eb82fcb7930ae6219a7ecfd55b217f5f0893484b7a13022ebb2b2bf20b5283" +checksum = "e94f17b526d0a461a191c78ea52bbce64071ed5c04c9ffe424dcb38f74171bb7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.48", + "syn 2.0.57", "wasm-bindgen-backend", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-shared" -version = "0.2.89" +version = "0.2.92" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ab9b36309365056cd639da3134bf87fa8f3d86008abf99e612384a6eecd459f" +checksum = "af190c94f2773fdb3729c55b007a722abb5384da03bc0986df4c289bf5567e96" [[package]] name = "web-sys" -version = "0.3.66" +version = "0.3.69" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "50c24a44ec86bb68fbecd1b3efed7e85ea5621b39b35ef2766b66cd984f8010f" +checksum = "77afa9a11836342370f4817622a2f0f418b134426d91a82dfb48f532d2ec13ef" dependencies = [ "js-sys", "wasm-bindgen", @@ -3063,7 +3063,7 @@ version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "33ab640c8d7e35bf8ba19b884ba838ceb4fba93a4e8c65a9059d08afcfc683d9" dependencies = [ - "windows-targets 0.52.0", + "windows-targets 0.52.4", ] [[package]] @@ -3081,7 +3081,7 @@ version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "282be5f36a8ce781fad8c8ae18fa3f9beff57ec1b52cb3de0789201425d9a33d" dependencies = [ - "windows-targets 0.52.0", + "windows-targets 0.52.4", ] [[package]] @@ -3116,17 +3116,17 @@ dependencies = [ [[package]] name = "windows-targets" -version = "0.52.0" +version = "0.52.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a18201040b24831fbb9e4eb208f8892e1f50a37feb53cc7ff887feb8f50e7cd" +checksum = "7dd37b7e5ab9018759f893a1952c9420d060016fc19a472b4bb20d1bdd694d1b" dependencies = [ - "windows_aarch64_gnullvm 0.52.0", - "windows_aarch64_msvc 0.52.0", - "windows_i686_gnu 0.52.0", - "windows_i686_msvc 0.52.0", - "windows_x86_64_gnu 0.52.0", - "windows_x86_64_gnullvm 0.52.0", - "windows_x86_64_msvc 0.52.0", + "windows_aarch64_gnullvm 0.52.4", + "windows_aarch64_msvc 0.52.4", + "windows_i686_gnu 0.52.4", + "windows_i686_msvc 0.52.4", + "windows_x86_64_gnu 0.52.4", + "windows_x86_64_gnullvm 0.52.4", + "windows_x86_64_msvc 0.52.4", ] [[package]] @@ -3143,9 +3143,9 @@ checksum = "2b38e32f0abccf9987a4e3079dfb67dcd799fb61361e53e2882c3cbaf0d905d8" [[package]] name = "windows_aarch64_gnullvm" -version = "0.52.0" +version = "0.52.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cb7764e35d4db8a7921e09562a0304bf2f93e0a51bfccee0bd0bb0b666b015ea" +checksum = "bcf46cf4c365c6f2d1cc93ce535f2c8b244591df96ceee75d8e83deb70a9cac9" [[package]] name = "windows_aarch64_msvc" @@ -3161,9 +3161,9 @@ checksum = "dc35310971f3b2dbbf3f0690a219f40e2d9afcf64f9ab7cc1be722937c26b4bc" [[package]] name = "windows_aarch64_msvc" -version = "0.52.0" +version = "0.52.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bbaa0368d4f1d2aaefc55b6fcfee13f41544ddf36801e793edbbfd7d7df075ef" +checksum = "da9f259dd3bcf6990b55bffd094c4f7235817ba4ceebde8e6d11cd0c5633b675" [[package]] name = "windows_i686_gnu" @@ -3179,9 +3179,9 @@ checksum = "a75915e7def60c94dcef72200b9a8e58e5091744960da64ec734a6c6e9b3743e" [[package]] name = "windows_i686_gnu" -version = "0.52.0" +version = "0.52.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a28637cb1fa3560a16915793afb20081aba2c92ee8af57b4d5f28e4b3e7df313" +checksum = "b474d8268f99e0995f25b9f095bc7434632601028cf86590aea5c8a5cb7801d3" [[package]] name = "windows_i686_msvc" @@ -3197,9 +3197,9 @@ checksum = "8f55c233f70c4b27f66c523580f78f1004e8b5a8b659e05a4eb49d4166cca406" [[package]] name = "windows_i686_msvc" -version = "0.52.0" +version = "0.52.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ffe5e8e31046ce6230cc7215707b816e339ff4d4d67c65dffa206fd0f7aa7b9a" +checksum = "1515e9a29e5bed743cb4415a9ecf5dfca648ce85ee42e15873c3cd8610ff8e02" [[package]] name = "windows_x86_64_gnu" @@ -3215,9 +3215,9 @@ checksum = "53d40abd2583d23e4718fddf1ebec84dbff8381c07cae67ff7768bbf19c6718e" [[package]] name = "windows_x86_64_gnu" -version = "0.52.0" +version = "0.52.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3d6fa32db2bc4a2f5abeacf2b69f7992cd09dca97498da74a151a3132c26befd" +checksum = "5eee091590e89cc02ad514ffe3ead9eb6b660aedca2183455434b93546371a03" [[package]] name = "windows_x86_64_gnullvm" @@ -3233,9 +3233,9 @@ checksum = "0b7b52767868a23d5bab768e390dc5f5c55825b6d30b86c844ff2dc7414044cc" [[package]] name = "windows_x86_64_gnullvm" -version = "0.52.0" +version = "0.52.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1a657e1e9d3f514745a572a6846d3c7aa7dbe1658c056ed9c3344c4109a6949e" +checksum = "77ca79f2451b49fa9e2af39f0747fe999fcda4f5e241b2898624dca97a1f2177" [[package]] name = "windows_x86_64_msvc" @@ -3251,18 +3251,9 @@ checksum = "ed94fce61571a4006852b7389a063ab983c02eb1bb37b47f8272ce92d06d9538" [[package]] name = "windows_x86_64_msvc" -version = "0.52.0" +version = "0.52.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dff9641d1cd4be8d1a070daf9e3773c5f67e78b4d9d42263020c057706765c04" - -[[package]] -name = "yaml-rust" -version = "0.4.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56c1936c4cc7a1c9ab21a1ebb602eb942ba868cbd44a99cb7cdc5892335e1c85" -dependencies = [ - "linked-hash-map", -] +checksum = "32b752e52a2da0ddfbdbcc6fceadfeede4c939ed16d13e648833a61dfb611ed8" [[package]] name = "zerocopy" @@ -3281,7 +3272,7 @@ checksum = "9ce1b18ccd8e73a9321186f97e46f9f04b778851177567b1975109d26a08d2a6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.48", + "syn 2.0.57", ] [[package]] @@ -3305,9 +3296,9 @@ dependencies = [ [[package]] name = "zstd-sys" -version = "2.0.9+zstd.1.5.5" +version = "2.0.10+zstd.1.5.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e16efa8a874a0481a574084d34cc26fdb3b99627480f785888deb6386506656" +checksum = "c253a4914af5bafc8fa8c86ee400827e83cf6ec01195ec1f1ed8441bf00d65aa" dependencies = [ "cc", "pkg-config", diff --git a/core/Cargo.toml b/core/Cargo.toml index 4dc5afe6f..880d18d19 100644 --- a/core/Cargo.toml +++ b/core/Cargo.toml @@ -29,12 +29,12 @@ include = [ [dependencies] parquet-format = "4.0.0" # This must be kept in sync with that from parquet crate -arrow = { version = "~50.0.0", features = ["prettyprint", "ffi", "chrono-tz"] } -arrow-array = { version = "~50.0.0" } -arrow-data = { version = "~50.0.0" } -arrow-schema = { version = "~50.0.0" } -arrow-string = { version = "~50.0.0" } -parquet = { version = "~50.0.0", default-features = false, features = ["experimental"] } +arrow = { git = "https://github.com/viirya/arrow-rs.git", rev = "3f1ae0c", features = ["prettyprint", "ffi", "chrono-tz"] } +arrow-array = { git = "https://github.com/viirya/arrow-rs.git", rev = "3f1ae0c" } +arrow-data = { git = "https://github.com/viirya/arrow-rs.git", rev = "3f1ae0c" } +arrow-schema = { git = "https://github.com/viirya/arrow-rs.git", rev = "3f1ae0c" } +arrow-string = { git = "https://github.com/viirya/arrow-rs.git", rev = "3f1ae0c" } +parquet = { git = "https://github.com/viirya/arrow-rs.git", rev = "3f1ae0c", default-features = false, features = ["experimental"] } half = { version = "~2.1", default-features = false } futures = "0.3.28" mimalloc = { version = "*", default-features = false, optional = true } @@ -66,9 +66,10 @@ itertools = "0.11.0" chrono = { version = "0.4", default-features = false, features = ["clock"] } chrono-tz = { version = "0.8" } paste = "1.0.14" -datafusion-common = { version = "36.0.0" } -datafusion = { default-features = false, version = "36.0.0", features = ["unicode_expressions"] } -datafusion-physical-expr = { version = "36.0.0", default-features = false , features = ["unicode_expressions"] } +datafusion-common = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "111a940" } +datafusion = { default-features = false, git = "https://github.com/viirya/arrow-datafusion.git", rev = "111a940", features = ["unicode_expressions"] } +datafusion-functions = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "111a940" } +datafusion-physical-expr = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "111a940", default-features = false, features = ["unicode_expressions"] } unicode-segmentation = "^1.10.1" once_cell = "1.18.0" regex = "1.9.6" diff --git a/core/src/execution/datafusion/expressions/avg.rs b/core/src/execution/datafusion/expressions/avg.rs index 1e04ab0e9..e35ff6120 100644 --- a/core/src/execution/datafusion/expressions/avg.rs +++ b/core/src/execution/datafusion/expressions/avg.rs @@ -27,7 +27,7 @@ use arrow_schema::{DataType, Field}; use datafusion::logical_expr::{ type_coercion::aggregates::avg_return_type, Accumulator, EmitTo, GroupsAccumulator, }; -use datafusion_common::{not_impl_err, DataFusionError, Result, ScalarValue}; +use datafusion_common::{not_impl_err, Result, ScalarValue}; use datafusion_physical_expr::{expressions::format_state_name, AggregateExpr, PhysicalExpr}; use std::{any::Any, sync::Arc}; diff --git a/core/src/execution/datafusion/expressions/avg_decimal.rs b/core/src/execution/datafusion/expressions/avg_decimal.rs index d99ed041c..870e6d1a6 100644 --- a/core/src/execution/datafusion/expressions/avg_decimal.rs +++ b/core/src/execution/datafusion/expressions/avg_decimal.rs @@ -25,7 +25,7 @@ use arrow_array::{ }; use arrow_schema::{DataType, Field}; use datafusion::logical_expr::{Accumulator, EmitTo, GroupsAccumulator}; -use datafusion_common::{not_impl_err, DataFusionError, Result, ScalarValue}; +use datafusion_common::{not_impl_err, Result, ScalarValue}; use datafusion_physical_expr::{expressions::format_state_name, AggregateExpr, PhysicalExpr}; use std::{any::Any, sync::Arc}; diff --git a/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs b/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs index dd90cd8e9..6a4d07b89 100644 --- a/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs +++ b/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs @@ -22,7 +22,7 @@ use arrow::record_batch::RecordBatch; use arrow_array::cast::as_primitive_array; use arrow_schema::{DataType, Schema}; use datafusion::physical_plan::ColumnarValue; -use datafusion_common::{internal_err, DataFusionError, Result, ScalarValue}; +use datafusion_common::{internal_err, Result, ScalarValue}; use datafusion_physical_expr::{aggregate::utils::down_cast_any_ref, PhysicalExpr}; use std::{ any::Any, diff --git a/core/src/execution/datafusion/expressions/if_expr.rs b/core/src/execution/datafusion/expressions/if_expr.rs index 826f017c1..6f2ed6a54 100644 --- a/core/src/execution/datafusion/expressions/if_expr.rs +++ b/core/src/execution/datafusion/expressions/if_expr.rs @@ -112,8 +112,8 @@ impl PhysicalExpr for IfExpr { fn children(&self) -> Vec> { vec![ - self.true_expr.clone(), self.if_expr.clone(), + self.true_expr.clone(), self.false_expr.clone(), ] } @@ -218,4 +218,18 @@ mod tests { Ok(()) } + + #[test] + fn test_if_children() { + let if_expr = lit(true); + let true_expr = lit(123i32); + let false_expr = lit(999i32); + + let expr = if_fn(if_expr, true_expr, false_expr); + let children = expr.unwrap().children(); + assert_eq!(children.len(), 3); + assert_eq!(children[0].to_string(), "true"); + assert_eq!(children[1].to_string(), "123"); + assert_eq!(children[2].to_string(), "999"); + } } diff --git a/core/src/execution/datafusion/expressions/scalar_funcs.rs b/core/src/execution/datafusion/expressions/scalar_funcs.rs index 4fca7237b..e6f8de16b 100644 --- a/core/src/execution/datafusion/expressions/scalar_funcs.rs +++ b/core/src/execution/datafusion/expressions/scalar_funcs.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use std::{cmp::min, str::FromStr, sync::Arc}; +use std::{any::Any, cmp::min, fmt::Debug, str::FromStr, sync::Arc}; use arrow::{ array::{ @@ -27,16 +27,18 @@ use arrow::{ use arrow_array::{Array, ArrowNativeTypeOp, Decimal128Array}; use arrow_schema::DataType; use datafusion::{ - logical_expr::{BuiltinScalarFunction, ScalarFunctionImplementation}, + execution::FunctionRegistry, + logical_expr::{ + BuiltinScalarFunction, ScalarFunctionDefinition, ScalarFunctionImplementation, + ScalarUDFImpl, Signature, Volatility, + }, physical_plan::ColumnarValue, }; use datafusion_common::{ cast::as_generic_string_array, exec_err, internal_err, DataFusionError, Result as DataFusionResult, ScalarValue, }; -use datafusion_physical_expr::{ - execution_props::ExecutionProps, functions::create_physical_fun, math_expressions, -}; +use datafusion_physical_expr::{math_expressions, udf::ScalarUDF}; use num::{ integer::{div_ceil, div_floor}, BigInt, Signed, ToPrimitive, @@ -46,20 +48,94 @@ use unicode_segmentation::UnicodeSegmentation; /// Create a physical scalar function. pub fn create_comet_physical_fun( fun_name: &str, - execution_props: &ExecutionProps, data_type: DataType, -) -> Result { + registry: &dyn FunctionRegistry, +) -> Result { match fun_name { - "ceil" => Ok(Arc::new(move |x| spark_ceil(x, &data_type))), - "floor" => Ok(Arc::new(move |x| spark_floor(x, &data_type))), - "rpad" => Ok(Arc::new(spark_rpad)), - "round" => Ok(Arc::new(move |x| spark_round(x, &data_type))), - "unscaled_value" => Ok(Arc::new(spark_unscaled_value)), - "make_decimal" => Ok(Arc::new(move |x| spark_make_decimal(x, &data_type))), - "decimal_div" => Ok(Arc::new(move |x| spark_decimal_div(x, &data_type))), + "ceil" => { + let scalar_func = CometScalarFunction::new( + "ceil".to_string(), + Signature::variadic_any(Volatility::Immutable), + data_type.clone(), + Arc::new(move |args| spark_ceil(args, &data_type)), + ); + Ok(ScalarFunctionDefinition::UDF(Arc::new( + ScalarUDF::new_from_impl(scalar_func), + ))) + } + "floor" => { + let scalar_func = CometScalarFunction::new( + "floor".to_string(), + Signature::variadic_any(Volatility::Immutable), + data_type.clone(), + Arc::new(move |args| spark_floor(args, &data_type)), + ); + Ok(ScalarFunctionDefinition::UDF(Arc::new( + ScalarUDF::new_from_impl(scalar_func), + ))) + } + "rpad" => { + let scalar_func = CometScalarFunction::new( + "rpad".to_string(), + Signature::variadic_any(Volatility::Immutable), + data_type.clone(), + Arc::new(spark_rpad), + ); + Ok(ScalarFunctionDefinition::UDF(Arc::new( + ScalarUDF::new_from_impl(scalar_func), + ))) + } + "round" => { + let scalar_func = CometScalarFunction::new( + "round".to_string(), + Signature::variadic_any(Volatility::Immutable), + data_type.clone(), + Arc::new(move |args| spark_round(args, &data_type)), + ); + Ok(ScalarFunctionDefinition::UDF(Arc::new( + ScalarUDF::new_from_impl(scalar_func), + ))) + } + "unscaled_value" => { + let scalar_func = CometScalarFunction::new( + "unscaled_value".to_string(), + Signature::variadic_any(Volatility::Immutable), + data_type.clone(), + Arc::new(spark_unscaled_value), + ); + Ok(ScalarFunctionDefinition::UDF(Arc::new( + ScalarUDF::new_from_impl(scalar_func), + ))) + } + "make_decimal" => { + let scalar_func = CometScalarFunction::new( + "make_decimal".to_string(), + Signature::variadic_any(Volatility::Immutable), + data_type.clone(), + Arc::new(move |args| spark_make_decimal(args, &data_type)), + ); + Ok(ScalarFunctionDefinition::UDF(Arc::new( + ScalarUDF::new_from_impl(scalar_func), + ))) + } + "decimal_div" => { + let scalar_func = CometScalarFunction::new( + "decimal_div".to_string(), + Signature::variadic_any(Volatility::Immutable), + data_type.clone(), + Arc::new(move |args| spark_decimal_div(args, &data_type)), + ); + Ok(ScalarFunctionDefinition::UDF(Arc::new( + ScalarUDF::new_from_impl(scalar_func), + ))) + } _ => { - let fun = &BuiltinScalarFunction::from_str(fun_name)?; - create_physical_fun(fun, execution_props) + let fun = BuiltinScalarFunction::from_str(fun_name); + if fun.is_err() { + Ok(ScalarFunctionDefinition::UDF(registry.udf(fun_name)?)) + } else { + Ok(ScalarFunctionDefinition::BuiltIn(fun?)) + } } } } @@ -89,6 +165,61 @@ macro_rules! downcast_compute_op { }}; } +struct CometScalarFunction { + name: String, + signature: Signature, + data_type: DataType, + func: ScalarFunctionImplementation, +} + +impl Debug for CometScalarFunction { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("CometScalarFunction") + .field("name", &self.name) + .field("signature", &self.signature) + .field("data_type", &self.data_type) + .finish() + } +} + +impl CometScalarFunction { + fn new( + name: String, + signature: Signature, + data_type: DataType, + func: ScalarFunctionImplementation, + ) -> Self { + Self { + name, + signature, + data_type, + func, + } + } +} + +impl ScalarUDFImpl for CometScalarFunction { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + self.name.as_str() + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, _: &[DataType]) -> DataFusionResult { + Ok(self.data_type.clone()) + } + + fn invoke(&self, args: &[ColumnarValue]) -> DataFusionResult { + (self.func)(args) + } +} + /// `ceil` function that simulates Spark `ceil` expression pub fn spark_ceil( args: &[ColumnarValue], diff --git a/core/src/execution/datafusion/expressions/subquery.rs b/core/src/execution/datafusion/expressions/subquery.rs index 7cae12963..bf37cb895 100644 --- a/core/src/execution/datafusion/expressions/subquery.rs +++ b/core/src/execution/datafusion/expressions/subquery.rs @@ -18,7 +18,7 @@ use arrow_array::RecordBatch; use arrow_schema::{DataType, Schema, TimeUnit}; use datafusion::logical_expr::ColumnarValue; -use datafusion_common::{internal_err, DataFusionError, ScalarValue}; +use datafusion_common::{internal_err, ScalarValue}; use datafusion_physical_expr::PhysicalExpr; use jni::{ objects::JByteArray, diff --git a/core/src/execution/datafusion/expressions/temporal.rs b/core/src/execution/datafusion/expressions/temporal.rs index 5bdb533d0..4ae3c2605 100644 --- a/core/src/execution/datafusion/expressions/temporal.rs +++ b/core/src/execution/datafusion/expressions/temporal.rs @@ -23,7 +23,7 @@ use std::{ }; use arrow::{ - compute::{hour_dyn, minute_dyn, second_dyn}, + compute::{date_part, DatePart}, record_batch::RecordBatch, }; use arrow_schema::{DataType, Schema, TimeUnit::Microsecond}; @@ -101,7 +101,7 @@ impl PhysicalExpr for HourExec { Some(self.timezone.clone().into()), )), ); - let result = hour_dyn(&array)?; + let result = date_part(&array, DatePart::Hour)?; Ok(ColumnarValue::Array(result)) } @@ -195,7 +195,7 @@ impl PhysicalExpr for MinuteExec { Some(self.timezone.clone().into()), )), ); - let result = minute_dyn(&array)?; + let result = date_part(&array, DatePart::Minute)?; Ok(ColumnarValue::Array(result)) } @@ -289,7 +289,7 @@ impl PhysicalExpr for SecondExec { Some(self.timezone.clone().into()), )), ); - let result = second_dyn(&array)?; + let result = date_part(&array, DatePart::Second)?; Ok(ColumnarValue::Array(result)) } diff --git a/core/src/execution/datafusion/operators/expand.rs b/core/src/execution/datafusion/operators/expand.rs index e3f681b77..5cf444b3b 100644 --- a/core/src/execution/datafusion/operators/expand.rs +++ b/core/src/execution/datafusion/operators/expand.rs @@ -20,12 +20,12 @@ use arrow_schema::SchemaRef; use datafusion::{ execution::TaskContext, physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, - SendableRecordBatchStream, + DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, PlanProperties, + RecordBatchStream, SendableRecordBatchStream, }, }; use datafusion_common::DataFusionError; -use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; +use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr}; use futures::{Stream, StreamExt}; use std::{ any::Any, @@ -41,6 +41,7 @@ pub struct CometExpandExec { projections: Vec>>, child: Arc, schema: SchemaRef, + cache: PlanProperties, } impl CometExpandExec { @@ -50,10 +51,17 @@ impl CometExpandExec { child: Arc, schema: SchemaRef, ) -> Self { + let cache = PlanProperties::new( + EquivalenceProperties::new(schema.clone()), + Partitioning::UnknownPartitioning(1), + ExecutionMode::Bounded, + ); + Self { projections, child, schema, + cache, } } } @@ -88,14 +96,6 @@ impl ExecutionPlan for CometExpandExec { self.schema.clone() } - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(1) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None - } - fn children(&self) -> Vec> { vec![self.child.clone()] } @@ -122,6 +122,10 @@ impl ExecutionPlan for CometExpandExec { ExpandStream::new(self.projections.clone(), child_stream, self.schema.clone()); Ok(Box::pin(expand_stream)) } + + fn properties(&self) -> &PlanProperties { + &self.cache + } } pub struct ExpandStream { diff --git a/core/src/execution/datafusion/planner.rs b/core/src/execution/datafusion/planner.rs index 556735f78..4253e70eb 100644 --- a/core/src/execution/datafusion/planner.rs +++ b/core/src/execution/datafusion/planner.rs @@ -23,7 +23,11 @@ use arrow_schema::{DataType, Field, Schema, TimeUnit}; use datafusion::{ arrow::{compute::SortOptions, datatypes::SchemaRef}, common::DataFusionError, - logical_expr::{BuiltinScalarFunction, Operator as DataFusionOperator}, + execution::FunctionRegistry, + functions::math, + logical_expr::{ + BuiltinScalarFunction, Operator as DataFusionOperator, ScalarFunctionDefinition, + }, physical_expr::{ execution_props::ExecutionProps, expressions::{ @@ -31,7 +35,6 @@ use datafusion::{ FirstValue, InListExpr, IsNotNullExpr, IsNullExpr, LastValue, Literal as DataFusionLiteral, Max, Min, NegativeExpr, NotExpr, Sum, UnKnownColumn, }, - functions::create_physical_expr, AggregateExpr, PhysicalExpr, PhysicalSortExpr, ScalarFunctionExpr, }, physical_plan::{ @@ -43,9 +46,10 @@ use datafusion::{ sorts::sort::SortExec, ExecutionPlan, Partitioning, }, + prelude::SessionContext, }; use datafusion_common::{ - tree_node::{TreeNode, TreeNodeRewriter, VisitRecursion}, + tree_node::{Transformed, TransformedResult, TreeNode, TreeNodeRecursion, TreeNodeRewriter}, JoinType as DFJoinType, ScalarValue, }; use itertools::Itertools; @@ -108,20 +112,28 @@ pub struct PhysicalPlanner { // The execution context id of this planner. exec_context_id: i64, execution_props: ExecutionProps, + session_ctx: Arc, } impl Default for PhysicalPlanner { fn default() -> Self { - Self::new() + let session_ctx = Arc::new(SessionContext::new()); + let execution_props = ExecutionProps::new(); + Self { + exec_context_id: TEST_EXEC_CONTEXT_ID, + execution_props, + session_ctx, + } } } impl PhysicalPlanner { - pub fn new() -> Self { + pub fn new(session_ctx: Arc) -> Self { let execution_props = ExecutionProps::new(); Self { exec_context_id: TEST_EXEC_CONTEXT_ID, execution_props, + session_ctx, } } @@ -129,6 +141,7 @@ impl PhysicalPlanner { Self { exec_context_id, execution_props: self.execution_props, + session_ctx: self.session_ctx.clone(), } } @@ -465,14 +478,13 @@ impl PhysicalPlanner { } ExprStruct::Abs(expr) => { let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema.clone())?; + let return_type = child.data_type(&input_schema)?; let args = vec![child]; - let expr = create_physical_expr( - &BuiltinScalarFunction::Abs, - &args, - &input_schema, - &self.execution_props, - )?; - Ok(expr) + let scalar_def = ScalarFunctionDefinition::UDF(math::abs()); + + let expr = + ScalarFunctionExpr::new("abs", scalar_def, args, return_type, None, false); + Ok(Arc::new(expr)) } ExprStruct::CaseWhen(case_when) => { let when_then_pairs = case_when @@ -638,8 +650,8 @@ impl PhysicalPlanner { let data_type = return_type.map(to_arrow_datatype).unwrap(); let fun_expr = create_comet_physical_fun( "decimal_div", - &self.execution_props, data_type.clone(), + &self.session_ctx.state(), )?; Ok(Arc::new(ScalarFunctionExpr::new( "decimal_div", @@ -934,6 +946,7 @@ impl PhysicalPlanner { join_params.join_on, join_params.join_filter, &join_params.join_type, + None, PartitionMode::Partitioned, // null doesn't equal to null in Spark join key. If the join key is // `EqualNullSafe`, Spark will rewrite it during planning. @@ -1238,12 +1251,19 @@ impl PhysicalPlanner { // scalar function // Note this assumes the `fun_name` is a defined function in DF. Otherwise, it'll // throw error. - let fun = &BuiltinScalarFunction::from_str(fun_name)?; - fun.return_type(&input_expr_types)? + let fun = BuiltinScalarFunction::from_str(fun_name); + if fun.is_err() { + self.session_ctx + .udf(fun_name)? + .inner() + .return_type(&input_expr_types)? + } else { + fun?.return_type(&input_expr_types)? + } } }; let fun_expr = - create_comet_physical_fun(fun_name, &self.execution_props, data_type.clone())?; + create_comet_physical_fun(fun_name, data_type.clone(), &self.session_ctx.state())?; let scalar_expr: Arc = Arc::new(ScalarFunctionExpr::new( fun_name, @@ -1310,7 +1330,7 @@ fn expr_to_columns( right_field_indices.push(column.index() - left_field_len); } } - VisitRecursion::Continue + TreeNodeRecursion::Continue }) })?; @@ -1346,50 +1366,51 @@ impl JoinFilterRewriter<'_> { } impl TreeNodeRewriter for JoinFilterRewriter<'_> { - type N = Arc; - - fn mutate(&mut self, node: Self::N) -> datafusion_common::Result { - let new_expr: Arc = - if let Some(column) = node.as_any().downcast_ref::() { - if column.index() < self.left_field_len { - // left side - let new_index = self - .left_field_indices - .iter() - .position(|&x| x == column.index()) - .ok_or_else(|| { - DataFusionError::Internal(format!( - "Column index {} not found in left field indices", - column.index() - )) - })?; - Arc::new(Column::new(column.name(), new_index)) - } else if column.index() < self.left_field_len + self.right_field_len { - // right side - let new_index = self - .right_field_indices - .iter() - .position(|&x| x + self.left_field_len == column.index()) - .ok_or_else(|| { - DataFusionError::Internal(format!( - "Column index {} not found in right field indices", - column.index() - )) - })?; - Arc::new(Column::new( - column.name(), - new_index + self.left_field_indices.len(), - )) - } else { - return Err(DataFusionError::Internal(format!( - "Column index {} out of range", - column.index() - ))); - } + type Node = Arc; + + fn f_down(&mut self, node: Self::Node) -> datafusion_common::Result> { + if let Some(column) = node.as_any().downcast_ref::() { + if column.index() < self.left_field_len { + // left side + let new_index = self + .left_field_indices + .iter() + .position(|&x| x == column.index()) + .ok_or_else(|| { + DataFusionError::Internal(format!( + "Column index {} not found in left field indices", + column.index() + )) + })?; + Ok(Transformed::yes(Arc::new(Column::new( + column.name(), + new_index, + )))) + } else if column.index() < self.left_field_len + self.right_field_len { + // right side + let new_index = self + .right_field_indices + .iter() + .position(|&x| x + self.left_field_len == column.index()) + .ok_or_else(|| { + DataFusionError::Internal(format!( + "Column index {} not found in right field indices", + column.index() + )) + })?; + Ok(Transformed::yes(Arc::new(Column::new( + column.name(), + new_index + self.left_field_indices.len(), + )))) } else { - node.clone() - }; - Ok(new_expr) + return Err(DataFusionError::Internal(format!( + "Column index {} out of range", + column.index() + ))); + } + } else { + Ok(Transformed::no(node)) + } } } @@ -1410,7 +1431,7 @@ fn rewrite_physical_expr( right_field_indices, ); - Ok(expr.rewrite(&mut rewriter)?) + Ok(expr.rewrite(&mut rewriter).data()?) } #[cfg(test)] @@ -1447,7 +1468,7 @@ mod tests { }; let op = create_filter(op_scan, 3); - let planner = PhysicalPlanner::new(); + let planner = PhysicalPlanner::default(); let row_count = 100; // Create a dictionary array with 100 values, and use it as input to the execution. @@ -1527,7 +1548,7 @@ mod tests { }; let op = create_filter_literal(op_scan, STRING_TYPE_ID, lit); - let planner = PhysicalPlanner::new(); + let planner = PhysicalPlanner::default(); let row_count = 100; @@ -1605,7 +1626,7 @@ mod tests { }; let op = create_filter(op_scan, 0); - let planner = PhysicalPlanner::new(); + let planner = PhysicalPlanner::default(); let (mut scans, datafusion_plan) = planner.create_plan(&op, &mut vec![]).unwrap(); diff --git a/core/src/execution/datafusion/shuffle_writer.rs b/core/src/execution/datafusion/shuffle_writer.rs index f836e3a40..3b92abbde 100644 --- a/core/src/execution/datafusion/shuffle_writer.rs +++ b/core/src/execution/datafusion/shuffle_writer.rs @@ -47,13 +47,13 @@ use datafusion::{ runtime_env::RuntimeEnv, }, physical_plan::{ - expressions::PhysicalSortExpr, metrics::{BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}, stream::RecordBatchStreamAdapter, - DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, - SendableRecordBatchStream, Statistics, + DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, PlanProperties, + RecordBatchStream, SendableRecordBatchStream, Statistics, }, }; +use datafusion_physical_expr::EquivalenceProperties; use futures::{lock::Mutex, Stream, StreamExt, TryFutureExt, TryStreamExt}; use itertools::Itertools; use simd_adler32::Adler32; @@ -79,6 +79,7 @@ pub struct ShuffleWriterExec { output_index_file: String, /// Metrics metrics: ExecutionPlanMetricsSet, + cache: PlanProperties, } impl DisplayAs for ShuffleWriterExec { @@ -103,14 +104,6 @@ impl ExecutionPlan for ShuffleWriterExec { self.input.schema() } - fn output_partitioning(&self) -> Partitioning { - self.partitioning.clone() - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None - } - fn children(&self) -> Vec> { vec![self.input.clone()] } @@ -163,6 +156,10 @@ impl ExecutionPlan for ShuffleWriterExec { fn statistics(&self) -> Result { self.input.statistics() } + + fn properties(&self) -> &PlanProperties { + &self.cache + } } impl ShuffleWriterExec { @@ -173,12 +170,19 @@ impl ShuffleWriterExec { output_data_file: String, output_index_file: String, ) -> Result { + let cache = PlanProperties::new( + EquivalenceProperties::new(input.schema().clone()), + partitioning.clone(), + ExecutionMode::Bounded, + ); + Ok(ShuffleWriterExec { input, partitioning, metrics: ExecutionPlanMetricsSet::new(), output_data_file, output_index_file, + cache, }) } } diff --git a/core/src/execution/datafusion/spark_hash.rs b/core/src/execution/datafusion/spark_hash.rs index 1d8d1f2c9..aa4269dd0 100644 --- a/core/src/execution/datafusion/spark_hash.rs +++ b/core/src/execution/datafusion/spark_hash.rs @@ -165,7 +165,6 @@ macro_rules! hash_array_primitive_float { } else { *hash = spark_compatible_murmur3_hash((*value as $ty).to_le_bytes(), *hash); } - *hash = spark_compatible_murmur3_hash((*value as $ty).to_le_bytes(), *hash); } } } @@ -364,107 +363,154 @@ mod tests { use crate::execution::datafusion::spark_hash::{create_hashes, pmod}; use datafusion::arrow::array::{ArrayRef, Int32Array, Int64Array, Int8Array, StringArray}; + macro_rules! test_hashes { + ($ty:ty, $values:expr, $expected:expr) => { + let i = Arc::new(<$ty>::from($values)) as ArrayRef; + let mut hashes = vec![42; $values.len()]; + create_hashes(&[i], &mut hashes).unwrap(); + assert_eq!(hashes, $expected); + }; + } + #[test] fn test_i8() { - let i = Arc::new(Int8Array::from(vec![ - Some(1), - Some(0), - Some(-1), - Some(i8::MAX), - Some(i8::MIN), - ])) as ArrayRef; - let mut hashes = vec![42; 5]; - create_hashes(&[i], &mut hashes).unwrap(); - - // generated with Spark Murmur3_x86_32 - let expected = vec![0xdea578e3, 0x379fae8f, 0xa0590e3d, 0x43b4d8ed, 0x422a1365]; - assert_eq!(hashes, expected); + test_hashes!( + Int8Array, + vec![Some(1), Some(0), Some(-1), Some(i8::MAX), Some(i8::MIN)], + vec![0xdea578e3, 0x379fae8f, 0xa0590e3d, 0x43b4d8ed, 0x422a1365] + ); + // with null input + test_hashes!( + Int8Array, + vec![Some(1), None, Some(-1), Some(i8::MAX), Some(i8::MIN)], + vec![0xdea578e3, 42, 0xa0590e3d, 0x43b4d8ed, 0x422a1365] + ); } #[test] fn test_i32() { - let i = Arc::new(Int32Array::from(vec![ - Some(1), - Some(0), - Some(-1), - Some(i32::MAX), - Some(i32::MIN), - ])) as ArrayRef; - let mut hashes = vec![42; 5]; - create_hashes(&[i], &mut hashes).unwrap(); - - // generated with Spark Murmur3_x86_32 - let expected = vec![0xdea578e3, 0x379fae8f, 0xa0590e3d, 0x07fb67e7, 0x2b1f0fc6]; - assert_eq!(hashes, expected); + test_hashes!( + Int32Array, + vec![Some(1), Some(0), Some(-1), Some(i32::MAX), Some(i32::MIN)], + vec![0xdea578e3, 0x379fae8f, 0xa0590e3d, 0x07fb67e7, 0x2b1f0fc6] + ); + // with null input + test_hashes!( + Int32Array, + vec![ + Some(1), + Some(0), + Some(-1), + None, + Some(i32::MAX), + Some(i32::MIN) + ], + vec![0xdea578e3, 0x379fae8f, 0xa0590e3d, 42, 0x07fb67e7, 0x2b1f0fc6] + ); } #[test] fn test_i64() { - let i = Arc::new(Int64Array::from(vec![ - Some(1), - Some(0), - Some(-1), - Some(i64::MAX), - Some(i64::MIN), - ])) as ArrayRef; - let mut hashes = vec![42; 5]; - create_hashes(&[i], &mut hashes).unwrap(); - - // generated with Spark Murmur3_x86_32 - let expected = vec![0x99f0149d, 0x9c67b85d, 0xc8008529, 0xa05b5d7b, 0xcd1e64fb]; - assert_eq!(hashes, expected); + test_hashes!( + Int64Array, + vec![Some(1), Some(0), Some(-1), Some(i64::MAX), Some(i64::MIN)], + vec![0x99f0149d, 0x9c67b85d, 0xc8008529, 0xa05b5d7b, 0xcd1e64fb] + ); + // with null input + test_hashes!( + Int64Array, + vec![ + Some(1), + Some(0), + Some(-1), + None, + Some(i64::MAX), + Some(i64::MIN) + ], + vec![0x99f0149d, 0x9c67b85d, 0xc8008529, 42, 0xa05b5d7b, 0xcd1e64fb] + ); } #[test] fn test_f32() { - let i = Arc::new(Float32Array::from(vec![ - Some(1.0), - Some(0.0), - Some(-0.0), - Some(-1.0), - Some(99999999999.99999999999), - Some(-99999999999.99999999999), - ])) as ArrayRef; - let mut hashes = vec![42; 6]; - create_hashes(&[i], &mut hashes).unwrap(); - - // generated with Spark Murmur3_x86_32 - let expected = vec![ - 0xe434cc39, 0x379fae8f, 0x379fae8f, 0xdc0da8eb, 0xcbdc340f, 0xc0361c86, - ]; - assert_eq!(hashes, expected); + test_hashes!( + Float32Array, + vec![ + Some(1.0), + Some(0.0), + Some(-0.0), + Some(-1.0), + Some(99999999999.99999999999), + Some(-99999999999.99999999999), + ], + vec![0xe434cc39, 0x379fae8f, 0x379fae8f, 0xdc0da8eb, 0xcbdc340f, 0xc0361c86] + ); + // with null input + test_hashes!( + Float32Array, + vec![ + Some(1.0), + Some(0.0), + Some(-0.0), + Some(-1.0), + None, + Some(99999999999.99999999999), + Some(-99999999999.99999999999) + ], + vec![0xe434cc39, 0x379fae8f, 0x379fae8f, 0xdc0da8eb, 42, 0xcbdc340f, 0xc0361c86] + ); } #[test] fn test_f64() { - let i = Arc::new(Float64Array::from(vec![ - Some(1.0), - Some(0.0), - Some(-0.0), - Some(-1.0), - Some(99999999999.99999999999), - Some(-99999999999.99999999999), - ])) as ArrayRef; - let mut hashes = vec![42; 6]; - create_hashes(&[i], &mut hashes).unwrap(); - - // generated with Spark Murmur3_x86_32 - let expected = vec![ - 0xe4876492, 0x9c67b85d, 0x9c67b85d, 0x13d81357, 0xb87e1595, 0xa0eef9f9, - ]; - assert_eq!(hashes, expected); + test_hashes!( + Float64Array, + vec![ + Some(1.0), + Some(0.0), + Some(-0.0), + Some(-1.0), + Some(99999999999.99999999999), + Some(-99999999999.99999999999), + ], + vec![0xe4876492, 0x9c67b85d, 0x9c67b85d, 0x13d81357, 0xb87e1595, 0xa0eef9f9] + ); + // with null input + test_hashes!( + Float64Array, + vec![ + Some(1.0), + Some(0.0), + Some(-0.0), + Some(-1.0), + None, + Some(99999999999.99999999999), + Some(-99999999999.99999999999) + ], + vec![0xe4876492, 0x9c67b85d, 0x9c67b85d, 0x13d81357, 42, 0xb87e1595, 0xa0eef9f9] + ); } #[test] fn test_str() { - let i = Arc::new(StringArray::from(vec!["hello", "bar", "", "😁", "天地"])); - let mut hashes = vec![42; 5]; - create_hashes(&[i], &mut hashes).unwrap(); - - // generated with Murmur3Hash(Seq(Literal("")), 42).eval() since Spark is tested against - // this as well - let expected = vec![3286402344, 2486176763, 142593372, 885025535, 2395000894]; - assert_eq!(hashes, expected); + test_hashes!( + StringArray, + vec!["hello", "bar", "", "😁", "天地"], + vec![3286402344, 2486176763, 142593372, 885025535, 2395000894] + ); + // test with null input + test_hashes!( + StringArray, + vec![ + Some("hello"), + Some("bar"), + None, + Some(""), + Some("😁"), + Some("天地") + ], + vec![3286402344, 2486176763, 42, 142593372, 885025535, 2395000894] + ); } #[test] diff --git a/core/src/execution/jni_api.rs b/core/src/execution/jni_api.rs index 20f98a3a4..8249097a1 100644 --- a/core/src/execution/jni_api.rs +++ b/core/src/execution/jni_api.rs @@ -321,7 +321,8 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlan( // Because we don't know if input arrays are dictionary-encoded when we create // query plan, we need to defer stream initialization to first time execution. if exec_context.root_op.is_none() { - let planner = PhysicalPlanner::new().with_exec_id(exec_context_id); + let planner = PhysicalPlanner::new(exec_context.session_ctx.clone()) + .with_exec_id(exec_context_id); let (scans, root_op) = planner.create_plan( &exec_context.spark_plan, &mut exec_context.input_sources.clone(), diff --git a/core/src/execution/operators/copy.rs b/core/src/execution/operators/copy.rs index 699ccf7ae..292271f9e 100644 --- a/core/src/execution/operators/copy.rs +++ b/core/src/execution/operators/copy.rs @@ -41,6 +41,7 @@ use super::copy_or_cast_array; pub struct CopyExec { input: Arc, schema: SchemaRef, + cache: PlanProperties, } impl CopyExec { @@ -59,7 +60,17 @@ impl CopyExec { let schema = Arc::new(Schema::new(fields)); - Self { input, schema } + let cache = PlanProperties::new( + EquivalenceProperties::new(schema.clone()), + Partitioning::UnknownPartitioning(1), + ExecutionMode::Bounded, + ); + + Self { + input, + schema, + cache, + } } } @@ -82,14 +93,6 @@ impl ExecutionPlan for CopyExec { self.schema.clone() } - fn output_partitioning(&self) -> Partitioning { - self.input.output_partitioning() - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.input.output_ordering() - } - fn children(&self) -> Vec> { vec![self.input.clone()] } @@ -103,6 +106,7 @@ impl ExecutionPlan for CopyExec { Ok(Arc::new(CopyExec { input: new_input, schema: self.schema.clone(), + cache: self.cache.clone(), })) } @@ -118,6 +122,10 @@ impl ExecutionPlan for CopyExec { fn statistics(&self) -> DataFusionResult { self.input.statistics() } + + fn properties(&self) -> &PlanProperties { + &self.cache + } } struct CopyStream { diff --git a/core/src/execution/operators/scan.rs b/core/src/execution/operators/scan.rs index e31230c58..99c7c8391 100644 --- a/core/src/execution/operators/scan.rs +++ b/core/src/execution/operators/scan.rs @@ -61,6 +61,7 @@ pub struct ScanExec { /// The input batch of input data. Used to determine the schema of the input data. /// It is also used in unit test to mock the input data from JVM. pub batch: Arc>>, + cache: PlanProperties, } impl ScanExec { @@ -76,11 +77,20 @@ impl ScanExec { InputBatch::EOF }; + let schema = scan_schema(&first_batch, &data_types); + + let cache = PlanProperties::new( + EquivalenceProperties::new(schema), + Partitioning::UnknownPartitioning(1), + ExecutionMode::Bounded, + ); + Ok(Self { exec_context_id, input_source, data_types, batch: Arc::new(Mutex::new(Some(first_batch))), + cache, }) } @@ -197,6 +207,34 @@ impl ScanExec { } } +fn scan_schema(input_batch: &InputBatch, data_types: &[DataType]) -> SchemaRef { + let fields = match input_batch { + // Note that if `columns` is empty, we'll get an empty schema + InputBatch::Batch(columns, _) => { + columns + .iter() + .enumerate() + .map(|(idx, c)| { + let datatype = ScanExec::unpack_dictionary_type(c.data_type()); + // We don't use the field name. Put a placeholder. + if matches!(datatype, DataType::Dictionary(_, _)) { + Field::new_dict(format!("col_{}", idx), datatype, true, idx as i64, false) + } else { + Field::new(format!("col_{}", idx), datatype, true) + } + }) + .collect::>() + } + _ => data_types + .iter() + .enumerate() + .map(|(idx, dt)| Field::new(format!("col_{}", idx), dt.clone(), true)) + .collect(), + }; + + Arc::new(Schema::new(fields)) +} + impl ExecutionPlan for ScanExec { fn as_any(&self) -> &dyn Any { self @@ -207,47 +245,7 @@ impl ExecutionPlan for ScanExec { // Spark plan to DataFusion plan. At the moment, `batch` is not EOF. let binding = self.batch.try_lock().unwrap(); let input_batch = binding.as_ref().unwrap(); - - let fields = match input_batch { - // Note that if `columns` is empty, we'll get an empty schema - InputBatch::Batch(columns, _) => { - columns - .iter() - .enumerate() - .map(|(idx, c)| { - let datatype = Self::unpack_dictionary_type(c.data_type()); - // We don't use the field name. Put a placeholder. - if matches!(datatype, DataType::Dictionary(_, _)) { - Field::new_dict( - format!("col_{}", idx), - datatype, - true, - idx as i64, - false, - ) - } else { - Field::new(format!("col_{}", idx), datatype, true) - } - }) - .collect::>() - } - _ => self - .data_types - .iter() - .enumerate() - .map(|(idx, dt)| Field::new(format!("col_{}", idx), dt.clone(), true)) - .collect(), - }; - - Arc::new(Schema::new(fields)) - } - - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(1) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None + scan_schema(input_batch, &self.data_types) } fn children(&self) -> Vec> { @@ -268,6 +266,10 @@ impl ExecutionPlan for ScanExec { ) -> datafusion::common::Result { Ok(Box::pin(ScanStream::new(self.clone(), self.schema()))) } + + fn properties(&self) -> &PlanProperties { + &self.cache + } } impl DisplayAs for ScanExec { diff --git a/core/src/parquet/util/jni.rs b/core/src/parquet/util/jni.rs index 225abfc03..62787213f 100644 --- a/core/src/parquet/util/jni.rs +++ b/core/src/parquet/util/jni.rs @@ -82,6 +82,7 @@ pub fn convert_encoding(ordinal: jint) -> Encoding { match ordinal { 0 => Encoding::PLAIN, 1 => Encoding::RLE, + #[allow(deprecated)] 3 => Encoding::BIT_PACKED, 4 => Encoding::PLAIN_DICTIONARY, 5 => Encoding::DELTA_BINARY_PACKED, diff --git a/dev/diffs/3.4.2.diff b/dev/diffs/3.4.2.diff index 98c5f42b1..e6de19e05 100644 --- a/dev/diffs/3.4.2.diff +++ b/dev/diffs/3.4.2.diff @@ -249,7 +249,7 @@ index 9ddb4abe98b..1bebe99f1cc 100644 sql( """ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala -index f33432ddb6f..fe9f74ff8f1 100644 +index f33432ddb6f..6160c8d241a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen @@ -270,7 +270,26 @@ index f33432ddb6f..fe9f74ff8f1 100644 case _ => Nil } } -@@ -1729,6 +1733,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1238,7 +1242,8 @@ abstract class DynamicPartitionPruningSuiteBase + } + } + +- test("Plan broadcast pruning only when the broadcast can be reused") { ++ test("Plan broadcast pruning only when the broadcast can be reused", ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { + Given("dynamic pruning filter on the build side") + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + val df = sql( +@@ -1485,7 +1490,7 @@ abstract class DynamicPartitionPruningSuiteBase + } + + test("SPARK-38148: Do not add dynamic partition pruning if there exists static partition " + +- "pruning") { ++ "pruning", IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { + Seq( + "f.store_id = 1" -> false, +@@ -1729,6 +1734,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat case s: BatchScanExec => // we use f1 col for v2 tables due to schema pruning s.output.exists(_.exists(_.argString(maxFields = 100).contains("f1"))) 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 d1593f725..309fcaf69 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 @@ -36,7 +36,6 @@ import org.apache.spark.SparkConf; import org.apache.spark.TaskContext; import org.apache.spark.executor.ShuffleWriteMetrics; -import org.apache.spark.internal.config.package$; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.serializer.SerializationStream; import org.apache.spark.serializer.SerializerInstance; @@ -102,7 +101,7 @@ public final class CometDiskBlockWriter { private final File file; private long totalWritten = 0L; private boolean initialized = false; - private final int initialBufferSize; + private final int columnarBatchSize; private final boolean isAsync; private final int asyncThreadNum; private final ExecutorService threadPool; @@ -152,8 +151,7 @@ public final class CometDiskBlockWriter { this.asyncThreadNum = asyncThreadNum; this.threadPool = threadPool; - this.initialBufferSize = - (int) (long) conf.get(package$.MODULE$.SHUFFLE_SORT_INIT_BUFFER_SIZE()); + this.columnarBatchSize = (int) CometConf$.MODULE$.COMET_COLUMNAR_SHUFFLE_BATCH_SIZE().get(); this.numElementsForSpillThreshold = (int) CometConf$.MODULE$.COMET_EXEC_SHUFFLE_SPILL_THRESHOLD().get(); @@ -264,10 +262,11 @@ public void insertRow(UnsafeRow row, int partitionId) throws IOException { // While proceeding with possible spilling and inserting the record, we need to synchronize // it, because other threads may be spilling this writer at the same time. synchronized (CometDiskBlockWriter.this) { - if (activeWriter.numRecords() >= numElementsForSpillThreshold) { + if (activeWriter.numRecords() >= numElementsForSpillThreshold + || activeWriter.numRecords() >= columnarBatchSize) { + int threshold = Math.min(numElementsForSpillThreshold, columnarBatchSize); logger.info( - "Spilling data because number of spilledRecords crossed the threshold " - + numElementsForSpillThreshold); + "Spilling data because number of spilledRecords crossed the threshold " + threshold); // Spill the current writer doSpill(false); if (activeWriter.numRecords() != 0) { @@ -347,7 +346,7 @@ class ArrowIPCWriter extends SpillWriter { private final RowPartition rowPartition; ArrowIPCWriter() { - rowPartition = new RowPartition(initialBufferSize); + rowPartition = new RowPartition(columnarBatchSize); this.allocatedPages = new LinkedList<>(); this.allocator = CometDiskBlockWriter.this.allocator; diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index fcbf42f5b..a10ac573e 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.comet._ import org.apache.spark.sql.comet.execution.shuffle.{CometColumnarShuffle, CometNativeShuffle} import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec +import org.apache.spark.sql.comet.util.Utils import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{BroadcastQueryStageExec, ShuffleQueryStageExec} import org.apache.spark.sql.execution.aggregate.HashAggregateExec @@ -43,7 +44,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.comet.CometConf._ -import org.apache.comet.CometSparkSessionExtensions.{isANSIEnabled, isCometBroadCastEnabled, isCometColumnarShuffleEnabled, isCometEnabled, isCometExecEnabled, isCometOperatorEnabled, isCometScan, isCometScanEnabled, isCometShuffleEnabled, isSchemaSupported} +import org.apache.comet.CometSparkSessionExtensions.{isANSIEnabled, isCometBroadCastForceEnabled, isCometColumnarShuffleEnabled, isCometEnabled, isCometExecEnabled, isCometOperatorEnabled, isCometScan, isCometScanEnabled, isCometShuffleEnabled, isSchemaSupported, shouldApplyRowToColumnar} import org.apache.comet.parquet.{CometParquetScan, SupportsComet} import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.QueryPlanSerde @@ -68,7 +69,7 @@ class CometSparkSessionExtensions override def preColumnarTransitions: Rule[SparkPlan] = CometExecRule(session) override def postColumnarTransitions: Rule[SparkPlan] = - EliminateRedundantColumnarToRow(session) + EliminateRedundantTransitions(session) } case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] { @@ -238,6 +239,11 @@ class CometSparkSessionExtensions val nativeOp = QueryPlanSerde.operator2Proto(op).get CometScanWrapper(nativeOp, op) + case op if shouldApplyRowToColumnar(conf, op) => + val cometOp = CometRowToColumnarExec(op) + val nativeOp = QueryPlanSerde.operator2Proto(cometOp).get + CometScanWrapper(nativeOp, cometOp) + case op: ProjectExec => val newOp = transform1(op) newOp match { @@ -442,14 +448,32 @@ class CometSparkSessionExtensions s } - case b: BroadcastExchangeExec - if isCometNative(b.child) && isCometOperatorEnabled(conf, "broadcastExchangeExec") && - isCometBroadCastEnabled(conf) => - QueryPlanSerde.operator2Proto(b) match { - case Some(nativeOp) => - val cometOp = CometBroadcastExchangeExec(b, b.child) - CometSinkPlaceHolder(nativeOp, b, cometOp) - case None => b + // `CometBroadcastExchangeExec`'s broadcast output is not compatible with Spark's broadcast + // exchange. It is only used for Comet native execution. We only transform Spark broadcast + // exchange to Comet broadcast exchange if its downstream is a Comet native plan or if the + // broadcast exchange is forced to be enabled by Comet config. + case plan if plan.children.exists(_.isInstanceOf[BroadcastExchangeExec]) => + val newChildren = plan.children.map { + case b: BroadcastExchangeExec + if isCometNative(b.child) && + isCometOperatorEnabled(conf, "broadcastExchangeExec") => + QueryPlanSerde.operator2Proto(b) match { + case Some(nativeOp) => + val cometOp = CometBroadcastExchangeExec(b, b.child) + CometSinkPlaceHolder(nativeOp, b, cometOp) + case None => b + } + case other => other + } + if (!newChildren.exists(_.isInstanceOf[BroadcastExchangeExec])) { + val newPlan = apply(plan.withNewChildren(newChildren)) + if (isCometNative(newPlan) || isCometBroadCastForceEnabled(conf)) { + newPlan + } else { + plan + } + } else { + plan } // For AQE shuffle stage on a Comet shuffle exchange @@ -574,18 +598,32 @@ class CometSparkSessionExtensions } } - // CometExec already wraps a `ColumnarToRowExec` for row-based operators. Therefore, - // `ColumnarToRowExec` is redundant and can be eliminated. + // This rule is responsible for eliminating redundant transitions between row-based and + // columnar-based operators for Comet. Currently, two potential redundant transitions are: + // 1. `ColumnarToRowExec` on top of an ending `CometCollectLimitExec` operator, which is + // redundant as `CometCollectLimitExec` already wraps a `ColumnarToRowExec` for row-based + // output. + // 2. Consecutive operators of `CometRowToColumnarExec` and `ColumnarToRowExec`. // - // It was added during ApplyColumnarRulesAndInsertTransitions' insertTransitions phase when Spark - // requests row-based output such as `collect` call. It's correct to add a redundant - // `ColumnarToRowExec` for `CometExec`. However, for certain operators such as - // `CometCollectLimitExec` which overrides `executeCollect`, the redundant `ColumnarToRowExec` - // makes the override ineffective. The purpose of this rule is to eliminate the redundant - // `ColumnarToRowExec` for such operators. - case class EliminateRedundantColumnarToRow(session: SparkSession) extends Rule[SparkPlan] { + // Note about the first case: The `ColumnarToRowExec` was added during + // ApplyColumnarRulesAndInsertTransitions' insertTransitions phase when Spark requests row-based + // output such as a `collect` call. It's correct to add a redundant `ColumnarToRowExec` for + // `CometExec`. However, for certain operators such as `CometCollectLimitExec` which overrides + // `executeCollect`, the redundant `ColumnarToRowExec` makes the override ineffective. + // + // Note about the second case: When `spark.comet.rowToColumnar.enabled` is set, Comet will add + // `CometRowToColumnarExec` on top of row-based operators first, but the downstream operator + // only takes row-based input as it's a vanilla Spark operator(as Comet cannot convert it for + // various reasons) or Spark requests row-based output such as a `collect` call. Spark will adds + // another `ColumnarToRowExec` on top of `CometRowToColumnarExec`. In this case, the pair could + // be removed. + case class EliminateRedundantTransitions(session: SparkSession) extends Rule[SparkPlan] { override def apply(plan: SparkPlan): SparkPlan = { - plan match { + val eliminatedPlan = plan transformUp { + case ColumnarToRowExec(rowToColumnar: CometRowToColumnarExec) => rowToColumnar.child + } + + eliminatedPlan match { case ColumnarToRowExec(child: CometCollectLimitExec) => child case other => @@ -651,8 +689,8 @@ object CometSparkSessionExtensions extends Logging { !conf.getConfString(operatorDisabledFlag, "false").toBoolean } - private[comet] def isCometBroadCastEnabled(conf: SQLConf): Boolean = { - COMET_EXEC_BROADCAST_ENABLED.get(conf) + private[comet] def isCometBroadCastForceEnabled(conf: SQLConf): Boolean = { + COMET_EXEC_BROADCAST_FORCE_ENABLED.get(conf) } private[comet] def isCometShuffleEnabled(conf: SQLConf): Boolean = @@ -698,6 +736,18 @@ object CometSparkSessionExtensions extends Logging { op.isInstanceOf[CometBatchScanExec] || op.isInstanceOf[CometScanExec] } + private def shouldApplyRowToColumnar(conf: SQLConf, op: SparkPlan): Boolean = { + // Only consider converting leaf nodes to columnar currently, so that all the following + // operators can have a chance to be converted to columnar. + // TODO: consider converting other intermediate operators to columnar. + op.isInstanceOf[LeafExecNode] && !op.supportsColumnar && isSchemaSupported(op.schema) && + COMET_ROW_TO_COLUMNAR_ENABLED.get(conf) && { + val simpleClassName = Utils.getSimpleName(op.getClass) + val nodeName = simpleClassName.replaceAll("Exec$", "") + COMET_ROW_TO_COLUMNAR_SUPPORTED_OPERATOR_LIST.get(conf).contains(nodeName) + } + } + /** Used for operations that weren't available in Spark 3.2 */ def isSpark32: Boolean = { org.apache.spark.SPARK_VERSION.matches("3\\.2\\..*") 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 cc5a32245..ca5a2cfd7 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.optimizer.{BuildRight, NormalizeNaNAndZero} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, SinglePartition} import org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils -import org.apache.spark.sql.comet.{CometBroadcastExchangeExec, CometSinkPlaceHolder, DecimalPrecision} +import org.apache.spark.sql.comet.{CometBroadcastExchangeExec, CometRowToColumnarExec, CometSinkPlaceHolder, DecimalPrecision} import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec import org.apache.spark.sql.execution import org.apache.spark.sql.execution._ @@ -2101,6 +2101,7 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde { private def isCometSink(op: SparkPlan): Boolean = { op match { case s if isCometScan(s) => true + case _: CometRowToColumnarExec => true case _: CometSinkPlaceHolder => true case _: CoalesceExec => true case _: CollectLimitExec => true diff --git a/spark/src/main/scala/org/apache/spark/shuffle/sort/RowPartition.scala b/spark/src/main/scala/org/apache/spark/shuffle/sort/RowPartition.scala index 873e422fb..32d64fad4 100644 --- a/spark/src/main/scala/org/apache/spark/shuffle/sort/RowPartition.scala +++ b/spark/src/main/scala/org/apache/spark/shuffle/sort/RowPartition.scala @@ -22,8 +22,8 @@ package org.apache.spark.shuffle.sort import scala.collection.mutable.ArrayBuffer class RowPartition(initialSize: Int) { - private val rowAddresses: ArrayBuffer[Long] = new ArrayBuffer[Long](initialSize) - private val rowSizes: ArrayBuffer[Int] = new ArrayBuffer[Int](initialSize) + private var rowAddresses: ArrayBuffer[Long] = new ArrayBuffer[Long](initialSize) + private var rowSizes: ArrayBuffer[Int] = new ArrayBuffer[Int](initialSize) def addRow(addr: Long, size: Int): Unit = { rowAddresses += addr @@ -32,11 +32,20 @@ class RowPartition(initialSize: Int) { def getNumRows: Int = rowAddresses.size - def getRowAddresses: Array[Long] = rowAddresses.toArray - def getRowSizes: Array[Int] = rowSizes.toArray + def getRowAddresses: Array[Long] = { + val array = rowAddresses.toArray + rowAddresses = null + array + } + + def getRowSizes: Array[Int] = { + val array = rowSizes.toArray + rowSizes = null + array + } def reset(): Unit = { - rowAddresses.clear() - rowSizes.clear() + rowAddresses = new ArrayBuffer[Long](initialSize) + rowSizes = new ArrayBuffer[Int](initialSize) } } 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 24f9f3279..a8322be7d 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 @@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.plans.logical.Statistics 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} import org.apache.spark.sql.execution.exchange.BroadcastExchangeLike import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} @@ -42,6 +43,7 @@ import org.apache.spark.util.io.ChunkedByteBuffer import com.google.common.base.Objects +import org.apache.comet.CometRuntimeException import org.apache.comet.shims.ShimCometBroadcastExchangeExec /** @@ -95,6 +97,16 @@ case class CometBroadcastExchangeExec(originalPlan: SparkPlan, child: SparkPlan) @transient private lazy val maxBroadcastRows = 512000000 + private var numPartitions: Option[Int] = None + + def setNumPartitions(numPartitions: Int): CometBroadcastExchangeExec = { + this.numPartitions = Some(numPartitions) + this + } + def getNumPartitions(): Int = { + numPartitions.getOrElse(child.executeColumnar().getNumPartitions) + } + @transient override lazy val relationFuture: Future[broadcast.Broadcast[Any]] = { SQLExecution.withThreadLocalCaptured[broadcast.Broadcast[Any]]( @@ -108,7 +120,21 @@ case class CometBroadcastExchangeExec(originalPlan: SparkPlan, child: SparkPlan) interruptOnCancel = true) val beforeCollect = System.nanoTime() - val countsAndBytes = child.asInstanceOf[CometExec].getByteArrayRdd().collect() + val countsAndBytes = child match { + case c: CometPlan => CometExec.getByteArrayRdd(c).collect() + case AQEShuffleReadExec(s: ShuffleQueryStageExec, _) + if s.plan.isInstanceOf[CometPlan] => + CometExec.getByteArrayRdd(s.plan.asInstanceOf[CometPlan]).collect() + case AQEShuffleReadExec(s: ShuffleQueryStageExec, _) => + throw new CometRuntimeException( + "Child of CometBroadcastExchangeExec should be CometExec, " + + s"but got: ${s.plan.getClass}") + case _ => + throw new CometRuntimeException( + "Child of CometBroadcastExchangeExec should be CometExec, " + + s"but got: ${child.getClass}") + } + val numRows = countsAndBytes.map(_._1).sum val input = countsAndBytes.iterator.map(countAndBytes => countAndBytes._2) @@ -191,7 +217,7 @@ case class CometBroadcastExchangeExec(originalPlan: SparkPlan, child: SparkPlan) override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { val broadcasted = executeBroadcast[Array[ChunkedByteBuffer]]() - new CometBatchRDD(sparkContext, broadcasted.value.length, broadcasted) + new CometBatchRDD(sparkContext, getNumPartitions(), broadcasted) } override protected[sql] def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometRowToColumnarExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometRowToColumnarExec.scala new file mode 100644 index 000000000..5679e865c --- /dev/null +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometRowToColumnarExec.scala @@ -0,0 +1,84 @@ +/* + * 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 org.apache.spark.TaskContext +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder} +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.comet.execution.arrow.CometArrowConverters +import org.apache.spark.sql.execution.{RowToColumnarTransition, SparkPlan} +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.vectorized.ColumnarBatch + +case class CometRowToColumnarExec(child: SparkPlan) + extends RowToColumnarTransition + with CometPlan { + override def output: Seq[Attribute] = child.output + + override def outputPartitioning: Partitioning = child.outputPartitioning + + override def outputOrdering: Seq[SortOrder] = child.outputOrdering + + override protected def doExecute(): RDD[InternalRow] = { + child.execute() + } + + override def doExecuteBroadcast[T](): Broadcast[T] = { + child.executeBroadcast() + } + + override def supportsColumnar: Boolean = true + + override lazy val metrics: Map[String, SQLMetric] = Map( + "numInputRows" -> SQLMetrics.createMetric(sparkContext, "number of input rows"), + "numOutputBatches" -> SQLMetrics.createMetric(sparkContext, "number of output batches")) + + override def doExecuteColumnar(): RDD[ColumnarBatch] = { + val numInputRows = longMetric("numInputRows") + val numOutputBatches = longMetric("numOutputBatches") + val maxRecordsPerBatch = conf.arrowMaxRecordsPerBatch + val timeZoneId = conf.sessionLocalTimeZone + val schema = child.schema + + child + .execute() + .mapPartitionsInternal { iter => + val context = TaskContext.get() + CometArrowConverters.toArrowBatchIterator( + iter, + schema, + maxRecordsPerBatch, + timeZoneId, + context) + } + .map { batch => + numInputRows += batch.numRows() + numOutputBatches += 1 + batch + } + } + + override protected def withNewChildInternal(newChild: SparkPlan): CometRowToColumnarExec = + copy(child = newChild) + +} 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 84734a175..8545eee90 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,8 +19,7 @@ package org.apache.spark.sql.comet -import java.io.{ByteArrayOutputStream, DataInputStream, DataOutputStream} -import java.nio.ByteBuffer +import java.io.{ByteArrayOutputStream, DataInputStream} import java.nio.channels.Channels import scala.collection.mutable.ArrayBuffer @@ -35,19 +34,19 @@ import org.apache.spark.sql.catalyst.optimizer.BuildSide import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.comet.execution.shuffle.{ArrowReaderIterator, CometShuffleExchangeExec} +import org.apache.spark.sql.comet.util.Utils import org.apache.spark.sql.execution.{BinaryExecNode, ColumnarToRowExec, ExecSubqueryExpression, ExplainUtils, LeafExecNode, ScalarSubquery, SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.adaptive.{AQEShuffleReadExec, BroadcastQueryStageExec, ShuffleQueryStageExec} import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.util.io.{ChunkedByteBuffer, ChunkedByteBufferOutputStream} +import org.apache.spark.util.io.ChunkedByteBuffer import com.google.common.base.Objects import org.apache.comet.{CometConf, CometExecIterator, CometRuntimeException} import org.apache.comet.serde.OperatorOuterClass.Operator -import org.apache.comet.vector.NativeUtil /** * A Comet physical operator @@ -72,32 +71,11 @@ abstract class CometExec extends CometPlan { override def outputPartitioning: Partitioning = originalPlan.outputPartitioning - /** - * Executes this Comet operator and serialized output ColumnarBatch into bytes. - */ - def getByteArrayRdd(): RDD[(Long, ChunkedByteBuffer)] = { - executeColumnar().mapPartitionsInternal { iter => - val codec = CompressionCodec.createCodec(SparkEnv.get.conf) - val cbbos = new ChunkedByteBufferOutputStream(1024 * 1024, ByteBuffer.allocate) - val out = new DataOutputStream(codec.compressedOutputStream(cbbos)) - - val count = new NativeUtil().serializeBatches(iter, out) - - out.flush() - out.close() - if (out.size() > 0) { - Iterator((count, cbbos.toChunkedByteBuffer)) - } else { - Iterator((count, new ChunkedByteBuffer(Array.empty[ByteBuffer]))) - } - } - } - /** * Executes the Comet operator and returns the result as an iterator of ColumnarBatch. */ def executeColumnarCollectIterator(): (Long, Iterator[ColumnarBatch]) = { - val countsAndBytes = getByteArrayRdd().collect() + val countsAndBytes = CometExec.getByteArrayRdd(this).collect() val total = countsAndBytes.map(_._1).sum val rows = countsAndBytes.iterator .flatMap(countAndBytes => CometExec.decodeBatches(countAndBytes._2)) @@ -128,6 +106,15 @@ object CometExec { new CometExecIterator(newIterId, inputs, bytes, nativeMetrics) } + /** + * Executes this Comet operator and serialized output ColumnarBatch into bytes. + */ + def getByteArrayRdd(cometPlan: CometPlan): RDD[(Long, ChunkedByteBuffer)] = { + cometPlan.executeColumnar().mapPartitionsInternal { iter => + Utils.serializeBatches(iter) + } + } + /** * Decodes the byte arrays back to ColumnarBatchs and put them into buffer. */ @@ -235,12 +222,55 @@ abstract class CometNativeExec extends CometExec { // Collect the input ColumnarBatches from the child operators and create a CometExecIterator // to execute the native plan. + val sparkPlans = ArrayBuffer.empty[SparkPlan] val inputs = ArrayBuffer.empty[RDD[ColumnarBatch]] - foreachUntilCometInput(this)(inputs += _.executeColumnar()) + foreachUntilCometInput(this)(sparkPlans += _) + + // Find the first non broadcast plan + val firstNonBroadcastPlan = sparkPlans.zipWithIndex.find { + case (_: CometBroadcastExchangeExec, _) => false + case (BroadcastQueryStageExec(_, _: CometBroadcastExchangeExec, _), _) => false + case _ => true + } + + // If the first non broadcast plan is not found, it means all the plans are broadcast plans. + // This is not expected, so throw an exception. + if (firstNonBroadcastPlan.isEmpty) { + throw new CometRuntimeException(s"Cannot find the first non broadcast plan: $this") + } + + // If the first non broadcast plan is found, we need to adjust the partition number of + // the broadcast plans to make sure they have the same partition number as the first non + // broadcast plan. + val firstNonBroadcastPlanRDD = firstNonBroadcastPlan.get._1.executeColumnar() + val firstNonBroadcastPlanNumPartitions = firstNonBroadcastPlanRDD.getNumPartitions + + // Spark doesn't need to zip Broadcast RDDs, so it doesn't schedule Broadcast RDDs with + // same partition number. But for Comet, we need to zip them so we need to adjust the + // partition number of Broadcast RDDs to make sure they have the same partition number. + sparkPlans.zipWithIndex.foreach { case (plan, idx) => + plan match { + case c: CometBroadcastExchangeExec => + inputs += c.setNumPartitions(firstNonBroadcastPlanNumPartitions).executeColumnar() + case BroadcastQueryStageExec(_, c: CometBroadcastExchangeExec, _) => + inputs += c.setNumPartitions(firstNonBroadcastPlanNumPartitions).executeColumnar() + case _ if idx == firstNonBroadcastPlan.get._2 => + inputs += firstNonBroadcastPlanRDD + case _ => + val rdd = plan.executeColumnar() + if (rdd.getNumPartitions != firstNonBroadcastPlanNumPartitions) { + throw new CometRuntimeException( + s"Partition number mismatch: ${rdd.getNumPartitions} != " + + s"$firstNonBroadcastPlanNumPartitions") + } else { + inputs += rdd + } + } + } if (inputs.isEmpty) { - throw new CometRuntimeException(s"No input for CometNativeExec: $this") + throw new CometRuntimeException(s"No input for CometNativeExec:\n $this") } ZippedPartitionsRDD(sparkContext, inputs.toSeq)(createCometExecIter(_)) @@ -270,7 +300,8 @@ abstract class CometNativeExec extends CometExec { case _: CometScanExec | _: CometBatchScanExec | _: ShuffleQueryStageExec | _: AQEShuffleReadExec | _: CometShuffleExchangeExec | _: CometUnionExec | _: CometTakeOrderedAndProjectExec | _: CometCoalesceExec | _: ReusedExchangeExec | - _: CometBroadcastExchangeExec | _: BroadcastQueryStageExec => + _: CometBroadcastExchangeExec | _: BroadcastQueryStageExec | + _: CometRowToColumnarExec => func(plan) case _: CometPlan => // Other Comet operators, continue to traverse the tree. 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 fa45aa382..3e14838b8 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 @@ -1,81 +1,78 @@ == Physical Plan == -TakeOrderedAndProject (77) -+- * HashAggregate (76) - +- Exchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (24) - : +- * Filter (23) - : +- Window (22) - : +- * Sort (21) - : +- Window (20) - : +- * Sort (19) - : +- Exchange (18) - : +- * HashAggregate (17) - : +- Exchange (16) - : +- * HashAggregate (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildLeft (10) - : : :- BroadcastExchange (5) - : : : +- * ColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : +- * ColumnarToRow (9) - : : +- CometProject (8) - : : +- CometFilter (7) - : : +- CometScan parquet spark_catalog.default.web_returns (6) - : +- ReusedExchange (12) - :- * Project (48) - : +- * Filter (47) - : +- Window (46) - : +- * Sort (45) - : +- Window (44) - : +- * Sort (43) - : +- Exchange (42) - : +- * HashAggregate (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (35) - : : +- * BroadcastHashJoin Inner BuildLeft (34) - : : :- BroadcastExchange (29) - : : : +- * ColumnarToRow (28) - : : : +- CometProject (27) - : : : +- CometFilter (26) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (25) - : : +- * ColumnarToRow (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan parquet spark_catalog.default.catalog_returns (30) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * Sort (67) - +- Exchange (66) - +- * HashAggregate (65) - +- Exchange (64) - +- * HashAggregate (63) - +- * Project (62) - +- * BroadcastHashJoin Inner BuildRight (61) - :- * Project (59) - : +- * BroadcastHashJoin Inner BuildLeft (58) - : :- BroadcastExchange (53) - : : +- * ColumnarToRow (52) - : : +- CometProject (51) - : : +- CometFilter (50) - : : +- CometScan parquet spark_catalog.default.store_sales (49) - : +- * ColumnarToRow (57) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (60) +TakeOrderedAndProject (74) ++- * HashAggregate (73) + +- Exchange (72) + +- * HashAggregate (71) + +- Union (70) + :- * Project (23) + : +- * Filter (22) + : +- Window (21) + : +- * Sort (20) + : +- Window (19) + : +- * Sort (18) + : +- Exchange (17) + : +- * HashAggregate (16) + : +- Exchange (15) + : +- * HashAggregate (14) + : +- * Project (13) + : +- * BroadcastHashJoin Inner BuildRight (12) + : :- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.web_returns (5) + : +- ReusedExchange (11) + :- * Project (46) + : +- * Filter (45) + : +- Window (44) + : +- * Sort (43) + : +- Window (42) + : +- * Sort (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- Exchange (38) + : +- * HashAggregate (37) + : +- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * ColumnarToRow (33) + : : +- CometProject (32) + : : +- CometBroadcastHashJoin (31) + : : :- CometBroadcastExchange (27) + : : : +- CometProject (26) + : : : +- CometFilter (25) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (24) + : : +- CometProject (30) + : : +- CometFilter (29) + : : +- CometScan parquet spark_catalog.default.catalog_returns (28) + : +- ReusedExchange (34) + +- * Project (69) + +- * Filter (68) + +- Window (67) + +- * Sort (66) + +- Window (65) + +- * Sort (64) + +- Exchange (63) + +- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- * Project (59) + +- * BroadcastHashJoin Inner BuildRight (58) + :- * ColumnarToRow (56) + : +- CometProject (55) + : +- CometBroadcastHashJoin (54) + : :- CometBroadcastExchange (50) + : : +- CometProject (49) + : : +- CometFilter (48) + : : +- CometScan parquet spark_catalog.default.store_sales (47) + : +- CometProject (53) + : +- CometFilter (52) + : +- CometScan parquet spark_catalog.default.store_returns (51) + +- ReusedExchange (57) (unknown) Scan parquet spark_catalog.default.web_sales @@ -94,12 +91,9 @@ Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6], [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -(4) ColumnarToRow [codegen id : 1] +(4) CometBroadcastExchange Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(5) BroadcastExchange -Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=1] +Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] (unknown) Scan parquet spark_catalog.default.web_returns Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] @@ -108,83 +102,82 @@ Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) -(8) CometProject +(7) CometProject Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -(9) ColumnarToRow -Input [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] +(8) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] +Right output [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] +Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner -(10) BroadcastHashJoin [codegen id : 3] -Left keys [2]: [ws_order_number#2, ws_item_sk#1] -Right keys [2]: [wr_order_number#9, wr_item_sk#8] -Join type: Inner -Join condition: None - -(11) Project [codegen id : 3] -Output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] +(9) CometProject Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] +Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] + +(10) ColumnarToRow [codegen id : 2] +Input [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -(12) ReusedExchange [Reuses operator id: 82] +(11) ReusedExchange [Reuses operator id: 79] Output [1]: [d_date_sk#13] -(13) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#6] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(14) Project [codegen id : 3] +(13) Project [codegen id : 2] Output [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] 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] -(15) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 2] Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] Keys [1]: [ws_item_sk#1] Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] Aggregate Attributes [6]: [sum#14, sum#15, sum#16, isEmpty#17, sum#18, isEmpty#19] Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -(16) Exchange +(15) Exchange Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(17) HashAggregate [codegen id : 4] +(16) HashAggregate [codegen id : 3] Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Keys [1]: [ws_item_sk#1] Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29] Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32] -(18) Exchange +(17) Exchange Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(19) Sort [codegen id : 5] +(18) Sort [codegen id : 4] Input [3]: [item#30, return_ratio#31, currency_ratio#32] Arguments: [return_ratio#31 ASC NULLS FIRST], false, 0 -(20) Window +(19) Window Input [3]: [item#30, return_ratio#31, currency_ratio#32] Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] -(21) Sort [codegen id : 6] +(20) Sort [codegen id : 5] Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 -(22) Window +(21) Window Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] -(23) Filter [codegen id : 7] +(22) Filter [codegen id : 6] Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) -(24) Project [codegen id : 7] +(23) Project [codegen id : 6] Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] @@ -196,20 +189,17 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(26) CometFilter +(25) CometFilter Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) -(27) CometProject +(26) CometProject Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41], [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -(28) ColumnarToRow [codegen id : 8] +(27) CometBroadcastExchange Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] - -(29) BroadcastExchange -Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=4] +Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] (unknown) Scan parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] @@ -218,83 +208,82 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(31) CometFilter +(29) CometFilter Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] Condition : (((isnotnull(cr_return_amount#46) AND (cr_return_amount#46 > 10000.00)) AND isnotnull(cr_order_number#44)) AND isnotnull(cr_item_sk#43)) -(32) CometProject +(30) CometProject Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46], [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] -(33) ColumnarToRow -Input [4]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] - -(34) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [cs_order_number#37, cs_item_sk#36] -Right keys [2]: [cr_order_number#44, cr_item_sk#43] -Join type: Inner -Join condition: None +(31) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +Right output [4]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] +Arguments: [cs_order_number#37, cs_item_sk#36], [cr_order_number#44, cr_item_sk#43], Inner -(35) Project [codegen id : 10] -Output [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] +(32) CometProject Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] +Arguments: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46], [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] -(36) ReusedExchange [Reuses operator id: 82] +(33) ColumnarToRow [codegen id : 8] +Input [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] + +(34) ReusedExchange [Reuses operator id: 79] Output [1]: [d_date_sk#48] -(37) BroadcastHashJoin [codegen id : 10] +(35) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_sold_date_sk#41] Right keys [1]: [d_date_sk#48] Join type: Inner Join condition: None -(38) Project [codegen id : 10] +(36) Project [codegen id : 8] Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46, d_date_sk#48] -(39) HashAggregate [codegen id : 10] +(37) HashAggregate [codegen id : 8] Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] Keys [1]: [cs_item_sk#36] Functions [4]: [partial_sum(coalesce(cr_return_quantity#45, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] Aggregate Attributes [6]: [sum#49, sum#50, sum#51, isEmpty#52, sum#53, isEmpty#54] Results [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -(40) Exchange +(38) Exchange Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(41) HashAggregate [codegen id : 11] +(39) HashAggregate [codegen id : 9] Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] Keys [1]: [cs_item_sk#36] Functions [4]: [sum(coalesce(cr_return_quantity#45, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#45, 0))#61, sum(coalesce(cs_quantity#38, 0))#62, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64] Results [3]: [cs_item_sk#36 AS item#65, (cast(sum(coalesce(cr_return_quantity#45, 0))#61 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#62 as decimal(15,4))) AS return_ratio#66, (cast(sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64 as decimal(15,4))) AS currency_ratio#67] -(42) Exchange +(40) Exchange Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(43) Sort [codegen id : 12] +(41) Sort [codegen id : 10] Input [3]: [item#65, return_ratio#66, currency_ratio#67] Arguments: [return_ratio#66 ASC NULLS FIRST], false, 0 -(44) Window +(42) Window Input [3]: [item#65, return_ratio#66, currency_ratio#67] Arguments: [rank(return_ratio#66) windowspecdefinition(return_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#68], [return_ratio#66 ASC NULLS FIRST] -(45) Sort [codegen id : 13] +(43) Sort [codegen id : 11] Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] Arguments: [currency_ratio#67 ASC NULLS FIRST], false, 0 -(46) Window +(44) Window Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] Arguments: [rank(currency_ratio#67) windowspecdefinition(currency_ratio#67 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#69], [currency_ratio#67 ASC NULLS FIRST] -(47) Filter [codegen id : 14] +(45) Filter [codegen id : 12] Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] Condition : ((return_rank#68 <= 10) OR (currency_rank#69 <= 10)) -(48) Project [codegen id : 14] +(46) Project [codegen id : 12] Output [5]: [catalog AS channel#70, item#65, return_ratio#66, return_rank#68, currency_rank#69] Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] @@ -306,20 +295,17 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#76), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(50) CometFilter +(48) CometFilter Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] Condition : (((((((isnotnull(ss_net_profit#75) AND isnotnull(ss_net_paid#74)) AND isnotnull(ss_quantity#73)) AND (ss_net_profit#75 > 1.00)) AND (ss_net_paid#74 > 0.00)) AND (ss_quantity#73 > 0)) AND isnotnull(ss_ticket_number#72)) AND isnotnull(ss_item_sk#71)) -(51) CometProject +(49) CometProject Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76], [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -(52) ColumnarToRow [codegen id : 15] +(50) CometBroadcastExchange Input [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] - -(53) BroadcastExchange -Input [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=7] +Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] (unknown) Scan parquet spark_catalog.default.store_returns Output [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] @@ -328,118 +314,117 @@ Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(55) CometFilter +(52) CometFilter Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] Condition : (((isnotnull(sr_return_amt#81) AND (sr_return_amt#81 > 10000.00)) AND isnotnull(sr_ticket_number#79)) AND isnotnull(sr_item_sk#78)) -(56) CometProject +(53) CometProject Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] Arguments: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81], [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] -(57) ColumnarToRow -Input [4]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] - -(58) BroadcastHashJoin [codegen id : 17] -Left keys [2]: [ss_ticket_number#72, ss_item_sk#71] -Right keys [2]: [sr_ticket_number#79, sr_item_sk#78] -Join type: Inner -Join condition: None +(54) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +Right output [4]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] +Arguments: [ss_ticket_number#72, ss_item_sk#71], [sr_ticket_number#79, sr_item_sk#78], Inner -(59) Project [codegen id : 17] -Output [6]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] +(55) CometProject Input [9]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] +Arguments: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81], [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] + +(56) ColumnarToRow [codegen id : 14] +Input [6]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] -(60) ReusedExchange [Reuses operator id: 82] +(57) ReusedExchange [Reuses operator id: 79] Output [1]: [d_date_sk#83] -(61) BroadcastHashJoin [codegen id : 17] +(58) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_sold_date_sk#76] Right keys [1]: [d_date_sk#83] Join type: Inner Join condition: None -(62) Project [codegen id : 17] +(59) Project [codegen id : 14] Output [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] Input [7]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81, d_date_sk#83] -(63) HashAggregate [codegen id : 17] +(60) HashAggregate [codegen id : 14] Input [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] Keys [1]: [ss_item_sk#71] Functions [4]: [partial_sum(coalesce(sr_return_quantity#80, 0)), partial_sum(coalesce(ss_quantity#73, 0)), partial_sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] Aggregate Attributes [6]: [sum#84, sum#85, sum#86, isEmpty#87, sum#88, isEmpty#89] Results [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -(64) Exchange +(61) Exchange Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Arguments: hashpartitioning(ss_item_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Arguments: hashpartitioning(ss_item_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(65) HashAggregate [codegen id : 18] +(62) HashAggregate [codegen id : 15] Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] Keys [1]: [ss_item_sk#71] Functions [4]: [sum(coalesce(sr_return_quantity#80, 0)), sum(coalesce(ss_quantity#73, 0)), sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#80, 0))#96, sum(coalesce(ss_quantity#73, 0))#97, sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98, sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99] Results [3]: [ss_item_sk#71 AS item#100, (cast(sum(coalesce(sr_return_quantity#80, 0))#96 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#73, 0))#97 as decimal(15,4))) AS return_ratio#101, (cast(sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99 as decimal(15,4))) AS currency_ratio#102] -(66) Exchange +(63) Exchange Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(67) Sort [codegen id : 19] +(64) Sort [codegen id : 16] Input [3]: [item#100, return_ratio#101, currency_ratio#102] Arguments: [return_ratio#101 ASC NULLS FIRST], false, 0 -(68) Window +(65) Window Input [3]: [item#100, return_ratio#101, currency_ratio#102] Arguments: [rank(return_ratio#101) windowspecdefinition(return_ratio#101 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#103], [return_ratio#101 ASC NULLS FIRST] -(69) Sort [codegen id : 20] +(66) Sort [codegen id : 17] Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] Arguments: [currency_ratio#102 ASC NULLS FIRST], false, 0 -(70) Window +(67) Window Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] Arguments: [rank(currency_ratio#102) windowspecdefinition(currency_ratio#102 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#104], [currency_ratio#102 ASC NULLS FIRST] -(71) Filter [codegen id : 21] +(68) Filter [codegen id : 18] Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] Condition : ((return_rank#103 <= 10) OR (currency_rank#104 <= 10)) -(72) Project [codegen id : 21] +(69) Project [codegen id : 18] Output [5]: [store AS channel#105, item#100, return_ratio#101, return_rank#103, currency_rank#104] Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] -(73) Union +(70) Union -(74) HashAggregate [codegen id : 22] +(71) HashAggregate [codegen id : 19] Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] Aggregate Attributes: [] Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(75) Exchange +(72) Exchange Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(76) HashAggregate [codegen id : 23] +(73) HashAggregate [codegen id : 20] Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] Aggregate Attributes: [] Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(77) TakeOrderedAndProject +(74) TakeOrderedAndProject Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Arguments: 100, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (82) -+- * ColumnarToRow (81) - +- CometProject (80) - +- CometFilter (79) - +- CometScan parquet spark_catalog.default.date_dim (78) +BroadcastExchange (79) ++- * ColumnarToRow (78) + +- CometProject (77) + +- CometFilter (76) + +- CometScan parquet spark_catalog.default.date_dim (75) (unknown) Scan parquet spark_catalog.default.date_dim @@ -449,23 +434,23 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(79) CometFilter +(76) CometFilter Input [3]: [d_date_sk#13, d_year#106, d_moy#107] Condition : ((((isnotnull(d_year#106) AND isnotnull(d_moy#107)) AND (d_year#106 = 2001)) AND (d_moy#107 = 12)) AND isnotnull(d_date_sk#13)) -(80) CometProject +(77) CometProject Input [3]: [d_date_sk#13, d_year#106, d_moy#107] Arguments: [d_date_sk#13], [d_date_sk#13] -(81) ColumnarToRow [codegen id : 1] +(78) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(82) BroadcastExchange +(79) BroadcastExchange Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 49 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 47 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#7 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 f007c1c66..43ebf34cc 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 @@ -1,131 +1,119 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] - WholeStageCodegen (23) + WholeStageCodegen (20) HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (22) + WholeStageCodegen (19) HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter Union - WholeStageCodegen (7) + WholeStageCodegen (6) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (6) + WholeStageCodegen (5) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (5) + WholeStageCodegen (4) Sort [return_ratio] InputAdapter Exchange #2 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [ws_item_sk] #3 - WholeStageCodegen (3) + WholeStageCodegen (2) 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] Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + CometBroadcastExchange #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_return_amt,wr_order_number,wr_item_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 - WholeStageCodegen (14) + WholeStageCodegen (12) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (13) + WholeStageCodegen (11) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (12) + WholeStageCodegen (10) Sort [return_ratio] InputAdapter Exchange #6 - WholeStageCodegen (11) + WholeStageCodegen (9) HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [cs_item_sk] #7 - WholeStageCodegen (10) + WholeStageCodegen (8) 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] Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + CometBroadcastExchange #8 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] CometFilter [cr_return_amount,cr_order_number,cr_item_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 - WholeStageCodegen (21) + WholeStageCodegen (18) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (20) + WholeStageCodegen (17) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (19) + WholeStageCodegen (16) Sort [return_ratio] InputAdapter Exchange #9 - WholeStageCodegen (18) + WholeStageCodegen (15) HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [ss_item_sk] #10 - WholeStageCodegen (17) + WholeStageCodegen (14) 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] Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometBroadcastExchange #11 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] 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 468df4fec..10ba5439e 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 @@ -1,10 +1,10 @@ == Physical Plan == -TakeOrderedAndProject (72) -+- * HashAggregate (71) - +- Exchange (70) - +- * HashAggregate (69) - +- * Expand (68) - +- Union (67) +TakeOrderedAndProject (70) ++- * HashAggregate (69) + +- Exchange (68) + +- * HashAggregate (67) + +- * Expand (66) + +- Union (65) :- * HashAggregate (20) : +- Exchange (19) : +- * HashAggregate (18) @@ -45,32 +45,30 @@ TakeOrderedAndProject (72) : +- * ColumnarToRow (34) : +- CometFilter (33) : +- CometScan parquet spark_catalog.default.catalog_page (32) - +- * HashAggregate (66) - +- Exchange (65) - +- * HashAggregate (64) - +- * Project (63) - +- * BroadcastHashJoin Inner BuildRight (62) - :- * Project (57) - : +- * BroadcastHashJoin Inner BuildRight (56) - : :- Union (54) - : : :- * ColumnarToRow (44) - : : : +- CometProject (43) - : : : +- CometFilter (42) - : : : +- CometScan parquet spark_catalog.default.web_sales (41) - : : +- * Project (53) - : : +- * BroadcastHashJoin Inner BuildLeft (52) - : : :- BroadcastExchange (47) - : : : +- * ColumnarToRow (46) - : : : +- CometScan parquet spark_catalog.default.web_returns (45) - : : +- * ColumnarToRow (51) - : : +- CometProject (50) - : : +- CometFilter (49) - : : +- CometScan parquet spark_catalog.default.web_sales (48) - : +- ReusedExchange (55) - +- BroadcastExchange (61) - +- * ColumnarToRow (60) - +- CometFilter (59) - +- CometScan parquet spark_catalog.default.web_site (58) + +- * HashAggregate (64) + +- Exchange (63) + +- * HashAggregate (62) + +- * Project (61) + +- * BroadcastHashJoin Inner BuildRight (60) + :- * Project (55) + : +- * BroadcastHashJoin Inner BuildRight (54) + : :- * ColumnarToRow (52) + : : +- CometUnion (51) + : : :- CometProject (43) + : : : +- CometFilter (42) + : : : +- CometScan parquet spark_catalog.default.web_sales (41) + : : +- CometProject (50) + : : +- CometBroadcastHashJoin (49) + : : :- CometBroadcastExchange (45) + : : : +- CometScan parquet spark_catalog.default.web_returns (44) + : : +- CometProject (48) + : : +- CometFilter (47) + : : +- CometScan parquet spark_catalog.default.web_sales (46) + : +- ReusedExchange (53) + +- BroadcastExchange (59) + +- * ColumnarToRow (58) + +- CometFilter (57) + +- CometScan parquet spark_catalog.default.web_site (56) (unknown) Scan parquet spark_catalog.default.store_sales @@ -112,7 +110,7 @@ Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_a (8) ColumnarToRow [codegen id : 3] Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -(9) ReusedExchange [Reuses operator id: 77] +(9) ReusedExchange [Reuses operator id: 75] Output [1]: [d_date_sk#22] (10) BroadcastHashJoin [codegen id : 3] @@ -210,7 +208,7 @@ Child 1 Input [6]: [page_sk#57, date_sk#58, sales_price#59, profit#60, return_am (28) ColumnarToRow [codegen id : 7] Input [6]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52] -(29) ReusedExchange [Reuses operator id: 77] +(29) ReusedExchange [Reuses operator id: 75] Output [1]: [d_date_sk#63] (30) BroadcastHashJoin [codegen id : 7] @@ -285,9 +283,6 @@ Condition : isnotnull(ws_web_site_sk#83) Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Arguments: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93], [ws_web_site_sk#83 AS wsr_web_site_sk#88, ws_sold_date_sk#86 AS date_sk#89, ws_ext_sales_price#84 AS sales_price#90, ws_net_profit#85 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] -(44) ColumnarToRow [codegen id : 9] -Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] - (unknown) Scan parquet spark_catalog.default.web_returns Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Batched: true @@ -295,12 +290,9 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#87)] ReadSchema: struct -(46) ColumnarToRow [codegen id : 10] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] - -(47) BroadcastExchange +(45) CometBroadcastExchange Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[1, int, true] as bigint) & 4294967295))),false), [plan_id=5] +Arguments: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (unknown) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] @@ -309,39 +301,40 @@ Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(49) CometFilter +(47) CometFilter Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) -(50) CometProject +(48) CometProject Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -(51) ColumnarToRow -Input [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] +(49) CometBroadcastHashJoin +Left output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Right output [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] +Arguments: [wr_item_sk#94, wr_order_number#95], [ws_item_sk#99, ws_order_number#101], Inner -(52) BroadcastHashJoin [codegen id : 11] -Left keys [2]: [wr_item_sk#94, wr_order_number#95] -Right keys [2]: [ws_item_sk#99, ws_order_number#101] -Join type: Inner -Join condition: None - -(53) Project [codegen id : 11] -Output [6]: [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] +(50) CometProject Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] +Arguments: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108], [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] -(54) Union +(51) CometUnion +Child 0 Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] +Child 1 Input [6]: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108] + +(52) ColumnarToRow [codegen id : 11] +Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] -(55) ReusedExchange [Reuses operator id: 77] +(53) ReusedExchange [Reuses operator id: 75] Output [1]: [d_date_sk#109] -(56) BroadcastHashJoin [codegen id : 14] +(54) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#89] Right keys [1]: [d_date_sk#109] Join type: Inner Join condition: None -(57) Project [codegen id : 14] +(55) Project [codegen id : 11] Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] @@ -352,81 +345,81 @@ Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(59) CometFilter +(57) CometFilter Input [2]: [web_site_sk#110, web_site_id#111] Condition : isnotnull(web_site_sk#110) -(60) ColumnarToRow [codegen id : 13] +(58) ColumnarToRow [codegen id : 10] Input [2]: [web_site_sk#110, web_site_id#111] -(61) BroadcastExchange +(59) BroadcastExchange Input [2]: [web_site_sk#110, web_site_id#111] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(62) BroadcastHashJoin [codegen id : 14] +(60) BroadcastHashJoin [codegen id : 11] Left keys [1]: [wsr_web_site_sk#88] Right keys [1]: [web_site_sk#110] Join type: Inner Join condition: None -(63) Project [codegen id : 14] +(61) Project [codegen id : 11] Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#111] -(64) HashAggregate [codegen id : 14] +(62) HashAggregate [codegen id : 11] Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] Keys [1]: [web_site_id#111] Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] Aggregate Attributes [4]: [sum#112, sum#113, sum#114, sum#115] Results [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] -(65) Exchange +(63) Exchange Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] -Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(66) HashAggregate [codegen id : 15] +(64) HashAggregate [codegen id : 12] Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] Keys [1]: [web_site_id#111] Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#120, sum(UnscaledValue(return_amt#92))#121, sum(UnscaledValue(profit#91))#122, sum(UnscaledValue(net_loss#93))#123] Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#90))#120,17,2) AS sales#124, MakeDecimal(sum(UnscaledValue(return_amt#92))#121,17,2) AS returns#125, (MakeDecimal(sum(UnscaledValue(profit#91))#122,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#123,17,2)) AS profit#126, web channel AS channel#127, concat(web_site, web_site_id#111) AS id#128] -(67) Union +(65) Union -(68) Expand [codegen id : 16] +(66) Expand [codegen id : 13] Input [5]: [sales#37, returns#38, profit#39, channel#40, id#41] Arguments: [[sales#37, returns#38, profit#39, channel#40, id#41, 0], [sales#37, returns#38, profit#39, channel#40, null, 1], [sales#37, returns#38, profit#39, null, null, 3]], [sales#37, returns#38, profit#39, channel#129, id#130, spark_grouping_id#131] -(69) HashAggregate [codegen id : 16] +(67) HashAggregate [codegen id : 13] Input [6]: [sales#37, returns#38, profit#39, channel#129, id#130, spark_grouping_id#131] Keys [3]: [channel#129, id#130, spark_grouping_id#131] Functions [3]: [partial_sum(sales#37), partial_sum(returns#38), partial_sum(profit#39)] Aggregate Attributes [6]: [sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] Results [9]: [channel#129, id#130, spark_grouping_id#131, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -(70) Exchange +(68) Exchange Input [9]: [channel#129, id#130, spark_grouping_id#131, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Arguments: hashpartitioning(channel#129, id#130, spark_grouping_id#131, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Arguments: hashpartitioning(channel#129, id#130, spark_grouping_id#131, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(71) HashAggregate [codegen id : 17] +(69) HashAggregate [codegen id : 14] Input [9]: [channel#129, id#130, spark_grouping_id#131, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] Keys [3]: [channel#129, id#130, spark_grouping_id#131] Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] Aggregate Attributes [3]: [sum(sales#37)#144, sum(returns#38)#145, sum(profit#39)#146] Results [5]: [channel#129, id#130, sum(sales#37)#144 AS sales#147, sum(returns#38)#145 AS returns#148, sum(profit#39)#146 AS profit#149] -(72) TakeOrderedAndProject +(70) TakeOrderedAndProject Input [5]: [channel#129, id#130, sales#147, returns#148, profit#149] Arguments: 100, [channel#129 ASC NULLS FIRST, id#130 ASC NULLS FIRST], [channel#129, id#130, sales#147, returns#148, profit#149] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (77) -+- * ColumnarToRow (76) - +- CometProject (75) - +- CometFilter (74) - +- CometScan parquet spark_catalog.default.date_dim (73) +BroadcastExchange (75) ++- * ColumnarToRow (74) + +- CometProject (73) + +- CometFilter (72) + +- CometScan parquet spark_catalog.default.date_dim (71) (unknown) Scan parquet spark_catalog.default.date_dim @@ -436,20 +429,20 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] ReadSchema: struct -(74) CometFilter +(72) CometFilter Input [2]: [d_date_sk#22, d_date#150] Condition : (((isnotnull(d_date#150) AND (d_date#150 >= 2000-08-23)) AND (d_date#150 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) -(75) CometProject +(73) CometProject Input [2]: [d_date_sk#22, d_date#150] Arguments: [d_date_sk#22], [d_date_sk#22] -(76) ColumnarToRow [codegen id : 1] +(74) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(77) BroadcastExchange +(75) BroadcastExchange Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 @@ -459,6 +452,6 @@ Subquery:4 Hosting operator id = 24 Hosting Expression = cr_returned_date_sk#56 Subquery:5 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 44 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 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 33a1fb7a8..3d539d591 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 @@ -1,9 +1,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (17) + WholeStageCodegen (14) HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (16) + WholeStageCodegen (13) HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] Expand [sales,returns,profit,channel,id] InputAdapter @@ -75,45 +75,36 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter CometFilter [cp_catalog_page_sk] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (15) + WholeStageCodegen (12) 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] InputAdapter Exchange [web_site_id] #7 - WholeStageCodegen (14) + WholeStageCodegen (11) HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,web_site_id] BroadcastHashJoin [wsr_web_site_sk,web_site_sk] Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (9) - ColumnarToRow - InputAdapter - CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk] - CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (11) - Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + ColumnarToRow + InputAdapter + CometUnion + CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] + CometBroadcastExchange #8 + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] + CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #9 - WholeStageCodegen (13) + WholeStageCodegen (10) ColumnarToRow InputAdapter CometFilter [web_site_sk] 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 c6911fbe5..c21eda6be 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,187 +1,183 @@ == Physical Plan == -* Sort (183) -+- Exchange (182) - +- * Project (181) - +- * SortMergeJoin Inner (180) - :- * Sort (110) - : +- Exchange (109) - : +- * HashAggregate (108) - : +- * HashAggregate (107) - : +- * Project (106) - : +- * BroadcastHashJoin Inner BuildRight (105) - : :- * Project (99) - : : +- * BroadcastHashJoin Inner BuildRight (98) - : : :- * Project (96) - : : : +- * BroadcastHashJoin Inner BuildRight (95) - : : : :- * Project (90) - : : : : +- * BroadcastHashJoin Inner BuildRight (89) - : : : : :- * Project (87) - : : : : : +- * BroadcastHashJoin Inner BuildRight (86) - : : : : : :- * Project (81) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (80) - : : : : : : :- * Project (78) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (77) - : : : : : : : :- * Project (72) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (71) - : : : : : : : : :- * Project (66) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (65) - : : : : : : : : : :- * Project (63) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (62) - : : : : : : : : : : :- * Project (57) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (56) - : : : : : : : : : : : :- * Project (54) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (53) - : : : : : : : : : : : : :- * Project (48) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : : : : : : : : : : : :- * Project (42) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : : : : : : : : : : : :- * Project (36) - : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : : : : : : : : : : : : : :- * Project (33) - : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (32) - : : : : : : : : : : : : : : : : :- * Sort (12) - : : : : : : : : : : : : : : : : : +- Exchange (11) - : : : : : : : : : : : : : : : : : +- * Project (10) - : : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : : : : : : : : : : : : : : : :- BroadcastExchange (4) - : : : : : : : : : : : : : : : : : : +- * ColumnarToRow (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (8) - : : : : : : : : : : : : : : : : : +- CometProject (7) - : : : : : : : : : : : : : : : : : +- CometFilter (6) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (5) - : : : : : : : : : : : : : : : : +- * ColumnarToRow (31) - : : : : : : : : : : : : : : : : +- CometSort (30) - : : : : : : : : : : : : : : : : +- CometProject (29) - : : : : : : : : : : : : : : : : +- CometFilter (28) - : : : : : : : : : : : : : : : : +- CometHashAggregate (27) - : : : : : : : : : : : : : : : : +- CometExchange (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometProject (24) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (23) - : : : : : : : : : : : : : : : : :- CometSort (17) - : : : : : : : : : : : : : : : : : +- CometExchange (16) - : : : : : : : : : : : : : : : : : +- CometProject (15) - : : : : : : : : : : : : : : : : : +- CometFilter (14) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : : : : : : : : : : : +- CometSort (22) - : : : : : : : : : : : : : : : : +- CometExchange (21) - : : : : : : : : : : : : : : : : +- CometProject (20) - : : : : : : : : : : : : : : : : +- CometFilter (19) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (18) - : : : : : : : : : : : : : : : +- ReusedExchange (34) - : : : : : : : : : : : : : : +- BroadcastExchange (40) - : : : : : : : : : : : : : : +- * ColumnarToRow (39) - : : : : : : : : : : : : : : +- CometFilter (38) - : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (37) - : : : : : : : : : : : : : +- BroadcastExchange (46) - : : : : : : : : : : : : : +- * ColumnarToRow (45) - : : : : : : : : : : : : : +- CometFilter (44) - : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (43) - : : : : : : : : : : : : +- BroadcastExchange (52) - : : : : : : : : : : : : +- * ColumnarToRow (51) - : : : : : : : : : : : : +- CometFilter (50) - : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (49) - : : : : : : : : : : : +- ReusedExchange (55) - : : : : : : : : : : +- BroadcastExchange (61) - : : : : : : : : : : +- * ColumnarToRow (60) - : : : : : : : : : : +- CometFilter (59) - : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (58) - : : : : : : : : : +- ReusedExchange (64) - : : : : : : : : +- BroadcastExchange (70) - : : : : : : : : +- * ColumnarToRow (69) - : : : : : : : : +- CometFilter (68) - : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (67) - : : : : : : : +- BroadcastExchange (76) - : : : : : : : +- * ColumnarToRow (75) - : : : : : : : +- CometFilter (74) - : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (73) - : : : : : : +- ReusedExchange (79) - : : : : : +- BroadcastExchange (85) - : : : : : +- * ColumnarToRow (84) - : : : : : +- CometFilter (83) - : : : : : +- CometScan parquet spark_catalog.default.customer_address (82) - : : : : +- ReusedExchange (88) - : : : +- BroadcastExchange (94) - : : : +- * ColumnarToRow (93) - : : : +- CometFilter (92) - : : : +- CometScan parquet spark_catalog.default.income_band (91) - : : +- ReusedExchange (97) - : +- BroadcastExchange (104) - : +- * ColumnarToRow (103) - : +- CometProject (102) - : +- CometFilter (101) - : +- CometScan parquet spark_catalog.default.item (100) - +- * Sort (179) - +- Exchange (178) - +- * HashAggregate (177) - +- * HashAggregate (176) - +- * Project (175) - +- * BroadcastHashJoin Inner BuildRight (174) - :- * Project (172) - : +- * BroadcastHashJoin Inner BuildRight (171) - : :- * Project (169) - : : +- * BroadcastHashJoin Inner BuildRight (168) - : : :- * Project (166) - : : : +- * BroadcastHashJoin Inner BuildRight (165) - : : : :- * Project (163) - : : : : +- * BroadcastHashJoin Inner BuildRight (162) - : : : : :- * Project (160) - : : : : : +- * BroadcastHashJoin Inner BuildRight (159) - : : : : : :- * Project (157) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (156) - : : : : : : :- * Project (154) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (153) - : : : : : : : :- * Project (151) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (150) - : : : : : : : : :- * Project (148) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (147) - : : : : : : : : : :- * Project (145) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (144) - : : : : : : : : : : :- * Project (142) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (141) - : : : : : : : : : : : :- * Project (139) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (138) - : : : : : : : : : : : : :- * Project (136) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (135) - : : : : : : : : : : : : : :- * Project (133) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (132) - : : : : : : : : : : : : : : :- * Project (130) - : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (129) - : : : : : : : : : : : : : : : :- * Sort (122) - : : : : : : : : : : : : : : : : +- Exchange (121) - : : : : : : : : : : : : : : : : +- * Project (120) - : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (119) - : : : : : : : : : : : : : : : : :- BroadcastExchange (114) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (113) - : : : : : : : : : : : : : : : : : +- CometFilter (112) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (111) - : : : : : : : : : : : : : : : : +- * ColumnarToRow (118) - : : : : : : : : : : : : : : : : +- CometProject (117) - : : : : : : : : : : : : : : : : +- CometFilter (116) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (115) - : : : : : : : : : : : : : : : +- * ColumnarToRow (128) - : : : : : : : : : : : : : : : +- CometSort (127) - : : : : : : : : : : : : : : : +- CometProject (126) - : : : : : : : : : : : : : : : +- CometFilter (125) - : : : : : : : : : : : : : : : +- CometHashAggregate (124) - : : : : : : : : : : : : : : : +- ReusedExchange (123) - : : : : : : : : : : : : : : +- ReusedExchange (131) - : : : : : : : : : : : : : +- ReusedExchange (134) - : : : : : : : : : : : : +- ReusedExchange (137) - : : : : : : : : : : : +- ReusedExchange (140) - : : : : : : : : : : +- ReusedExchange (143) - : : : : : : : : : +- ReusedExchange (146) - : : : : : : : : +- ReusedExchange (149) - : : : : : : : +- ReusedExchange (152) - : : : : : : +- ReusedExchange (155) - : : : : : +- ReusedExchange (158) - : : : : +- ReusedExchange (161) - : : : +- ReusedExchange (164) - : : +- ReusedExchange (167) - : +- ReusedExchange (170) - +- ReusedExchange (173) +* Sort (179) ++- Exchange (178) + +- * Project (177) + +- * SortMergeJoin Inner (176) + :- * Sort (108) + : +- Exchange (107) + : +- * HashAggregate (106) + : +- * HashAggregate (105) + : +- * Project (104) + : +- * BroadcastHashJoin Inner BuildRight (103) + : :- * Project (97) + : : +- * BroadcastHashJoin Inner BuildRight (96) + : : :- * Project (94) + : : : +- * BroadcastHashJoin Inner BuildRight (93) + : : : :- * Project (88) + : : : : +- * BroadcastHashJoin Inner BuildRight (87) + : : : : :- * Project (85) + : : : : : +- * BroadcastHashJoin Inner BuildRight (84) + : : : : : :- * Project (79) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (78) + : : : : : : :- * Project (76) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (75) + : : : : : : : :- * Project (70) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (69) + : : : : : : : : :- * Project (64) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (63) + : : : : : : : : : :- * Project (61) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : : : : : : : : :- * Project (55) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : : : : : : : : :- * Project (52) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (51) + : : : : : : : : : : : : :- * Project (46) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (45) + : : : : : : : : : : : : : :- * Project (40) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (39) + : : : : : : : : : : : : : : :- * Project (34) + : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : : : : : : : : : : : : : :- * ColumnarToRow (31) + : : : : : : : : : : : : : : : : +- CometProject (30) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) + : : : : : : : : : : : : : : : : :- CometSort (10) + : : : : : : : : : : : : : : : : : +- CometExchange (9) + : : : : : : : : : : : : : : : : : +- CometProject (8) + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) + : : : : : : : : : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : : : : : : : : : : : : +- CometProject (6) + : : : : : : : : : : : : : : : : : +- CometFilter (5) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : : : : : : : : : : : : : : +- CometSort (28) + : : : : : : : : : : : : : : : : +- CometProject (27) + : : : : : : : : : : : : : : : : +- CometFilter (26) + : : : : : : : : : : : : : : : : +- CometHashAggregate (25) + : : : : : : : : : : : : : : : : +- CometExchange (24) + : : : : : : : : : : : : : : : : +- CometHashAggregate (23) + : : : : : : : : : : : : : : : : +- CometProject (22) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) + : : : : : : : : : : : : : : : : :- CometSort (15) + : : : : : : : : : : : : : : : : : +- CometExchange (14) + : : : : : : : : : : : : : : : : : +- CometProject (13) + : : : : : : : : : : : : : : : : : +- CometFilter (12) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (11) + : : : : : : : : : : : : : : : : +- CometSort (20) + : : : : : : : : : : : : : : : : +- CometExchange (19) + : : : : : : : : : : : : : : : : +- CometProject (18) + : : : : : : : : : : : : : : : : +- CometFilter (17) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (16) + : : : : : : : : : : : : : : : +- ReusedExchange (32) + : : : : : : : : : : : : : : +- BroadcastExchange (38) + : : : : : : : : : : : : : : +- * ColumnarToRow (37) + : : : : : : : : : : : : : : +- CometFilter (36) + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (35) + : : : : : : : : : : : : : +- BroadcastExchange (44) + : : : : : : : : : : : : : +- * ColumnarToRow (43) + : : : : : : : : : : : : : +- CometFilter (42) + : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (41) + : : : : : : : : : : : : +- BroadcastExchange (50) + : : : : : : : : : : : : +- * ColumnarToRow (49) + : : : : : : : : : : : : +- CometFilter (48) + : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (47) + : : : : : : : : : : : +- ReusedExchange (53) + : : : : : : : : : : +- BroadcastExchange (59) + : : : : : : : : : : +- * ColumnarToRow (58) + : : : : : : : : : : +- CometFilter (57) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (56) + : : : : : : : : : +- ReusedExchange (62) + : : : : : : : : +- BroadcastExchange (68) + : : : : : : : : +- * ColumnarToRow (67) + : : : : : : : : +- CometFilter (66) + : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (65) + : : : : : : : +- BroadcastExchange (74) + : : : : : : : +- * ColumnarToRow (73) + : : : : : : : +- CometFilter (72) + : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (71) + : : : : : : +- ReusedExchange (77) + : : : : : +- BroadcastExchange (83) + : : : : : +- * ColumnarToRow (82) + : : : : : +- CometFilter (81) + : : : : : +- CometScan parquet spark_catalog.default.customer_address (80) + : : : : +- ReusedExchange (86) + : : : +- BroadcastExchange (92) + : : : +- * ColumnarToRow (91) + : : : +- CometFilter (90) + : : : +- CometScan parquet spark_catalog.default.income_band (89) + : : +- ReusedExchange (95) + : +- BroadcastExchange (102) + : +- * ColumnarToRow (101) + : +- CometProject (100) + : +- CometFilter (99) + : +- CometScan parquet spark_catalog.default.item (98) + +- * Sort (175) + +- Exchange (174) + +- * HashAggregate (173) + +- * HashAggregate (172) + +- * Project (171) + +- * BroadcastHashJoin Inner BuildRight (170) + :- * Project (168) + : +- * BroadcastHashJoin Inner BuildRight (167) + : :- * Project (165) + : : +- * BroadcastHashJoin Inner BuildRight (164) + : : :- * Project (162) + : : : +- * BroadcastHashJoin Inner BuildRight (161) + : : : :- * Project (159) + : : : : +- * BroadcastHashJoin Inner BuildRight (158) + : : : : :- * Project (156) + : : : : : +- * BroadcastHashJoin Inner BuildRight (155) + : : : : : :- * Project (153) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (152) + : : : : : : :- * Project (150) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (149) + : : : : : : : :- * Project (147) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (146) + : : : : : : : : :- * Project (144) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (143) + : : : : : : : : : :- * Project (141) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (140) + : : : : : : : : : : :- * Project (138) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (137) + : : : : : : : : : : : :- * Project (135) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (134) + : : : : : : : : : : : : :- * Project (132) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (131) + : : : : : : : : : : : : : :- * Project (129) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (128) + : : : : : : : : : : : : : : :- * ColumnarToRow (126) + : : : : : : : : : : : : : : : +- CometProject (125) + : : : : : : : : : : : : : : : +- CometSortMergeJoin (124) + : : : : : : : : : : : : : : : :- CometSort (118) + : : : : : : : : : : : : : : : : +- CometExchange (117) + : : : : : : : : : : : : : : : : +- CometProject (116) + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (115) + : : : : : : : : : : : : : : : : :- CometBroadcastExchange (111) + : : : : : : : : : : : : : : : : : +- CometFilter (110) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (109) + : : : : : : : : : : : : : : : : +- CometProject (114) + : : : : : : : : : : : : : : : : +- CometFilter (113) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (112) + : : : : : : : : : : : : : : : +- CometSort (123) + : : : : : : : : : : : : : : : +- CometProject (122) + : : : : : : : : : : : : : : : +- CometFilter (121) + : : : : : : : : : : : : : : : +- CometHashAggregate (120) + : : : : : : : : : : : : : : : +- ReusedExchange (119) + : : : : : : : : : : : : : : +- ReusedExchange (127) + : : : : : : : : : : : : : +- ReusedExchange (130) + : : : : : : : : : : : : +- ReusedExchange (133) + : : : : : : : : : : : +- ReusedExchange (136) + : : : : : : : : : : +- ReusedExchange (139) + : : : : : : : : : +- ReusedExchange (142) + : : : : : : : : +- ReusedExchange (145) + : : : : : : : +- ReusedExchange (148) + : : : : : : +- ReusedExchange (151) + : : : : : +- ReusedExchange (154) + : : : : +- ReusedExchange (157) + : : : +- ReusedExchange (160) + : : +- ReusedExchange (163) + : +- ReusedExchange (166) + +- ReusedExchange (169) (unknown) Scan parquet spark_catalog.default.store_sales @@ -196,12 +192,9 @@ ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) -(7) CometProject +(6) CometProject Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] -(8) ColumnarToRow -Input [2]: [sr_item_sk#14, sr_ticket_number#15] +(7) CometBroadcastHashJoin +Left output [12]: [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_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Right output [2]: [sr_item_sk#14, sr_ticket_number#15] +Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner -(9) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [ss_item_sk#1, ss_ticket_number#8] -Right keys [2]: [sr_item_sk#14, sr_ticket_number#15] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 2] -Output [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] +(8) CometProject Input [14]: [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_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] +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, 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] -(11) Exchange +(9) CometExchange 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: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(12) Sort [codegen id : 3] +(10) CometSort 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 ASC NULLS FIRST], false, 0 +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] (unknown) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] @@ -246,19 +235,19 @@ Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] ReadSchema: struct -(14) CometFilter +(12) CometFilter Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) -(15) CometProject +(13) CometProject Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -(16) CometExchange +(14) CometExchange Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(17) CometSort +(15) CometSort Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] @@ -269,80 +258,79 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(19) CometFilter +(17) CometFilter Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) -(20) CometProject +(18) CometProject Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(21) CometExchange +(19) CometExchange Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(22) CometSort +(20) CometSort Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] -(23) CometSortMergeJoin +(21) CometSortMergeJoin Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] Right output [5]: [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_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner -(24) CometProject +(22) CometProject 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] -(25) CometHashAggregate +(23) CometHashAggregate Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Keys [1]: [cs_item_sk#17] Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] -(26) CometExchange +(24) CometExchange Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(27) CometHashAggregate +(25) CometHashAggregate Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] Keys [1]: [cs_item_sk#17] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] -(28) CometFilter +(26) CometFilter Input [3]: [cs_item_sk#17, sale#30, refund#31] Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) -(29) CometProject +(27) CometProject Input [3]: [cs_item_sk#17, sale#30, refund#31] Arguments: [cs_item_sk#17], [cs_item_sk#17] -(30) CometSort +(28) CometSort Input [1]: [cs_item_sk#17] Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] -(31) ColumnarToRow [codegen id : 4] -Input [1]: [cs_item_sk#17] +(29) CometSortMergeJoin +Left output [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] +Right output [1]: [cs_item_sk#17] +Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner -(32) SortMergeJoin [codegen id : 20] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [cs_item_sk#17] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 20] -Output [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] +(30) CometProject Input [12]: [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, cs_item_sk#17] +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, 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] + +(31) ColumnarToRow [codegen id : 16] +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] -(34) ReusedExchange [Reuses operator id: 187] +(32) ReusedExchange [Reuses operator id: 183] Output [2]: [d_date_sk#32, d_year#33] -(35) BroadcastHashJoin [codegen id : 20] +(33) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ss_sold_date_sk#12] Right keys [1]: [d_date_sk#32] Join type: Inner Join condition: None -(36) Project [codegen id : 20] +(34) Project [codegen id : 16] Output [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, d_year#33] Input [13]: [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, d_date_sk#32, d_year#33] @@ -353,24 +341,24 @@ Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_zip)] ReadSchema: struct -(38) CometFilter +(36) CometFilter Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(s_zip#36)) -(39) ColumnarToRow [codegen id : 6] +(37) ColumnarToRow [codegen id : 2] Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -(40) BroadcastExchange +(38) BroadcastExchange Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(41) BroadcastHashJoin [codegen id : 20] +(39) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ss_store_sk#6] Right keys [1]: [s_store_sk#34] Join type: Inner Join condition: None -(42) Project [codegen id : 20] +(40) Project [codegen id : 16] Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36] Input [14]: [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, d_year#33, s_store_sk#34, s_store_name#35, s_zip#36] @@ -381,24 +369,24 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(44) CometFilter +(42) CometFilter Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] Condition : (((((isnotnull(c_customer_sk#37) AND isnotnull(c_first_sales_date_sk#42)) AND isnotnull(c_first_shipto_date_sk#41)) AND isnotnull(c_current_cdemo_sk#38)) AND isnotnull(c_current_hdemo_sk#39)) AND isnotnull(c_current_addr_sk#40)) -(45) ColumnarToRow [codegen id : 7] +(43) ColumnarToRow [codegen id : 3] Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] -(46) BroadcastExchange +(44) BroadcastExchange Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(47) BroadcastHashJoin [codegen id : 20] +(45) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ss_customer_sk#2] Right keys [1]: [c_customer_sk#37] Join type: Inner Join condition: None -(48) Project [codegen id : 20] +(46) Project [codegen id : 16] Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] @@ -409,37 +397,37 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter +(48) CometFilter Input [2]: [d_date_sk#43, d_year#44] Condition : isnotnull(d_date_sk#43) -(51) ColumnarToRow [codegen id : 8] +(49) ColumnarToRow [codegen id : 4] Input [2]: [d_date_sk#43, d_year#44] -(52) BroadcastExchange +(50) BroadcastExchange Input [2]: [d_date_sk#43, d_year#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(53) BroadcastHashJoin [codegen id : 20] +(51) BroadcastHashJoin [codegen id : 16] Left keys [1]: [c_first_sales_date_sk#42] Right keys [1]: [d_date_sk#43] Join type: Inner Join condition: None -(54) Project [codegen id : 20] +(52) Project [codegen id : 16] Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44] Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42, d_date_sk#43, d_year#44] -(55) ReusedExchange [Reuses operator id: 52] +(53) ReusedExchange [Reuses operator id: 50] Output [2]: [d_date_sk#45, d_year#46] -(56) BroadcastHashJoin [codegen id : 20] +(54) BroadcastHashJoin [codegen id : 16] Left keys [1]: [c_first_shipto_date_sk#41] Right keys [1]: [d_date_sk#45] Join type: Inner Join condition: None -(57) Project [codegen id : 20] +(55) Project [codegen id : 16] Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44, d_date_sk#45, d_year#46] @@ -450,37 +438,37 @@ Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status)] ReadSchema: struct -(59) CometFilter +(57) CometFilter Input [2]: [cd_demo_sk#47, cd_marital_status#48] Condition : (isnotnull(cd_demo_sk#47) AND isnotnull(cd_marital_status#48)) -(60) ColumnarToRow [codegen id : 10] +(58) ColumnarToRow [codegen id : 6] Input [2]: [cd_demo_sk#47, cd_marital_status#48] -(61) BroadcastExchange +(59) BroadcastExchange Input [2]: [cd_demo_sk#47, cd_marital_status#48] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(62) BroadcastHashJoin [codegen id : 20] +(60) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ss_cdemo_sk#3] Right keys [1]: [cd_demo_sk#47] Join type: Inner Join condition: None -(63) Project [codegen id : 20] +(61) Project [codegen id : 16] Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48] Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_demo_sk#47, cd_marital_status#48] -(64) ReusedExchange [Reuses operator id: 61] +(62) ReusedExchange [Reuses operator id: 59] Output [2]: [cd_demo_sk#49, cd_marital_status#50] -(65) BroadcastHashJoin [codegen id : 20] +(63) BroadcastHashJoin [codegen id : 16] Left keys [1]: [c_current_cdemo_sk#38] Right keys [1]: [cd_demo_sk#49] Join type: Inner Join condition: NOT (cd_marital_status#48 = cd_marital_status#50) -(66) Project [codegen id : 20] +(64) Project [codegen id : 16] Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48, cd_demo_sk#49, cd_marital_status#50] @@ -491,24 +479,24 @@ Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(68) CometFilter +(66) CometFilter Input [1]: [p_promo_sk#51] Condition : isnotnull(p_promo_sk#51) -(69) ColumnarToRow [codegen id : 12] +(67) ColumnarToRow [codegen id : 8] Input [1]: [p_promo_sk#51] -(70) BroadcastExchange +(68) BroadcastExchange Input [1]: [p_promo_sk#51] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(71) BroadcastHashJoin [codegen id : 20] +(69) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ss_promo_sk#7] Right keys [1]: [p_promo_sk#51] Join type: Inner Join condition: None -(72) Project [codegen id : 20] +(70) Project [codegen id : 16] Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, p_promo_sk#51] @@ -519,37 +507,37 @@ Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] ReadSchema: struct -(74) CometFilter +(72) CometFilter Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] Condition : (isnotnull(hd_demo_sk#52) AND isnotnull(hd_income_band_sk#53)) -(75) ColumnarToRow [codegen id : 13] +(73) ColumnarToRow [codegen id : 9] Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] -(76) BroadcastExchange +(74) BroadcastExchange Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -(77) BroadcastHashJoin [codegen id : 20] +(75) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ss_hdemo_sk#4] Right keys [1]: [hd_demo_sk#52] Join type: Inner Join condition: None -(78) Project [codegen id : 20] +(76) Project [codegen id : 16] Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53] Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_demo_sk#52, hd_income_band_sk#53] -(79) ReusedExchange [Reuses operator id: 76] +(77) ReusedExchange [Reuses operator id: 74] Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -(80) BroadcastHashJoin [codegen id : 20] +(78) BroadcastHashJoin [codegen id : 16] Left keys [1]: [c_current_hdemo_sk#39] Right keys [1]: [hd_demo_sk#54] Join type: Inner Join condition: None -(81) Project [codegen id : 20] +(79) Project [codegen id : 16] Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55] Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_demo_sk#54, hd_income_band_sk#55] @@ -560,37 +548,37 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(83) CometFilter +(81) CometFilter Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] Condition : isnotnull(ca_address_sk#56) -(84) ColumnarToRow [codegen id : 15] +(82) ColumnarToRow [codegen id : 11] Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] -(85) BroadcastExchange +(83) BroadcastExchange Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] -(86) BroadcastHashJoin [codegen id : 20] +(84) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ss_addr_sk#5] Right keys [1]: [ca_address_sk#56] Join type: Inner Join condition: None -(87) Project [codegen id : 20] +(85) Project [codegen id : 16] Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] -(88) ReusedExchange [Reuses operator id: 85] +(86) ReusedExchange [Reuses operator id: 83] Output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(89) BroadcastHashJoin [codegen id : 20] +(87) BroadcastHashJoin [codegen id : 16] Left keys [1]: [c_current_addr_sk#40] Right keys [1]: [ca_address_sk#61] Join type: Inner Join condition: None -(90) Project [codegen id : 20] +(88) Project [codegen id : 16] Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] @@ -601,37 +589,37 @@ Location [not included in comparison]/{warehouse_dir}/income_band] PushedFilters: [IsNotNull(ib_income_band_sk)] ReadSchema: struct -(92) CometFilter +(90) CometFilter Input [1]: [ib_income_band_sk#66] Condition : isnotnull(ib_income_band_sk#66) -(93) ColumnarToRow [codegen id : 17] +(91) ColumnarToRow [codegen id : 13] Input [1]: [ib_income_band_sk#66] -(94) BroadcastExchange +(92) BroadcastExchange Input [1]: [ib_income_band_sk#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -(95) BroadcastHashJoin [codegen id : 20] +(93) BroadcastHashJoin [codegen id : 16] Left keys [1]: [hd_income_band_sk#53] Right keys [1]: [ib_income_band_sk#66] Join type: Inner Join condition: None -(96) Project [codegen id : 20] +(94) Project [codegen id : 16] Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ib_income_band_sk#66] -(97) ReusedExchange [Reuses operator id: 94] +(95) ReusedExchange [Reuses operator id: 92] Output [1]: [ib_income_band_sk#67] -(98) BroadcastHashJoin [codegen id : 20] +(96) BroadcastHashJoin [codegen id : 16] Left keys [1]: [hd_income_band_sk#55] Right keys [1]: [ib_income_band_sk#67] Join type: Inner Join condition: None -(99) Project [codegen id : 20] +(97) Project [codegen id : 16] Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ib_income_band_sk#67] @@ -642,50 +630,50 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), In(i_color, [burlywood ,floral ,indian ,medium ,purple ,spring ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct -(101) CometFilter +(99) CometFilter Input [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] Condition : ((((((isnotnull(i_current_price#69) AND i_color#70 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#69 >= 64.00)) AND (i_current_price#69 <= 74.00)) AND (i_current_price#69 >= 65.00)) AND (i_current_price#69 <= 79.00)) AND isnotnull(i_item_sk#68)) -(102) CometProject +(100) CometProject Input [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] Arguments: [i_item_sk#68, i_product_name#71], [i_item_sk#68, i_product_name#71] -(103) ColumnarToRow [codegen id : 19] +(101) ColumnarToRow [codegen id : 15] Input [2]: [i_item_sk#68, i_product_name#71] -(104) BroadcastExchange +(102) BroadcastExchange Input [2]: [i_item_sk#68, i_product_name#71] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -(105) BroadcastHashJoin [codegen id : 20] +(103) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#68] Join type: Inner Join condition: None -(106) Project [codegen id : 20] +(104) Project [codegen id : 16] Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#44, d_year#46, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] -(107) HashAggregate [codegen id : 20] +(105) HashAggregate [codegen id : 16] Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#44, d_year#46, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] Keys [15]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46] Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] Aggregate Attributes [4]: [count#72, sum#73, sum#74, sum#75] Results [19]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46, count#76, sum#77, sum#78, sum#79] -(108) HashAggregate [codegen id : 20] +(106) HashAggregate [codegen id : 16] Input [19]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46, count#76, sum#77, sum#78, sum#79] Keys [15]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46] Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] Aggregate Attributes [4]: [count(1)#80, sum(UnscaledValue(ss_wholesale_cost#9))#81, sum(UnscaledValue(ss_list_price#10))#82, sum(UnscaledValue(ss_coupon_amt#11))#83] Results [17]: [i_product_name#71 AS product_name#84, i_item_sk#68 AS item_sk#85, s_store_name#35 AS store_name#86, s_zip#36 AS store_zip#87, ca_street_number#57 AS b_street_number#88, ca_street_name#58 AS b_streen_name#89, ca_city#59 AS b_city#90, ca_zip#60 AS b_zip#91, ca_street_number#62 AS c_street_number#92, ca_street_name#63 AS c_street_name#93, ca_city#64 AS c_city#94, ca_zip#65 AS c_zip#95, d_year#33 AS syear#96, count(1)#80 AS cnt#97, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#81,17,2) AS s1#98, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#82,17,2) AS s2#99, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#83,17,2) AS s3#100] -(109) Exchange +(107) Exchange Input [17]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100] -Arguments: hashpartitioning(item_sk#85, store_name#86, store_zip#87, 5), ENSURE_REQUIREMENTS, [plan_id=15] +Arguments: hashpartitioning(item_sk#85, store_name#86, store_zip#87, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(110) Sort [codegen id : 21] +(108) Sort [codegen id : 17] Input [17]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100] Arguments: [item_sk#85 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, store_zip#87 ASC NULLS FIRST], false, 0 @@ -697,16 +685,13 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#112), dynamicpruningexpression(ss_s PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct -(112) CometFilter +(110) CometFilter Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] Condition : (((((((isnotnull(ss_item_sk#101) AND isnotnull(ss_ticket_number#108)) AND isnotnull(ss_store_sk#106)) AND isnotnull(ss_customer_sk#102)) AND isnotnull(ss_cdemo_sk#103)) AND isnotnull(ss_promo_sk#107)) AND isnotnull(ss_hdemo_sk#104)) AND isnotnull(ss_addr_sk#105)) -(113) ColumnarToRow [codegen id : 22] +(111) CometBroadcastExchange Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] - -(114) BroadcastExchange -Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=16] +Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] (unknown) Scan parquet spark_catalog.default.store_returns Output [3]: [sr_item_sk#114, sr_ticket_number#115, sr_returned_date_sk#116] @@ -715,310 +700,305 @@ Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct -(116) CometFilter +(113) CometFilter Input [3]: [sr_item_sk#114, sr_ticket_number#115, sr_returned_date_sk#116] Condition : (isnotnull(sr_item_sk#114) AND isnotnull(sr_ticket_number#115)) -(117) CometProject +(114) CometProject Input [3]: [sr_item_sk#114, sr_ticket_number#115, sr_returned_date_sk#116] Arguments: [sr_item_sk#114, sr_ticket_number#115], [sr_item_sk#114, sr_ticket_number#115] -(118) ColumnarToRow -Input [2]: [sr_item_sk#114, sr_ticket_number#115] +(115) CometBroadcastHashJoin +Left output [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +Right output [2]: [sr_item_sk#114, sr_ticket_number#115] +Arguments: [ss_item_sk#101, ss_ticket_number#108], [sr_item_sk#114, sr_ticket_number#115], Inner -(119) BroadcastHashJoin [codegen id : 23] -Left keys [2]: [ss_item_sk#101, ss_ticket_number#108] -Right keys [2]: [sr_item_sk#114, sr_ticket_number#115] -Join type: Inner -Join condition: None - -(120) Project [codegen id : 23] -Output [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +(116) CometProject Input [14]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112, sr_item_sk#114, sr_ticket_number#115] +Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112], [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -(121) Exchange +(117) CometExchange Input [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Arguments: hashpartitioning(ss_item_sk#101, 5), ENSURE_REQUIREMENTS, [plan_id=17] +Arguments: hashpartitioning(ss_item_sk#101, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] -(122) Sort [codegen id : 24] +(118) CometSort Input [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Arguments: [ss_item_sk#101 ASC NULLS FIRST], false, 0 +Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112], [ss_item_sk#101 ASC NULLS FIRST] -(123) ReusedExchange [Reuses operator id: 26] +(119) ReusedExchange [Reuses operator id: 24] Output [4]: [cs_item_sk#117, sum#118, sum#119, isEmpty#120] -(124) CometHashAggregate +(120) CometHashAggregate Input [4]: [cs_item_sk#117, sum#118, sum#119, isEmpty#120] Keys [1]: [cs_item_sk#117] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#121)), sum(((cr_refunded_cash#122 + cr_reversed_charge#123) + cr_store_credit#124))] -(125) CometFilter +(121) CometFilter Input [3]: [cs_item_sk#117, sale#30, refund#31] Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) -(126) CometProject +(122) CometProject Input [3]: [cs_item_sk#117, sale#30, refund#31] Arguments: [cs_item_sk#117], [cs_item_sk#117] -(127) CometSort +(123) CometSort Input [1]: [cs_item_sk#117] Arguments: [cs_item_sk#117], [cs_item_sk#117 ASC NULLS FIRST] -(128) ColumnarToRow [codegen id : 25] -Input [1]: [cs_item_sk#117] - -(129) SortMergeJoin [codegen id : 41] -Left keys [1]: [ss_item_sk#101] -Right keys [1]: [cs_item_sk#117] -Join type: Inner -Join condition: None +(124) CometSortMergeJoin +Left output [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +Right output [1]: [cs_item_sk#117] +Arguments: [ss_item_sk#101], [cs_item_sk#117], Inner -(130) Project [codegen id : 41] -Output [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +(125) CometProject Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112, cs_item_sk#117] +Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112], [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -(131) ReusedExchange [Reuses operator id: 191] +(126) ColumnarToRow [codegen id : 33] +Input [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] + +(127) ReusedExchange [Reuses operator id: 187] Output [2]: [d_date_sk#125, d_year#126] -(132) BroadcastHashJoin [codegen id : 41] +(128) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_sold_date_sk#112] Right keys [1]: [d_date_sk#125] Join type: Inner Join condition: None -(133) Project [codegen id : 41] +(129) Project [codegen id : 33] Output [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126] Input [13]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112, d_date_sk#125, d_year#126] -(134) ReusedExchange [Reuses operator id: 40] +(130) ReusedExchange [Reuses operator id: 38] Output [3]: [s_store_sk#127, s_store_name#128, s_zip#129] -(135) BroadcastHashJoin [codegen id : 41] +(131) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_store_sk#106] Right keys [1]: [s_store_sk#127] Join type: Inner Join condition: None -(136) Project [codegen id : 41] +(132) Project [codegen id : 33] Output [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129] Input [14]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_sk#127, s_store_name#128, s_zip#129] -(137) ReusedExchange [Reuses operator id: 46] +(133) ReusedExchange [Reuses operator id: 44] Output [6]: [c_customer_sk#130, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135] -(138) BroadcastHashJoin [codegen id : 41] +(134) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_customer_sk#102] Right keys [1]: [c_customer_sk#130] Join type: Inner Join condition: None -(139) Project [codegen id : 41] +(135) Project [codegen id : 33] Output [16]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135] Input [18]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_customer_sk#130, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135] -(140) ReusedExchange [Reuses operator id: 52] +(136) ReusedExchange [Reuses operator id: 50] Output [2]: [d_date_sk#136, d_year#137] -(141) BroadcastHashJoin [codegen id : 41] +(137) BroadcastHashJoin [codegen id : 33] Left keys [1]: [c_first_sales_date_sk#135] Right keys [1]: [d_date_sk#136] Join type: Inner Join condition: None -(142) Project [codegen id : 41] +(138) Project [codegen id : 33] Output [16]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, d_year#137] Input [18]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135, d_date_sk#136, d_year#137] -(143) ReusedExchange [Reuses operator id: 52] +(139) ReusedExchange [Reuses operator id: 50] Output [2]: [d_date_sk#138, d_year#139] -(144) BroadcastHashJoin [codegen id : 41] +(140) BroadcastHashJoin [codegen id : 33] Left keys [1]: [c_first_shipto_date_sk#134] Right keys [1]: [d_date_sk#138] Join type: Inner Join condition: None -(145) Project [codegen id : 41] +(141) Project [codegen id : 33] Output [16]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139] Input [18]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, d_year#137, d_date_sk#138, d_year#139] -(146) ReusedExchange [Reuses operator id: 61] +(142) ReusedExchange [Reuses operator id: 59] Output [2]: [cd_demo_sk#140, cd_marital_status#141] -(147) BroadcastHashJoin [codegen id : 41] +(143) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_cdemo_sk#103] Right keys [1]: [cd_demo_sk#140] Join type: Inner Join condition: None -(148) Project [codegen id : 41] +(144) Project [codegen id : 33] Output [16]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, cd_marital_status#141] Input [18]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, cd_demo_sk#140, cd_marital_status#141] -(149) ReusedExchange [Reuses operator id: 61] +(145) ReusedExchange [Reuses operator id: 59] Output [2]: [cd_demo_sk#142, cd_marital_status#143] -(150) BroadcastHashJoin [codegen id : 41] +(146) BroadcastHashJoin [codegen id : 33] Left keys [1]: [c_current_cdemo_sk#131] Right keys [1]: [cd_demo_sk#142] Join type: Inner Join condition: NOT (cd_marital_status#141 = cd_marital_status#143) -(151) Project [codegen id : 41] +(147) Project [codegen id : 33] Output [14]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139] Input [18]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, cd_marital_status#141, cd_demo_sk#142, cd_marital_status#143] -(152) ReusedExchange [Reuses operator id: 70] +(148) ReusedExchange [Reuses operator id: 68] Output [1]: [p_promo_sk#144] -(153) BroadcastHashJoin [codegen id : 41] +(149) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_promo_sk#107] Right keys [1]: [p_promo_sk#144] Join type: Inner Join condition: None -(154) Project [codegen id : 41] +(150) Project [codegen id : 33] Output [13]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139] Input [15]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, p_promo_sk#144] -(155) ReusedExchange [Reuses operator id: 76] +(151) ReusedExchange [Reuses operator id: 74] Output [2]: [hd_demo_sk#145, hd_income_band_sk#146] -(156) BroadcastHashJoin [codegen id : 41] +(152) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_hdemo_sk#104] Right keys [1]: [hd_demo_sk#145] Join type: Inner Join condition: None -(157) Project [codegen id : 41] +(153) Project [codegen id : 33] Output [13]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146] Input [15]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, hd_demo_sk#145, hd_income_band_sk#146] -(158) ReusedExchange [Reuses operator id: 76] +(154) ReusedExchange [Reuses operator id: 74] Output [2]: [hd_demo_sk#147, hd_income_band_sk#148] -(159) BroadcastHashJoin [codegen id : 41] +(155) BroadcastHashJoin [codegen id : 33] Left keys [1]: [c_current_hdemo_sk#132] Right keys [1]: [hd_demo_sk#147] Join type: Inner Join condition: None -(160) Project [codegen id : 41] +(156) Project [codegen id : 33] Output [13]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148] Input [15]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_demo_sk#147, hd_income_band_sk#148] -(161) ReusedExchange [Reuses operator id: 85] +(157) ReusedExchange [Reuses operator id: 83] Output [5]: [ca_address_sk#149, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153] -(162) BroadcastHashJoin [codegen id : 41] +(158) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_addr_sk#105] Right keys [1]: [ca_address_sk#149] Join type: Inner Join condition: None -(163) Project [codegen id : 41] +(159) Project [codegen id : 33] Output [16]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153] Input [18]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_address_sk#149, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153] -(164) ReusedExchange [Reuses operator id: 85] +(160) ReusedExchange [Reuses operator id: 83] Output [5]: [ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -(165) BroadcastHashJoin [codegen id : 41] +(161) BroadcastHashJoin [codegen id : 33] Left keys [1]: [c_current_addr_sk#133] Right keys [1]: [ca_address_sk#154] Join type: Inner Join condition: None -(166) Project [codegen id : 41] +(162) Project [codegen id : 33] Output [19]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] Input [21]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -(167) ReusedExchange [Reuses operator id: 94] +(163) ReusedExchange [Reuses operator id: 92] Output [1]: [ib_income_band_sk#159] -(168) BroadcastHashJoin [codegen id : 41] +(164) BroadcastHashJoin [codegen id : 33] Left keys [1]: [hd_income_band_sk#146] Right keys [1]: [ib_income_band_sk#159] Join type: Inner Join condition: None -(169) Project [codegen id : 41] +(165) Project [codegen id : 33] Output [18]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] Input [20]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ib_income_band_sk#159] -(170) ReusedExchange [Reuses operator id: 94] +(166) ReusedExchange [Reuses operator id: 92] Output [1]: [ib_income_band_sk#160] -(171) BroadcastHashJoin [codegen id : 41] +(167) BroadcastHashJoin [codegen id : 33] Left keys [1]: [hd_income_band_sk#148] Right keys [1]: [ib_income_band_sk#160] Join type: Inner Join condition: None -(172) Project [codegen id : 41] +(168) Project [codegen id : 33] Output [17]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] Input [19]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ib_income_band_sk#160] -(173) ReusedExchange [Reuses operator id: 104] +(169) ReusedExchange [Reuses operator id: 102] Output [2]: [i_item_sk#161, i_product_name#162] -(174) BroadcastHashJoin [codegen id : 41] +(170) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_item_sk#101] Right keys [1]: [i_item_sk#161] Join type: Inner Join condition: None -(175) Project [codegen id : 41] +(171) Project [codegen id : 33] Output [18]: [ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, d_year#137, d_year#139, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, i_item_sk#161, i_product_name#162] Input [19]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, i_item_sk#161, i_product_name#162] -(176) HashAggregate [codegen id : 41] +(172) HashAggregate [codegen id : 33] Input [18]: [ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, d_year#137, d_year#139, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, i_item_sk#161, i_product_name#162] Keys [15]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139] Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#109)), partial_sum(UnscaledValue(ss_list_price#110)), partial_sum(UnscaledValue(ss_coupon_amt#111))] Aggregate Attributes [4]: [count#72, sum#163, sum#164, sum#165] Results [19]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139, count#76, sum#166, sum#167, sum#168] -(177) HashAggregate [codegen id : 41] +(173) HashAggregate [codegen id : 33] Input [19]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139, count#76, sum#166, sum#167, sum#168] Keys [15]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139] Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#109)), sum(UnscaledValue(ss_list_price#110)), sum(UnscaledValue(ss_coupon_amt#111))] Aggregate Attributes [4]: [count(1)#80, sum(UnscaledValue(ss_wholesale_cost#109))#81, sum(UnscaledValue(ss_list_price#110))#82, sum(UnscaledValue(ss_coupon_amt#111))#83] Results [8]: [i_item_sk#161 AS item_sk#169, s_store_name#128 AS store_name#170, s_zip#129 AS store_zip#171, d_year#126 AS syear#172, count(1)#80 AS cnt#173, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#109))#81,17,2) AS s1#174, MakeDecimal(sum(UnscaledValue(ss_list_price#110))#82,17,2) AS s2#175, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#111))#83,17,2) AS s3#176] -(178) Exchange +(174) Exchange Input [8]: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] -Arguments: hashpartitioning(item_sk#169, store_name#170, store_zip#171, 5), ENSURE_REQUIREMENTS, [plan_id=18] +Arguments: hashpartitioning(item_sk#169, store_name#170, store_zip#171, 5), ENSURE_REQUIREMENTS, [plan_id=16] -(179) Sort [codegen id : 42] +(175) Sort [codegen id : 34] Input [8]: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] Arguments: [item_sk#169 ASC NULLS FIRST, store_name#170 ASC NULLS FIRST, store_zip#171 ASC NULLS FIRST], false, 0 -(180) SortMergeJoin [codegen id : 43] +(176) SortMergeJoin [codegen id : 35] Left keys [3]: [item_sk#85, store_name#86, store_zip#87] Right keys [3]: [item_sk#169, store_name#170, store_zip#171] Join type: Inner Join condition: (cnt#173 <= cnt#97) -(181) Project [codegen id : 43] +(177) Project [codegen id : 35] Output [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] Input [25]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] -(182) Exchange +(178) Exchange Input [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] -Arguments: rangepartitioning(product_name#84 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, cnt#173 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=19] +Arguments: rangepartitioning(product_name#84 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, cnt#173 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=17] -(183) Sort [codegen id : 44] +(179) Sort [codegen id : 36] Input [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] Arguments: [product_name#84 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, cnt#173 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (187) -+- * ColumnarToRow (186) - +- CometFilter (185) - +- CometScan parquet spark_catalog.default.date_dim (184) +BroadcastExchange (183) ++- * ColumnarToRow (182) + +- CometFilter (181) + +- CometScan parquet spark_catalog.default.date_dim (180) (unknown) Scan parquet spark_catalog.default.date_dim @@ -1028,22 +1008,22 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(185) CometFilter +(181) CometFilter 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)) -(186) ColumnarToRow [codegen id : 1] +(182) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] -(187) BroadcastExchange +(183) BroadcastExchange Input [2]: [d_date_sk#32, d_year#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] -Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#112 IN dynamicpruning#113 -BroadcastExchange (191) -+- * ColumnarToRow (190) - +- CometFilter (189) - +- CometScan parquet spark_catalog.default.date_dim (188) +Subquery:2 Hosting operator id = 109 Hosting Expression = ss_sold_date_sk#112 IN dynamicpruning#113 +BroadcastExchange (187) ++- * ColumnarToRow (186) + +- CometFilter (185) + +- CometScan parquet spark_catalog.default.date_dim (184) (unknown) Scan parquet spark_catalog.default.date_dim @@ -1053,15 +1033,15 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(189) CometFilter +(185) CometFilter Input [2]: [d_date_sk#125, d_year#126] Condition : ((isnotnull(d_year#126) AND (d_year#126 = 2000)) AND isnotnull(d_date_sk#125)) -(190) ColumnarToRow [codegen id : 1] +(186) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#125, d_year#126] -(191) BroadcastExchange +(187) BroadcastExchange Input [2]: [d_date_sk#125, d_year#126] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] 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 9529a3b00..eda7db4e5 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,16 +1,16 @@ -WholeStageCodegen (44) +WholeStageCodegen (36) Sort [product_name,store_name,cnt] InputAdapter Exchange [product_name,store_name,cnt] #1 - WholeStageCodegen (43) + WholeStageCodegen (35) Project [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] SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] InputAdapter - WholeStageCodegen (21) + WholeStageCodegen (17) Sort [item_sk,store_name,store_zip] InputAdapter Exchange [item_sk,store_name,store_zip] #2 - WholeStageCodegen (20) + WholeStageCodegen (16) HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,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,count,sum,sum,sum] HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] @@ -43,76 +43,64 @@ WholeStageCodegen (44) BroadcastHashJoin [ss_store_sk,s_store_sk] Project [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,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - SortMergeJoin [ss_item_sk,cs_item_sk] - InputAdapter - WholeStageCodegen (3) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #3 - WholeStageCodegen (2) - Project [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] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [sale,refund] - CometHashAggregate [cs_item_sk,sum,sum,isEmpty] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + ColumnarToRow + InputAdapter + CometProject [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] + CometSortMergeJoin [ss_item_sk,cs_item_sk] + CometSort [ss_item_sk] + CometExchange [ss_item_sk] #3 + CometProject [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] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + CometBroadcastExchange #4 + CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [sale,refund] + CometHashAggregate [cs_item_sk,sum,sum,isEmpty] + CometExchange [cs_item_sk] #6 + CometHashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometSortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + CometSort [cs_item_sk,cs_order_number] + CometExchange [cs_item_sk,cs_order_number] #7 + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometSort [cr_item_sk,cr_order_number] + CometExchange [cr_item_sk,cr_order_number] #8 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #5 InputAdapter BroadcastExchange #9 - WholeStageCodegen (6) + WholeStageCodegen (2) ColumnarToRow 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 (7) + WholeStageCodegen (3) ColumnarToRow InputAdapter CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_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 (8) + WholeStageCodegen (4) ColumnarToRow InputAdapter CometFilter [d_date_sk] @@ -121,7 +109,7 @@ WholeStageCodegen (44) ReusedExchange [d_date_sk,d_year] #11 InputAdapter BroadcastExchange #12 - WholeStageCodegen (10) + WholeStageCodegen (6) ColumnarToRow InputAdapter CometFilter [cd_demo_sk,cd_marital_status] @@ -130,14 +118,14 @@ WholeStageCodegen (44) ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter BroadcastExchange #13 - WholeStageCodegen (12) + WholeStageCodegen (8) ColumnarToRow InputAdapter CometFilter [p_promo_sk] CometScan parquet spark_catalog.default.promotion [p_promo_sk] InputAdapter BroadcastExchange #14 - WholeStageCodegen (13) + WholeStageCodegen (9) ColumnarToRow InputAdapter CometFilter [hd_demo_sk,hd_income_band_sk] @@ -146,7 +134,7 @@ WholeStageCodegen (44) ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter BroadcastExchange #15 - WholeStageCodegen (15) + WholeStageCodegen (11) ColumnarToRow InputAdapter CometFilter [ca_address_sk] @@ -155,7 +143,7 @@ WholeStageCodegen (44) ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter BroadcastExchange #16 - WholeStageCodegen (17) + WholeStageCodegen (13) ColumnarToRow InputAdapter CometFilter [ib_income_band_sk] @@ -164,18 +152,18 @@ WholeStageCodegen (44) ReusedExchange [ib_income_band_sk] #16 InputAdapter BroadcastExchange #17 - WholeStageCodegen (19) + WholeStageCodegen (15) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] CometFilter [i_current_price,i_color,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter - WholeStageCodegen (42) + WholeStageCodegen (34) Sort [item_sk,store_name,store_zip] InputAdapter Exchange [item_sk,store_name,store_zip] #18 - WholeStageCodegen (41) + WholeStageCodegen (33) HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] @@ -208,44 +196,32 @@ WholeStageCodegen (44) BroadcastHashJoin [ss_store_sk,s_store_sk] Project [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,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - SortMergeJoin [ss_item_sk,cs_item_sk] - InputAdapter - WholeStageCodegen (24) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #19 - WholeStageCodegen (23) - Project [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] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #20 - WholeStageCodegen (22) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #21 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - WholeStageCodegen (25) - ColumnarToRow - InputAdapter - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [sale,refund] - CometHashAggregate [cs_item_sk,sum,sum,isEmpty] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 + ColumnarToRow + InputAdapter + CometProject [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] + CometSortMergeJoin [ss_item_sk,cs_item_sk] + CometSort [ss_item_sk] + CometExchange [ss_item_sk] #19 + CometProject [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] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + CometBroadcastExchange #20 + CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #21 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [sale,refund] + CometHashAggregate [cs_item_sk,sum,sum,isEmpty] + ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 InputAdapter ReusedExchange [d_date_sk,d_year] #21 InputAdapter 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 4de54cadc..53329906e 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,52 +1,51 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * Project (28) - : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * Project (22) - : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : :- * Project (16) - : : : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : : : : :- BroadcastExchange (4) - : : : : : : : +- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- CometProject (7) - : : : : : : +- CometFilter (6) - : : : : : : +- CometScan parquet spark_catalog.default.web_returns (5) - : : : : : +- BroadcastExchange (14) - : : : : : +- * ColumnarToRow (13) - : : : : : +- CometFilter (12) - : : : : : +- CometScan parquet spark_catalog.default.web_page (11) - : : : : +- BroadcastExchange (20) - : : : : +- * ColumnarToRow (19) - : : : : +- CometFilter (18) - : : : : +- CometScan parquet spark_catalog.default.customer_demographics (17) - : : : +- BroadcastExchange (26) - : : : +- * ColumnarToRow (25) - : : : +- CometFilter (24) - : : : +- CometScan parquet spark_catalog.default.customer_demographics (23) - : : +- BroadcastExchange (33) - : : +- * ColumnarToRow (32) - : : +- CometProject (31) - : : +- CometFilter (30) - : : +- CometScan parquet spark_catalog.default.customer_address (29) - : +- ReusedExchange (36) - +- BroadcastExchange (42) - +- * ColumnarToRow (41) - +- CometFilter (40) - +- CometScan parquet spark_catalog.default.reason (39) +TakeOrderedAndProject (47) ++- * HashAggregate (46) + +- Exchange (45) + +- * HashAggregate (44) + +- * Project (43) + +- * BroadcastHashJoin Inner BuildRight (42) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (34) + : : +- * BroadcastHashJoin Inner BuildRight (33) + : : :- * Project (27) + : : : +- * BroadcastHashJoin Inner BuildRight (26) + : : : :- * Project (21) + : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : :- * Project (15) + : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : :- * ColumnarToRow (9) + : : : : : : +- CometProject (8) + : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : :- CometBroadcastExchange (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : : : +- CometProject (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.web_returns (4) + : : : : : +- BroadcastExchange (13) + : : : : : +- * ColumnarToRow (12) + : : : : : +- CometFilter (11) + : : : : : +- CometScan parquet spark_catalog.default.web_page (10) + : : : : +- BroadcastExchange (19) + : : : : +- * ColumnarToRow (18) + : : : : +- CometFilter (17) + : : : : +- CometScan parquet spark_catalog.default.customer_demographics (16) + : : : +- BroadcastExchange (25) + : : : +- * ColumnarToRow (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (22) + : : +- BroadcastExchange (32) + : : +- * ColumnarToRow (31) + : : +- CometProject (30) + : : +- CometFilter (29) + : : +- CometScan parquet spark_catalog.default.customer_address (28) + : +- ReusedExchange (35) + +- BroadcastExchange (41) + +- * ColumnarToRow (40) + +- CometFilter (39) + +- CometScan parquet spark_catalog.default.reason (38) (unknown) Scan parquet spark_catalog.default.web_sales @@ -61,12 +60,9 @@ ReadSchema: struct= 100.00) AND (ws_sales_price#5 <= 150.00)) OR ((ws_sales_price#5 >= 50.00) AND (ws_sales_price#5 <= 100.00))) OR ((ws_sales_price#5 >= 150.00) AND (ws_sales_price#5 <= 200.00)))) AND ((((ws_net_profit#6 >= 100.00) AND (ws_net_profit#6 <= 200.00)) OR ((ws_net_profit#6 >= 150.00) AND (ws_net_profit#6 <= 300.00))) OR ((ws_net_profit#6 >= 50.00) AND (ws_net_profit#6 <= 250.00)))) -(3) ColumnarToRow [codegen id : 1] +(3) CometBroadcastExchange Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] - -(4) BroadcastExchange -Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[2, int, false] as bigint) & 4294967295))),false), [plan_id=1] +Arguments: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] (unknown) Scan parquet spark_catalog.default.web_returns Output [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] @@ -75,26 +71,25 @@ Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number), IsNotNull(wr_refunded_cdemo_sk), IsNotNull(wr_returning_cdemo_sk), IsNotNull(wr_refunded_addr_sk), IsNotNull(wr_reason_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] Condition : (((((isnotnull(wr_item_sk#9) AND isnotnull(wr_order_number#14)) AND isnotnull(wr_refunded_cdemo_sk#10)) AND isnotnull(wr_returning_cdemo_sk#12)) AND isnotnull(wr_refunded_addr_sk#11)) AND isnotnull(wr_reason_sk#13)) -(7) CometProject +(6) CometProject Input [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] Arguments: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16], [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] -(8) ColumnarToRow -Input [8]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] - -(9) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [ws_item_sk#1, ws_order_number#3] -Right keys [2]: [wr_item_sk#9, wr_order_number#14] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] +Right output [8]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] +Arguments: [ws_item_sk#1, ws_order_number#3], [wr_item_sk#9, wr_order_number#14], Inner -(10) Project [codegen id : 8] -Output [11]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] +(8) CometProject Input [15]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] +Arguments: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] + +(9) ColumnarToRow [codegen id : 7] +Input [11]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] (unknown) Scan parquet spark_catalog.default.web_page Output [1]: [wp_web_page_sk#18] @@ -103,24 +98,24 @@ Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct -(12) CometFilter +(11) CometFilter Input [1]: [wp_web_page_sk#18] Condition : isnotnull(wp_web_page_sk#18) -(13) ColumnarToRow [codegen id : 2] +(12) ColumnarToRow [codegen id : 1] Input [1]: [wp_web_page_sk#18] -(14) BroadcastExchange +(13) BroadcastExchange Input [1]: [wp_web_page_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(15) BroadcastHashJoin [codegen id : 8] +(14) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_web_page_sk#2] Right keys [1]: [wp_web_page_sk#18] Join type: Inner Join condition: None -(16) Project [codegen id : 8] +(15) Project [codegen id : 7] Output [10]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] Input [12]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, wp_web_page_sk#18] @@ -131,24 +126,24 @@ Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] ReadSchema: struct -(18) CometFilter +(17) CometFilter Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] Condition : (((isnotnull(cd_demo_sk#19) AND isnotnull(cd_marital_status#20)) AND isnotnull(cd_education_status#21)) AND ((((cd_marital_status#20 = M) AND (cd_education_status#21 = Advanced Degree )) OR ((cd_marital_status#20 = S) AND (cd_education_status#21 = College ))) OR ((cd_marital_status#20 = W) AND (cd_education_status#21 = 2 yr Degree )))) -(19) ColumnarToRow [codegen id : 3] +(18) ColumnarToRow [codegen id : 2] Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -(20) BroadcastExchange +(19) BroadcastExchange Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(21) BroadcastHashJoin [codegen id : 8] +(20) BroadcastHashJoin [codegen id : 7] Left keys [1]: [wr_refunded_cdemo_sk#10] Right keys [1]: [cd_demo_sk#19] Join type: Inner Join condition: ((((((cd_marital_status#20 = M) AND (cd_education_status#21 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#20 = S) AND (cd_education_status#21 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#20 = W) AND (cd_education_status#21 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) -(22) Project [codegen id : 8] +(21) Project [codegen id : 7] Output [10]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#20, cd_education_status#21] Input [13]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] @@ -159,24 +154,24 @@ Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status)] ReadSchema: struct -(24) CometFilter +(23) CometFilter Input [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] Condition : ((isnotnull(cd_demo_sk#22) AND isnotnull(cd_marital_status#23)) AND isnotnull(cd_education_status#24)) -(25) ColumnarToRow [codegen id : 4] +(24) ColumnarToRow [codegen id : 3] Input [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] -(26) BroadcastExchange +(25) BroadcastExchange Input [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] -Arguments: HashedRelationBroadcastMode(List(input[0, int, false], input[1, string, false], input[2, string, false]),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(input[0, int, false], input[1, string, false], input[2, string, false]),false), [plan_id=3] -(27) BroadcastHashJoin [codegen id : 8] +(26) BroadcastHashJoin [codegen id : 7] Left keys [3]: [wr_returning_cdemo_sk#12, cd_marital_status#20, cd_education_status#21] Right keys [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] Join type: Inner Join condition: None -(28) Project [codegen id : 8] +(27) Project [codegen id : 7] Output [7]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] Input [13]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#20, cd_education_status#21, cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] @@ -187,41 +182,41 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk), Or(Or(In(ca_state, [IN,NJ,OH]),In(ca_state, [CT,KY,WI])),In(ca_state, [AR,IA,LA]))] ReadSchema: struct -(30) CometFilter +(29) CometFilter Input [3]: [ca_address_sk#25, ca_state#26, ca_country#27] Condition : (((isnotnull(ca_country#27) AND (ca_country#27 = United States)) AND isnotnull(ca_address_sk#25)) AND ((ca_state#26 IN (IN,OH,NJ) OR ca_state#26 IN (WI,CT,KY)) OR ca_state#26 IN (LA,IA,AR))) -(31) CometProject +(30) CometProject Input [3]: [ca_address_sk#25, ca_state#26, ca_country#27] Arguments: [ca_address_sk#25, ca_state#26], [ca_address_sk#25, ca_state#26] -(32) ColumnarToRow [codegen id : 5] +(31) ColumnarToRow [codegen id : 4] Input [2]: [ca_address_sk#25, ca_state#26] -(33) BroadcastExchange +(32) BroadcastExchange Input [2]: [ca_address_sk#25, ca_state#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(34) BroadcastHashJoin [codegen id : 8] +(33) BroadcastHashJoin [codegen id : 7] Left keys [1]: [wr_refunded_addr_sk#11] Right keys [1]: [ca_address_sk#25] Join type: Inner Join condition: ((((ca_state#26 IN (IN,OH,NJ) AND (ws_net_profit#6 >= 100.00)) AND (ws_net_profit#6 <= 200.00)) OR ((ca_state#26 IN (WI,CT,KY) AND (ws_net_profit#6 >= 150.00)) AND (ws_net_profit#6 <= 300.00))) OR ((ca_state#26 IN (LA,IA,AR) AND (ws_net_profit#6 >= 50.00)) AND (ws_net_profit#6 <= 250.00))) -(35) Project [codegen id : 8] +(34) Project [codegen id : 7] Output [5]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] Input [9]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, ca_address_sk#25, ca_state#26] -(36) ReusedExchange [Reuses operator id: 53] +(35) ReusedExchange [Reuses operator id: 52] Output [1]: [d_date_sk#28] -(37) BroadcastHashJoin [codegen id : 8] +(36) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#7] Right keys [1]: [d_date_sk#28] Join type: Inner Join condition: None -(38) Project [codegen id : 8] +(37) Project [codegen id : 7] Output [4]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] Input [6]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, d_date_sk#28] @@ -232,57 +227,57 @@ Location [not included in comparison]/{warehouse_dir}/reason] PushedFilters: [IsNotNull(r_reason_sk)] ReadSchema: struct -(40) CometFilter +(39) CometFilter Input [2]: [r_reason_sk#29, r_reason_desc#30] Condition : isnotnull(r_reason_sk#29) -(41) ColumnarToRow [codegen id : 7] +(40) ColumnarToRow [codegen id : 6] Input [2]: [r_reason_sk#29, r_reason_desc#30] -(42) BroadcastExchange +(41) BroadcastExchange Input [2]: [r_reason_sk#29, r_reason_desc#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(43) BroadcastHashJoin [codegen id : 8] +(42) BroadcastHashJoin [codegen id : 7] Left keys [1]: [wr_reason_sk#13] Right keys [1]: [r_reason_sk#29] Join type: Inner Join condition: None -(44) Project [codegen id : 8] +(43) Project [codegen id : 7] Output [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#30] Input [6]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, r_reason_sk#29, r_reason_desc#30] -(45) HashAggregate [codegen id : 8] +(44) HashAggregate [codegen id : 7] Input [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#30] Keys [1]: [r_reason_desc#30] Functions [3]: [partial_avg(ws_quantity#4), partial_avg(UnscaledValue(wr_refunded_cash#16)), partial_avg(UnscaledValue(wr_fee#15))] Aggregate Attributes [6]: [sum#31, count#32, sum#33, count#34, sum#35, count#36] Results [7]: [r_reason_desc#30, sum#37, count#38, sum#39, count#40, sum#41, count#42] -(46) Exchange +(45) Exchange Input [7]: [r_reason_desc#30, sum#37, count#38, sum#39, count#40, sum#41, count#42] -Arguments: hashpartitioning(r_reason_desc#30, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: hashpartitioning(r_reason_desc#30, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(47) HashAggregate [codegen id : 9] +(46) HashAggregate [codegen id : 8] Input [7]: [r_reason_desc#30, sum#37, count#38, sum#39, count#40, sum#41, count#42] Keys [1]: [r_reason_desc#30] Functions [3]: [avg(ws_quantity#4), avg(UnscaledValue(wr_refunded_cash#16)), avg(UnscaledValue(wr_fee#15))] Aggregate Attributes [3]: [avg(ws_quantity#4)#43, avg(UnscaledValue(wr_refunded_cash#16))#44, avg(UnscaledValue(wr_fee#15))#45] Results [4]: [substr(r_reason_desc#30, 1, 20) AS substr(r_reason_desc, 1, 20)#46, avg(ws_quantity#4)#43 AS avg(ws_quantity)#47, cast((avg(UnscaledValue(wr_refunded_cash#16))#44 / 100.0) as decimal(11,6)) AS avg(wr_refunded_cash)#48, cast((avg(UnscaledValue(wr_fee#15))#45 / 100.0) as decimal(11,6)) AS avg(wr_fee)#49] -(48) TakeOrderedAndProject +(47) TakeOrderedAndProject Input [4]: [substr(r_reason_desc, 1, 20)#46, avg(ws_quantity)#47, avg(wr_refunded_cash)#48, avg(wr_fee)#49] Arguments: 100, [substr(r_reason_desc, 1, 20)#46 ASC NULLS FIRST, avg(ws_quantity)#47 ASC NULLS FIRST, avg(wr_refunded_cash)#48 ASC NULLS FIRST, avg(wr_fee)#49 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#46, avg(ws_quantity)#47, avg(wr_refunded_cash)#48, avg(wr_fee)#49] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (53) -+- * ColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan parquet spark_catalog.default.date_dim (49) +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) (unknown) Scan parquet spark_catalog.default.date_dim @@ -292,19 +287,19 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter +(49) CometFilter Input [2]: [d_date_sk#28, d_year#50] Condition : ((isnotnull(d_year#50) AND (d_year#50 = 2000)) AND isnotnull(d_date_sk#28)) -(51) CometProject +(50) CometProject Input [2]: [d_date_sk#28, d_year#50] Arguments: [d_date_sk#28], [d_date_sk#28] -(52) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#28] -(53) BroadcastExchange +(52) BroadcastExchange Input [1]: [d_date_sk#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] 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 e21f8091c..7c5ee727b 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,9 +1,9 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] - WholeStageCodegen (9) + WholeStageCodegen (8) HashAggregate [r_reason_desc,sum,count,sum,count,sum,count] [avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee)),substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),sum,count,sum,count,sum,count] InputAdapter Exchange [r_reason_desc] #1 - WholeStageCodegen (8) + WholeStageCodegen (7) HashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] BroadcastHashJoin [wr_reason_sk,r_reason_sk] @@ -17,52 +17,48 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu BroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] Project [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sales_price,ws_net_profit] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] + CometBroadcastExchange #2 + CometFilter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sales_price,ws_net_profit] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] CometFilter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) + WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) + WholeStageCodegen (2) ColumnarToRow InputAdapter CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #6 - WholeStageCodegen (4) + WholeStageCodegen (3) ColumnarToRow InputAdapter CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #7 - WholeStageCodegen (5) + WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [ca_address_sk,ca_state] @@ -72,7 +68,7 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #8 - WholeStageCodegen (7) + WholeStageCodegen (6) ColumnarToRow InputAdapter CometFilter [r_reason_sk] 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 bf9c2c22c..f8419179d 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 @@ -1,81 +1,78 @@ == Physical Plan == -TakeOrderedAndProject (77) -+- * HashAggregate (76) - +- Exchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (24) - : +- * Filter (23) - : +- Window (22) - : +- * Sort (21) - : +- Window (20) - : +- * Sort (19) - : +- Exchange (18) - : +- * HashAggregate (17) - : +- Exchange (16) - : +- * HashAggregate (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildLeft (10) - : : :- BroadcastExchange (5) - : : : +- * ColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : +- * ColumnarToRow (9) - : : +- CometProject (8) - : : +- CometFilter (7) - : : +- CometScan parquet spark_catalog.default.web_returns (6) - : +- ReusedExchange (12) - :- * Project (48) - : +- * Filter (47) - : +- Window (46) - : +- * Sort (45) - : +- Window (44) - : +- * Sort (43) - : +- Exchange (42) - : +- * HashAggregate (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (35) - : : +- * BroadcastHashJoin Inner BuildLeft (34) - : : :- BroadcastExchange (29) - : : : +- * ColumnarToRow (28) - : : : +- CometProject (27) - : : : +- CometFilter (26) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (25) - : : +- * ColumnarToRow (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan parquet spark_catalog.default.catalog_returns (30) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * Sort (67) - +- Exchange (66) - +- * HashAggregate (65) - +- Exchange (64) - +- * HashAggregate (63) - +- * Project (62) - +- * BroadcastHashJoin Inner BuildRight (61) - :- * Project (59) - : +- * BroadcastHashJoin Inner BuildLeft (58) - : :- BroadcastExchange (53) - : : +- * ColumnarToRow (52) - : : +- CometProject (51) - : : +- CometFilter (50) - : : +- CometScan parquet spark_catalog.default.store_sales (49) - : +- * ColumnarToRow (57) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (60) +TakeOrderedAndProject (74) ++- * HashAggregate (73) + +- Exchange (72) + +- * HashAggregate (71) + +- Union (70) + :- * Project (23) + : +- * Filter (22) + : +- Window (21) + : +- * Sort (20) + : +- Window (19) + : +- * Sort (18) + : +- Exchange (17) + : +- * HashAggregate (16) + : +- Exchange (15) + : +- * HashAggregate (14) + : +- * Project (13) + : +- * BroadcastHashJoin Inner BuildRight (12) + : :- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.web_returns (5) + : +- ReusedExchange (11) + :- * Project (46) + : +- * Filter (45) + : +- Window (44) + : +- * Sort (43) + : +- Window (42) + : +- * Sort (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- Exchange (38) + : +- * HashAggregate (37) + : +- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * ColumnarToRow (33) + : : +- CometProject (32) + : : +- CometBroadcastHashJoin (31) + : : :- CometBroadcastExchange (27) + : : : +- CometProject (26) + : : : +- CometFilter (25) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (24) + : : +- CometProject (30) + : : +- CometFilter (29) + : : +- CometScan parquet spark_catalog.default.catalog_returns (28) + : +- ReusedExchange (34) + +- * Project (69) + +- * Filter (68) + +- Window (67) + +- * Sort (66) + +- Window (65) + +- * Sort (64) + +- Exchange (63) + +- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- * Project (59) + +- * BroadcastHashJoin Inner BuildRight (58) + :- * ColumnarToRow (56) + : +- CometProject (55) + : +- CometBroadcastHashJoin (54) + : :- CometBroadcastExchange (50) + : : +- CometProject (49) + : : +- CometFilter (48) + : : +- CometScan parquet spark_catalog.default.store_sales (47) + : +- CometProject (53) + : +- CometFilter (52) + : +- CometScan parquet spark_catalog.default.store_returns (51) + +- ReusedExchange (57) (unknown) Scan parquet spark_catalog.default.web_sales @@ -94,12 +91,9 @@ Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6], [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -(4) ColumnarToRow [codegen id : 1] +(4) CometBroadcastExchange Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(5) BroadcastExchange -Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=1] +Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] (unknown) Scan parquet spark_catalog.default.web_returns Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] @@ -108,83 +102,82 @@ Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) -(8) CometProject +(7) CometProject Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -(9) ColumnarToRow -Input [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] +(8) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] +Right output [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] +Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner -(10) BroadcastHashJoin [codegen id : 3] -Left keys [2]: [ws_order_number#2, ws_item_sk#1] -Right keys [2]: [wr_order_number#9, wr_item_sk#8] -Join type: Inner -Join condition: None - -(11) Project [codegen id : 3] -Output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] +(9) CometProject Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] +Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] + +(10) ColumnarToRow [codegen id : 2] +Input [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -(12) ReusedExchange [Reuses operator id: 82] +(11) ReusedExchange [Reuses operator id: 79] Output [1]: [d_date_sk#13] -(13) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#6] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(14) Project [codegen id : 3] +(13) Project [codegen id : 2] Output [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] 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] -(15) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 2] Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] Keys [1]: [ws_item_sk#1] Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] Aggregate Attributes [6]: [sum#14, sum#15, sum#16, isEmpty#17, sum#18, isEmpty#19] Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -(16) Exchange +(15) Exchange Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(17) HashAggregate [codegen id : 4] +(16) HashAggregate [codegen id : 3] Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Keys [1]: [ws_item_sk#1] Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29] Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32] -(18) Exchange +(17) Exchange Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(19) Sort [codegen id : 5] +(18) Sort [codegen id : 4] Input [3]: [item#30, return_ratio#31, currency_ratio#32] Arguments: [return_ratio#31 ASC NULLS FIRST], false, 0 -(20) Window +(19) Window Input [3]: [item#30, return_ratio#31, currency_ratio#32] Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] -(21) Sort [codegen id : 6] +(20) Sort [codegen id : 5] Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 -(22) Window +(21) Window Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] -(23) Filter [codegen id : 7] +(22) Filter [codegen id : 6] Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) -(24) Project [codegen id : 7] +(23) Project [codegen id : 6] Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] @@ -196,20 +189,17 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(26) CometFilter +(25) CometFilter Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) -(27) CometProject +(26) CometProject Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41], [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -(28) ColumnarToRow [codegen id : 8] +(27) CometBroadcastExchange Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] - -(29) BroadcastExchange -Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=4] +Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] (unknown) Scan parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] @@ -218,83 +208,82 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(31) CometFilter +(29) CometFilter Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] Condition : (((isnotnull(cr_return_amount#46) AND (cr_return_amount#46 > 10000.00)) AND isnotnull(cr_order_number#44)) AND isnotnull(cr_item_sk#43)) -(32) CometProject +(30) CometProject Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46], [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] -(33) ColumnarToRow -Input [4]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] - -(34) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [cs_order_number#37, cs_item_sk#36] -Right keys [2]: [cr_order_number#44, cr_item_sk#43] -Join type: Inner -Join condition: None +(31) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +Right output [4]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] +Arguments: [cs_order_number#37, cs_item_sk#36], [cr_order_number#44, cr_item_sk#43], Inner -(35) Project [codegen id : 10] -Output [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] +(32) CometProject Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] +Arguments: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46], [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] -(36) ReusedExchange [Reuses operator id: 82] +(33) ColumnarToRow [codegen id : 8] +Input [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] + +(34) ReusedExchange [Reuses operator id: 79] Output [1]: [d_date_sk#48] -(37) BroadcastHashJoin [codegen id : 10] +(35) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_sold_date_sk#41] Right keys [1]: [d_date_sk#48] Join type: Inner Join condition: None -(38) Project [codegen id : 10] +(36) Project [codegen id : 8] Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46, d_date_sk#48] -(39) HashAggregate [codegen id : 10] +(37) HashAggregate [codegen id : 8] Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] Keys [1]: [cs_item_sk#36] Functions [4]: [partial_sum(coalesce(cr_return_quantity#45, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] Aggregate Attributes [6]: [sum#49, sum#50, sum#51, isEmpty#52, sum#53, isEmpty#54] Results [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -(40) Exchange +(38) Exchange Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(41) HashAggregate [codegen id : 11] +(39) HashAggregate [codegen id : 9] Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] Keys [1]: [cs_item_sk#36] Functions [4]: [sum(coalesce(cr_return_quantity#45, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#45, 0))#61, sum(coalesce(cs_quantity#38, 0))#62, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64] Results [3]: [cs_item_sk#36 AS item#65, (cast(sum(coalesce(cr_return_quantity#45, 0))#61 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#62 as decimal(15,4))) AS return_ratio#66, (cast(sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64 as decimal(15,4))) AS currency_ratio#67] -(42) Exchange +(40) Exchange Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(43) Sort [codegen id : 12] +(41) Sort [codegen id : 10] Input [3]: [item#65, return_ratio#66, currency_ratio#67] Arguments: [return_ratio#66 ASC NULLS FIRST], false, 0 -(44) Window +(42) Window Input [3]: [item#65, return_ratio#66, currency_ratio#67] Arguments: [rank(return_ratio#66) windowspecdefinition(return_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#68], [return_ratio#66 ASC NULLS FIRST] -(45) Sort [codegen id : 13] +(43) Sort [codegen id : 11] Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] Arguments: [currency_ratio#67 ASC NULLS FIRST], false, 0 -(46) Window +(44) Window Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] Arguments: [rank(currency_ratio#67) windowspecdefinition(currency_ratio#67 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#69], [currency_ratio#67 ASC NULLS FIRST] -(47) Filter [codegen id : 14] +(45) Filter [codegen id : 12] Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] Condition : ((return_rank#68 <= 10) OR (currency_rank#69 <= 10)) -(48) Project [codegen id : 14] +(46) Project [codegen id : 12] Output [5]: [catalog AS channel#70, item#65, return_ratio#66, return_rank#68, currency_rank#69] Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] @@ -306,20 +295,17 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#76), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(50) CometFilter +(48) CometFilter Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] Condition : (((((((isnotnull(ss_net_profit#75) AND isnotnull(ss_net_paid#74)) AND isnotnull(ss_quantity#73)) AND (ss_net_profit#75 > 1.00)) AND (ss_net_paid#74 > 0.00)) AND (ss_quantity#73 > 0)) AND isnotnull(ss_ticket_number#72)) AND isnotnull(ss_item_sk#71)) -(51) CometProject +(49) CometProject Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76], [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -(52) ColumnarToRow [codegen id : 15] +(50) CometBroadcastExchange Input [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] - -(53) BroadcastExchange -Input [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=7] +Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] (unknown) Scan parquet spark_catalog.default.store_returns Output [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] @@ -328,118 +314,117 @@ Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(55) CometFilter +(52) CometFilter Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] Condition : (((isnotnull(sr_return_amt#81) AND (sr_return_amt#81 > 10000.00)) AND isnotnull(sr_ticket_number#79)) AND isnotnull(sr_item_sk#78)) -(56) CometProject +(53) CometProject Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] Arguments: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81], [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] -(57) ColumnarToRow -Input [4]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] - -(58) BroadcastHashJoin [codegen id : 17] -Left keys [2]: [ss_ticket_number#72, ss_item_sk#71] -Right keys [2]: [sr_ticket_number#79, sr_item_sk#78] -Join type: Inner -Join condition: None +(54) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +Right output [4]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] +Arguments: [ss_ticket_number#72, ss_item_sk#71], [sr_ticket_number#79, sr_item_sk#78], Inner -(59) Project [codegen id : 17] -Output [6]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] +(55) CometProject Input [9]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] +Arguments: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81], [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] + +(56) ColumnarToRow [codegen id : 14] +Input [6]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] -(60) ReusedExchange [Reuses operator id: 82] +(57) ReusedExchange [Reuses operator id: 79] Output [1]: [d_date_sk#83] -(61) BroadcastHashJoin [codegen id : 17] +(58) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_sold_date_sk#76] Right keys [1]: [d_date_sk#83] Join type: Inner Join condition: None -(62) Project [codegen id : 17] +(59) Project [codegen id : 14] Output [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] Input [7]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81, d_date_sk#83] -(63) HashAggregate [codegen id : 17] +(60) HashAggregate [codegen id : 14] Input [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] Keys [1]: [ss_item_sk#71] Functions [4]: [partial_sum(coalesce(sr_return_quantity#80, 0)), partial_sum(coalesce(ss_quantity#73, 0)), partial_sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] Aggregate Attributes [6]: [sum#84, sum#85, sum#86, isEmpty#87, sum#88, isEmpty#89] Results [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -(64) Exchange +(61) Exchange Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Arguments: hashpartitioning(ss_item_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Arguments: hashpartitioning(ss_item_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(65) HashAggregate [codegen id : 18] +(62) HashAggregate [codegen id : 15] Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] Keys [1]: [ss_item_sk#71] Functions [4]: [sum(coalesce(sr_return_quantity#80, 0)), sum(coalesce(ss_quantity#73, 0)), sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#80, 0))#96, sum(coalesce(ss_quantity#73, 0))#97, sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98, sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99] Results [3]: [ss_item_sk#71 AS item#100, (cast(sum(coalesce(sr_return_quantity#80, 0))#96 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#73, 0))#97 as decimal(15,4))) AS return_ratio#101, (cast(sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99 as decimal(15,4))) AS currency_ratio#102] -(66) Exchange +(63) Exchange Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(67) Sort [codegen id : 19] +(64) Sort [codegen id : 16] Input [3]: [item#100, return_ratio#101, currency_ratio#102] Arguments: [return_ratio#101 ASC NULLS FIRST], false, 0 -(68) Window +(65) Window Input [3]: [item#100, return_ratio#101, currency_ratio#102] Arguments: [rank(return_ratio#101) windowspecdefinition(return_ratio#101 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#103], [return_ratio#101 ASC NULLS FIRST] -(69) Sort [codegen id : 20] +(66) Sort [codegen id : 17] Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] Arguments: [currency_ratio#102 ASC NULLS FIRST], false, 0 -(70) Window +(67) Window Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] Arguments: [rank(currency_ratio#102) windowspecdefinition(currency_ratio#102 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#104], [currency_ratio#102 ASC NULLS FIRST] -(71) Filter [codegen id : 21] +(68) Filter [codegen id : 18] Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] Condition : ((return_rank#103 <= 10) OR (currency_rank#104 <= 10)) -(72) Project [codegen id : 21] +(69) Project [codegen id : 18] Output [5]: [store AS channel#105, item#100, return_ratio#101, return_rank#103, currency_rank#104] Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] -(73) Union +(70) Union -(74) HashAggregate [codegen id : 22] +(71) HashAggregate [codegen id : 19] Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] Aggregate Attributes: [] Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(75) Exchange +(72) Exchange Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(76) HashAggregate [codegen id : 23] +(73) HashAggregate [codegen id : 20] Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] Aggregate Attributes: [] Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(77) TakeOrderedAndProject +(74) TakeOrderedAndProject Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Arguments: 100, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST, item#30 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (82) -+- * ColumnarToRow (81) - +- CometProject (80) - +- CometFilter (79) - +- CometScan parquet spark_catalog.default.date_dim (78) +BroadcastExchange (79) ++- * ColumnarToRow (78) + +- CometProject (77) + +- CometFilter (76) + +- CometScan parquet spark_catalog.default.date_dim (75) (unknown) Scan parquet spark_catalog.default.date_dim @@ -449,23 +434,23 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(79) CometFilter +(76) CometFilter Input [3]: [d_date_sk#13, d_year#106, d_moy#107] Condition : ((((isnotnull(d_year#106) AND isnotnull(d_moy#107)) AND (d_year#106 = 2001)) AND (d_moy#107 = 12)) AND isnotnull(d_date_sk#13)) -(80) CometProject +(77) CometProject Input [3]: [d_date_sk#13, d_year#106, d_moy#107] Arguments: [d_date_sk#13], [d_date_sk#13] -(81) ColumnarToRow [codegen id : 1] +(78) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(82) BroadcastExchange +(79) BroadcastExchange Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 49 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 47 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#7 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 f007c1c66..43ebf34cc 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 @@ -1,131 +1,119 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] - WholeStageCodegen (23) + WholeStageCodegen (20) HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (22) + WholeStageCodegen (19) HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter Union - WholeStageCodegen (7) + WholeStageCodegen (6) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (6) + WholeStageCodegen (5) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (5) + WholeStageCodegen (4) Sort [return_ratio] InputAdapter Exchange #2 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [ws_item_sk] #3 - WholeStageCodegen (3) + WholeStageCodegen (2) 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] Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + CometBroadcastExchange #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_return_amt,wr_order_number,wr_item_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 - WholeStageCodegen (14) + WholeStageCodegen (12) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (13) + WholeStageCodegen (11) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (12) + WholeStageCodegen (10) Sort [return_ratio] InputAdapter Exchange #6 - WholeStageCodegen (11) + WholeStageCodegen (9) HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [cs_item_sk] #7 - WholeStageCodegen (10) + WholeStageCodegen (8) 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] Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + CometBroadcastExchange #8 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] CometFilter [cr_return_amount,cr_order_number,cr_item_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 - WholeStageCodegen (21) + WholeStageCodegen (18) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (20) + WholeStageCodegen (17) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (19) + WholeStageCodegen (16) Sort [return_ratio] InputAdapter Exchange #9 - WholeStageCodegen (18) + WholeStageCodegen (15) HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [ss_item_sk] #10 - WholeStageCodegen (17) + WholeStageCodegen (14) 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] Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometBroadcastExchange #11 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] 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 0cae422d0..d0c446ece 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 @@ -1,13 +1,13 @@ == Physical Plan == -TakeOrderedAndProject (85) -+- * HashAggregate (84) - +- Exchange (83) - +- * HashAggregate (82) - +- Union (81) - :- * HashAggregate (70) - : +- Exchange (69) - : +- * HashAggregate (68) - : +- Union (67) +TakeOrderedAndProject (137) ++- * HashAggregate (136) + +- Exchange (135) + +- * HashAggregate (134) + +- Union (133) + :- * HashAggregate (68) + : +- Exchange (67) + : +- * HashAggregate (66) + : +- Union (65) : :- * HashAggregate (20) : : +- Exchange (19) : : +- * HashAggregate (18) @@ -48,42 +48,94 @@ TakeOrderedAndProject (85) : : +- * ColumnarToRow (34) : : +- CometFilter (33) : : +- CometScan parquet spark_catalog.default.catalog_page (32) - : +- * HashAggregate (66) - : +- Exchange (65) - : +- * HashAggregate (64) - : +- * Project (63) - : +- * BroadcastHashJoin Inner BuildRight (62) - : :- * Project (57) - : : +- * BroadcastHashJoin Inner BuildRight (56) - : : :- Union (54) - : : : :- * ColumnarToRow (44) - : : : : +- CometProject (43) - : : : : +- CometFilter (42) - : : : : +- CometScan parquet spark_catalog.default.web_sales (41) - : : : +- * Project (53) - : : : +- * BroadcastHashJoin Inner BuildLeft (52) - : : : :- BroadcastExchange (47) - : : : : +- * ColumnarToRow (46) - : : : : +- CometScan parquet spark_catalog.default.web_returns (45) - : : : +- * ColumnarToRow (51) - : : : +- CometProject (50) - : : : +- CometFilter (49) - : : : +- CometScan parquet spark_catalog.default.web_sales (48) - : : +- ReusedExchange (55) - : +- BroadcastExchange (61) - : +- * ColumnarToRow (60) - : +- CometFilter (59) - : +- CometScan parquet spark_catalog.default.web_site (58) - :- * HashAggregate (75) - : +- Exchange (74) - : +- * HashAggregate (73) - : +- * HashAggregate (72) - : +- ReusedExchange (71) - +- * HashAggregate (80) - +- Exchange (79) - +- * HashAggregate (78) - +- * HashAggregate (77) - +- ReusedExchange (76) + : +- * HashAggregate (64) + : +- Exchange (63) + : +- * HashAggregate (62) + : +- * Project (61) + : +- * BroadcastHashJoin Inner BuildRight (60) + : :- * Project (55) + : : +- * BroadcastHashJoin Inner BuildRight (54) + : : :- * ColumnarToRow (52) + : : : +- CometUnion (51) + : : : :- CometProject (43) + : : : : +- CometFilter (42) + : : : : +- CometScan parquet spark_catalog.default.web_sales (41) + : : : +- CometProject (50) + : : : +- CometBroadcastHashJoin (49) + : : : :- CometBroadcastExchange (45) + : : : : +- CometScan parquet spark_catalog.default.web_returns (44) + : : : +- CometProject (48) + : : : +- CometFilter (47) + : : : +- CometScan parquet spark_catalog.default.web_sales (46) + : : +- ReusedExchange (53) + : +- BroadcastExchange (59) + : +- * ColumnarToRow (58) + : +- CometFilter (57) + : +- CometScan parquet spark_catalog.default.web_site (56) + :- * HashAggregate (100) + : +- Exchange (99) + : +- * HashAggregate (98) + : +- * HashAggregate (97) + : +- Exchange (96) + : +- * HashAggregate (95) + : +- Union (94) + : :- * HashAggregate (70) + : : +- ReusedExchange (69) + : :- * HashAggregate (72) + : : +- ReusedExchange (71) + : +- * HashAggregate (93) + : +- Exchange (92) + : +- * HashAggregate (91) + : +- * Project (90) + : +- * BroadcastHashJoin Inner BuildRight (89) + : :- * Project (87) + : : +- * BroadcastHashJoin Inner BuildRight (86) + : : :- * ColumnarToRow (84) + : : : +- CometUnion (83) + : : : :- CometProject (75) + : : : : +- CometFilter (74) + : : : : +- CometScan parquet spark_catalog.default.web_sales (73) + : : : +- CometProject (82) + : : : +- CometBroadcastHashJoin (81) + : : : :- CometBroadcastExchange (77) + : : : : +- CometScan parquet spark_catalog.default.web_returns (76) + : : : +- CometProject (80) + : : : +- CometFilter (79) + : : : +- CometScan parquet spark_catalog.default.web_sales (78) + : : +- ReusedExchange (85) + : +- ReusedExchange (88) + +- * HashAggregate (132) + +- Exchange (131) + +- * HashAggregate (130) + +- * HashAggregate (129) + +- Exchange (128) + +- * HashAggregate (127) + +- Union (126) + :- * HashAggregate (102) + : +- ReusedExchange (101) + :- * HashAggregate (104) + : +- ReusedExchange (103) + +- * HashAggregate (125) + +- Exchange (124) + +- * HashAggregate (123) + +- * Project (122) + +- * BroadcastHashJoin Inner BuildRight (121) + :- * Project (119) + : +- * BroadcastHashJoin Inner BuildRight (118) + : :- * ColumnarToRow (116) + : : +- CometUnion (115) + : : :- CometProject (107) + : : : +- CometFilter (106) + : : : +- CometScan parquet spark_catalog.default.web_sales (105) + : : +- CometProject (114) + : : +- CometBroadcastHashJoin (113) + : : :- CometBroadcastExchange (109) + : : : +- CometScan parquet spark_catalog.default.web_returns (108) + : : +- CometProject (112) + : : +- CometFilter (111) + : : +- CometScan parquet spark_catalog.default.web_sales (110) + : +- ReusedExchange (117) + +- ReusedExchange (120) (unknown) Scan parquet spark_catalog.default.store_sales @@ -125,7 +177,7 @@ Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_a (8) ColumnarToRow [codegen id : 3] Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -(9) ReusedExchange [Reuses operator id: 90] +(9) ReusedExchange [Reuses operator id: 142] Output [1]: [d_date_sk#22] (10) BroadcastHashJoin [codegen id : 3] @@ -223,7 +275,7 @@ Child 1 Input [6]: [page_sk#57, date_sk#58, sales_price#59, profit#60, return_am (28) ColumnarToRow [codegen id : 7] Input [6]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52] -(29) ReusedExchange [Reuses operator id: 90] +(29) ReusedExchange [Reuses operator id: 142] Output [1]: [d_date_sk#63] (30) BroadcastHashJoin [codegen id : 7] @@ -298,9 +350,6 @@ Condition : isnotnull(ws_web_site_sk#83) Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Arguments: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93], [ws_web_site_sk#83 AS wsr_web_site_sk#88, ws_sold_date_sk#86 AS date_sk#89, ws_ext_sales_price#84 AS sales_price#90, ws_net_profit#85 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] -(44) ColumnarToRow [codegen id : 9] -Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] - (unknown) Scan parquet spark_catalog.default.web_returns Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Batched: true @@ -308,12 +357,9 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#87)] ReadSchema: struct -(46) ColumnarToRow [codegen id : 10] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] - -(47) BroadcastExchange +(45) CometBroadcastExchange Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[1, int, true] as bigint) & 4294967295))),false), [plan_id=5] +Arguments: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (unknown) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] @@ -322,39 +368,40 @@ Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(49) CometFilter +(47) CometFilter Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) -(50) CometProject +(48) CometProject Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -(51) ColumnarToRow -Input [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] +(49) CometBroadcastHashJoin +Left output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Right output [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] +Arguments: [wr_item_sk#94, wr_order_number#95], [ws_item_sk#99, ws_order_number#101], Inner -(52) BroadcastHashJoin [codegen id : 11] -Left keys [2]: [wr_item_sk#94, wr_order_number#95] -Right keys [2]: [ws_item_sk#99, ws_order_number#101] -Join type: Inner -Join condition: None - -(53) Project [codegen id : 11] -Output [6]: [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] +(50) CometProject Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] +Arguments: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108], [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] + +(51) CometUnion +Child 0 Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] +Child 1 Input [6]: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108] -(54) Union +(52) ColumnarToRow [codegen id : 11] +Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] -(55) ReusedExchange [Reuses operator id: 90] +(53) ReusedExchange [Reuses operator id: 142] Output [1]: [d_date_sk#109] -(56) BroadcastHashJoin [codegen id : 14] +(54) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#89] Right keys [1]: [d_date_sk#109] Join type: Inner Join condition: None -(57) Project [codegen id : 14] +(55) Project [codegen id : 11] Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] @@ -365,176 +412,440 @@ Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(59) CometFilter +(57) CometFilter Input [2]: [web_site_sk#110, web_site_id#111] Condition : isnotnull(web_site_sk#110) -(60) ColumnarToRow [codegen id : 13] +(58) ColumnarToRow [codegen id : 10] Input [2]: [web_site_sk#110, web_site_id#111] -(61) BroadcastExchange +(59) BroadcastExchange Input [2]: [web_site_sk#110, web_site_id#111] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(62) BroadcastHashJoin [codegen id : 14] +(60) BroadcastHashJoin [codegen id : 11] Left keys [1]: [wsr_web_site_sk#88] Right keys [1]: [web_site_sk#110] Join type: Inner Join condition: None -(63) Project [codegen id : 14] +(61) Project [codegen id : 11] Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#111] -(64) HashAggregate [codegen id : 14] +(62) HashAggregate [codegen id : 11] Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] Keys [1]: [web_site_id#111] Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] Aggregate Attributes [4]: [sum#112, sum#113, sum#114, sum#115] Results [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] -(65) Exchange +(63) Exchange Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] -Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(66) HashAggregate [codegen id : 15] +(64) HashAggregate [codegen id : 12] Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] Keys [1]: [web_site_id#111] Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#120, sum(UnscaledValue(return_amt#92))#121, sum(UnscaledValue(profit#91))#122, sum(UnscaledValue(net_loss#93))#123] Results [5]: [web channel AS channel#124, concat(web_site, web_site_id#111) AS id#125, MakeDecimal(sum(UnscaledValue(sales_price#90))#120,17,2) AS sales#126, MakeDecimal(sum(UnscaledValue(return_amt#92))#121,17,2) AS returns#127, (MakeDecimal(sum(UnscaledValue(profit#91))#122,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#123,17,2)) AS profit#128] -(67) Union +(65) Union -(68) HashAggregate [codegen id : 16] +(66) HashAggregate [codegen id : 13] Input [5]: [channel#37, id#38, sales#39, returns#40, profit#41] Keys [2]: [channel#37, id#38] Functions [3]: [partial_sum(sales#39), partial_sum(returns#40), partial_sum(profit#41)] Aggregate Attributes [6]: [sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] Results [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -(69) Exchange +(67) Exchange Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(70) HashAggregate [codegen id : 17] +(68) HashAggregate [codegen id : 14] Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] Keys [2]: [channel#37, id#38] Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] Aggregate Attributes [3]: [sum(sales#39)#141, sum(returns#40)#142, sum(profit#41)#143] Results [5]: [channel#37, id#38, cast(sum(sales#39)#141 as decimal(37,2)) AS sales#144, cast(sum(returns#40)#142 as decimal(37,2)) AS returns#145, cast(sum(profit#41)#143 as decimal(38,2)) AS profit#146] -(71) ReusedExchange [Reuses operator id: 69] -Output [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] +(69) ReusedExchange [Reuses operator id: 19] +Output [5]: [s_store_id#24, sum#147, sum#148, sum#149, sum#150] + +(70) HashAggregate [codegen id : 18] +Input [5]: [s_store_id#24, sum#147, sum#148, sum#149, sum#150] +Keys [1]: [s_store_id#24] +Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#33, sum(UnscaledValue(return_amt#10))#34, sum(UnscaledValue(profit#9))#35, sum(UnscaledValue(net_loss#11))#36] +Results [5]: [store channel AS channel#37, concat(store, s_store_id#24) AS id#38, MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#39, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#40, (MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2)) AS profit#41] + +(71) ReusedExchange [Reuses operator id: 39] +Output [5]: [cp_catalog_page_id#65, sum#151, sum#152, sum#153, sum#154] + +(72) HashAggregate [codegen id : 22] +Input [5]: [cp_catalog_page_id#65, sum#151, sum#152, sum#153, sum#154] +Keys [1]: [cp_catalog_page_id#65] +Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#49))#74, sum(UnscaledValue(return_amt#51))#75, sum(UnscaledValue(profit#50))#76, sum(UnscaledValue(net_loss#52))#77] +Results [5]: [catalog channel AS channel#78, concat(catalog_page, cp_catalog_page_id#65) AS id#79, MakeDecimal(sum(UnscaledValue(sales_price#49))#74,17,2) AS sales#80, MakeDecimal(sum(UnscaledValue(return_amt#51))#75,17,2) AS returns#81, (MakeDecimal(sum(UnscaledValue(profit#50))#76,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#52))#77,17,2)) AS profit#82] + +(unknown) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#155)] +PushedFilters: [IsNotNull(ws_web_site_sk)] +ReadSchema: struct + +(74) CometFilter +Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] +Condition : isnotnull(ws_web_site_sk#83) + +(75) CometProject +Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] +Arguments: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93], [ws_web_site_sk#83 AS wsr_web_site_sk#88, ws_sold_date_sk#86 AS date_sk#89, ws_ext_sales_price#84 AS sales_price#90, ws_net_profit#85 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] + +(unknown) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#155)] +ReadSchema: struct + +(77) CometBroadcastExchange +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Arguments: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] + +(unknown) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] +ReadSchema: struct + +(79) CometFilter +Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] +Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) + +(80) CometProject +Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] +Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] + +(81) CometBroadcastHashJoin +Left output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Right output [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] +Arguments: [wr_item_sk#94, wr_order_number#95], [ws_item_sk#99, ws_order_number#101], Inner + +(82) CometProject +Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] +Arguments: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108], [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] + +(83) CometUnion +Child 0 Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] +Child 1 Input [6]: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108] + +(84) ColumnarToRow [codegen id : 25] +Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] + +(85) ReusedExchange [Reuses operator id: 142] +Output [1]: [d_date_sk#109] + +(86) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [date_sk#89] +Right keys [1]: [d_date_sk#109] +Join type: Inner +Join condition: None + +(87) Project [codegen id : 25] +Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] +Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] + +(88) ReusedExchange [Reuses operator id: 59] +Output [2]: [web_site_sk#110, web_site_id#111] -(72) HashAggregate [codegen id : 34] +(89) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [wsr_web_site_sk#88] +Right keys [1]: [web_site_sk#110] +Join type: Inner +Join condition: None + +(90) Project [codegen id : 25] +Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] +Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#111] + +(91) HashAggregate [codegen id : 25] +Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] +Keys [1]: [web_site_id#111] +Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] +Aggregate Attributes [4]: [sum#156, sum#157, sum#158, sum#159] +Results [5]: [web_site_id#111, sum#160, sum#161, sum#162, sum#163] + +(92) Exchange +Input [5]: [web_site_id#111, sum#160, sum#161, sum#162, sum#163] +Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(93) HashAggregate [codegen id : 26] +Input [5]: [web_site_id#111, sum#160, sum#161, sum#162, sum#163] +Keys [1]: [web_site_id#111] +Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#120, sum(UnscaledValue(return_amt#92))#121, sum(UnscaledValue(profit#91))#122, sum(UnscaledValue(net_loss#93))#123] +Results [5]: [web channel AS channel#124, concat(web_site, web_site_id#111) AS id#125, MakeDecimal(sum(UnscaledValue(sales_price#90))#120,17,2) AS sales#126, MakeDecimal(sum(UnscaledValue(return_amt#92))#121,17,2) AS returns#127, (MakeDecimal(sum(UnscaledValue(profit#91))#122,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#123,17,2)) AS profit#128] + +(94) Union + +(95) HashAggregate [codegen id : 27] +Input [5]: [channel#37, id#38, sales#39, returns#40, profit#41] +Keys [2]: [channel#37, id#38] +Functions [3]: [partial_sum(sales#39), partial_sum(returns#40), partial_sum(profit#41)] +Aggregate Attributes [6]: [sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] +Results [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] + +(96) Exchange +Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] +Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(97) HashAggregate [codegen id : 28] Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] Keys [2]: [channel#37, id#38] Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] Aggregate Attributes [3]: [sum(sales#39)#141, sum(returns#40)#142, sum(profit#41)#143] -Results [4]: [channel#37, sum(sales#39)#141 AS sales#147, sum(returns#40)#142 AS returns#148, sum(profit#41)#143 AS profit#149] +Results [4]: [channel#37, sum(sales#39)#141 AS sales#164, sum(returns#40)#142 AS returns#165, sum(profit#41)#143 AS profit#166] -(73) HashAggregate [codegen id : 34] -Input [4]: [channel#37, sales#147, returns#148, profit#149] +(98) HashAggregate [codegen id : 28] +Input [4]: [channel#37, sales#164, returns#165, profit#166] Keys [1]: [channel#37] -Functions [3]: [partial_sum(sales#147), partial_sum(returns#148), partial_sum(profit#149)] -Aggregate Attributes [6]: [sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] -Results [7]: [channel#37, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +Functions [3]: [partial_sum(sales#164), partial_sum(returns#165), partial_sum(profit#166)] +Aggregate Attributes [6]: [sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] +Results [7]: [channel#37, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] -(74) Exchange -Input [7]: [channel#37, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] -Arguments: hashpartitioning(channel#37, 5), ENSURE_REQUIREMENTS, [plan_id=9] +(99) Exchange +Input [7]: [channel#37, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] +Arguments: hashpartitioning(channel#37, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(75) HashAggregate [codegen id : 35] -Input [7]: [channel#37, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +(100) HashAggregate [codegen id : 29] +Input [7]: [channel#37, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] Keys [1]: [channel#37] -Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] -Aggregate Attributes [3]: [sum(sales#147)#162, sum(returns#148)#163, sum(profit#149)#164] -Results [5]: [channel#37, null AS id#165, sum(sales#147)#162 AS sum(sales)#166, sum(returns#148)#163 AS sum(returns)#167, sum(profit#149)#164 AS sum(profit)#168] +Functions [3]: [sum(sales#164), sum(returns#165), sum(profit#166)] +Aggregate Attributes [3]: [sum(sales#164)#179, sum(returns#165)#180, sum(profit#166)#181] +Results [5]: [channel#37, null AS id#182, sum(sales#164)#179 AS sum(sales)#183, sum(returns#165)#180 AS sum(returns)#184, sum(profit#166)#181 AS sum(profit)#185] + +(101) ReusedExchange [Reuses operator id: 19] +Output [5]: [s_store_id#24, sum#186, sum#187, sum#188, sum#189] + +(102) HashAggregate [codegen id : 33] +Input [5]: [s_store_id#24, sum#186, sum#187, sum#188, sum#189] +Keys [1]: [s_store_id#24] +Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#33, sum(UnscaledValue(return_amt#10))#34, sum(UnscaledValue(profit#9))#35, sum(UnscaledValue(net_loss#11))#36] +Results [5]: [store channel AS channel#37, concat(store, s_store_id#24) AS id#38, MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#39, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#40, (MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2)) AS profit#41] + +(103) ReusedExchange [Reuses operator id: 39] +Output [5]: [cp_catalog_page_id#65, sum#190, sum#191, sum#192, sum#193] + +(104) HashAggregate [codegen id : 37] +Input [5]: [cp_catalog_page_id#65, sum#190, sum#191, sum#192, sum#193] +Keys [1]: [cp_catalog_page_id#65] +Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#49))#74, sum(UnscaledValue(return_amt#51))#75, sum(UnscaledValue(profit#50))#76, sum(UnscaledValue(net_loss#52))#77] +Results [5]: [catalog channel AS channel#78, concat(catalog_page, cp_catalog_page_id#65) AS id#79, MakeDecimal(sum(UnscaledValue(sales_price#49))#74,17,2) AS sales#80, MakeDecimal(sum(UnscaledValue(return_amt#51))#75,17,2) AS returns#81, (MakeDecimal(sum(UnscaledValue(profit#50))#76,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#52))#77,17,2)) AS profit#82] + +(unknown) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#194)] +PushedFilters: [IsNotNull(ws_web_site_sk)] +ReadSchema: struct + +(106) CometFilter +Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] +Condition : isnotnull(ws_web_site_sk#83) + +(107) CometProject +Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] +Arguments: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93], [ws_web_site_sk#83 AS wsr_web_site_sk#88, ws_sold_date_sk#86 AS date_sk#89, ws_ext_sales_price#84 AS sales_price#90, ws_net_profit#85 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] + +(unknown) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#194)] +ReadSchema: struct + +(109) CometBroadcastExchange +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Arguments: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] + +(unknown) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] +ReadSchema: struct + +(111) CometFilter +Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] +Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) + +(112) CometProject +Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] +Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -(76) ReusedExchange [Reuses operator id: 69] -Output [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] +(113) CometBroadcastHashJoin +Left output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Right output [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] +Arguments: [wr_item_sk#94, wr_order_number#95], [ws_item_sk#99, ws_order_number#101], Inner -(77) HashAggregate [codegen id : 52] +(114) CometProject +Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] +Arguments: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108], [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] + +(115) CometUnion +Child 0 Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] +Child 1 Input [6]: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108] + +(116) ColumnarToRow [codegen id : 40] +Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] + +(117) ReusedExchange [Reuses operator id: 142] +Output [1]: [d_date_sk#109] + +(118) BroadcastHashJoin [codegen id : 40] +Left keys [1]: [date_sk#89] +Right keys [1]: [d_date_sk#109] +Join type: Inner +Join condition: None + +(119) Project [codegen id : 40] +Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] +Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] + +(120) ReusedExchange [Reuses operator id: 59] +Output [2]: [web_site_sk#110, web_site_id#111] + +(121) BroadcastHashJoin [codegen id : 40] +Left keys [1]: [wsr_web_site_sk#88] +Right keys [1]: [web_site_sk#110] +Join type: Inner +Join condition: None + +(122) Project [codegen id : 40] +Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] +Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#111] + +(123) HashAggregate [codegen id : 40] +Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] +Keys [1]: [web_site_id#111] +Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] +Aggregate Attributes [4]: [sum#195, sum#196, sum#197, sum#198] +Results [5]: [web_site_id#111, sum#199, sum#200, sum#201, sum#202] + +(124) Exchange +Input [5]: [web_site_id#111, sum#199, sum#200, sum#201, sum#202] +Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, [plan_id=11] + +(125) HashAggregate [codegen id : 41] +Input [5]: [web_site_id#111, sum#199, sum#200, sum#201, sum#202] +Keys [1]: [web_site_id#111] +Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#120, sum(UnscaledValue(return_amt#92))#121, sum(UnscaledValue(profit#91))#122, sum(UnscaledValue(net_loss#93))#123] +Results [5]: [web channel AS channel#124, concat(web_site, web_site_id#111) AS id#125, MakeDecimal(sum(UnscaledValue(sales_price#90))#120,17,2) AS sales#126, MakeDecimal(sum(UnscaledValue(return_amt#92))#121,17,2) AS returns#127, (MakeDecimal(sum(UnscaledValue(profit#91))#122,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#123,17,2)) AS profit#128] + +(126) Union + +(127) HashAggregate [codegen id : 42] +Input [5]: [channel#37, id#38, sales#39, returns#40, profit#41] +Keys [2]: [channel#37, id#38] +Functions [3]: [partial_sum(sales#39), partial_sum(returns#40), partial_sum(profit#41)] +Aggregate Attributes [6]: [sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] +Results [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] + +(128) Exchange +Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] +Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [plan_id=12] + +(129) HashAggregate [codegen id : 43] Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] Keys [2]: [channel#37, id#38] Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] Aggregate Attributes [3]: [sum(sales#39)#141, sum(returns#40)#142, sum(profit#41)#143] -Results [3]: [sum(sales#39)#141 AS sales#147, sum(returns#40)#142 AS returns#148, sum(profit#41)#143 AS profit#149] +Results [3]: [sum(sales#39)#141 AS sales#164, sum(returns#40)#142 AS returns#165, sum(profit#41)#143 AS profit#166] -(78) HashAggregate [codegen id : 52] -Input [3]: [sales#147, returns#148, profit#149] +(130) HashAggregate [codegen id : 43] +Input [3]: [sales#164, returns#165, profit#166] Keys: [] -Functions [3]: [partial_sum(sales#147), partial_sum(returns#148), partial_sum(profit#149)] -Aggregate Attributes [6]: [sum#169, isEmpty#170, sum#171, isEmpty#172, sum#173, isEmpty#174] -Results [6]: [sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180] +Functions [3]: [partial_sum(sales#164), partial_sum(returns#165), partial_sum(profit#166)] +Aggregate Attributes [6]: [sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208] +Results [6]: [sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214] -(79) Exchange -Input [6]: [sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] +(131) Exchange +Input [6]: [sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] -(80) HashAggregate [codegen id : 53] -Input [6]: [sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180] +(132) HashAggregate [codegen id : 44] +Input [6]: [sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214] Keys: [] -Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] -Aggregate Attributes [3]: [sum(sales#147)#181, sum(returns#148)#182, sum(profit#149)#183] -Results [5]: [null AS channel#184, null AS id#185, sum(sales#147)#181 AS sum(sales)#186, sum(returns#148)#182 AS sum(returns)#187, sum(profit#149)#183 AS sum(profit)#188] +Functions [3]: [sum(sales#164), sum(returns#165), sum(profit#166)] +Aggregate Attributes [3]: [sum(sales#164)#215, sum(returns#165)#216, sum(profit#166)#217] +Results [5]: [null AS channel#218, null AS id#219, sum(sales#164)#215 AS sum(sales)#220, sum(returns#165)#216 AS sum(returns)#221, sum(profit#166)#217 AS sum(profit)#222] -(81) Union +(133) Union -(82) HashAggregate [codegen id : 54] +(134) HashAggregate [codegen id : 45] Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] Keys [5]: [channel#37, id#38, sales#144, returns#145, profit#146] Functions: [] Aggregate Attributes: [] Results [5]: [channel#37, id#38, sales#144, returns#145, profit#146] -(83) Exchange +(135) Exchange Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] -Arguments: hashpartitioning(channel#37, id#38, sales#144, returns#145, profit#146, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Arguments: hashpartitioning(channel#37, id#38, sales#144, returns#145, profit#146, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(84) HashAggregate [codegen id : 55] +(136) HashAggregate [codegen id : 46] Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] Keys [5]: [channel#37, id#38, sales#144, returns#145, profit#146] Functions: [] Aggregate Attributes: [] Results [5]: [channel#37, id#38, sales#144, returns#145, profit#146] -(85) TakeOrderedAndProject +(137) TakeOrderedAndProject Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] Arguments: 100, [channel#37 ASC NULLS FIRST, id#38 ASC NULLS FIRST], [channel#37, id#38, sales#144, returns#145, profit#146] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (90) -+- * ColumnarToRow (89) - +- CometProject (88) - +- CometFilter (87) - +- CometScan parquet spark_catalog.default.date_dim (86) +BroadcastExchange (142) ++- * ColumnarToRow (141) + +- CometProject (140) + +- CometFilter (139) + +- CometScan parquet spark_catalog.default.date_dim (138) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#189] +Output [2]: [d_date_sk#22, d_date#223] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] ReadSchema: struct -(87) CometFilter -Input [2]: [d_date_sk#22, d_date#189] -Condition : (((isnotnull(d_date#189) AND (d_date#189 >= 1998-08-04)) AND (d_date#189 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) +(139) CometFilter +Input [2]: [d_date_sk#22, d_date#223] +Condition : (((isnotnull(d_date#223) AND (d_date#223 >= 1998-08-04)) AND (d_date#223 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) -(88) CometProject -Input [2]: [d_date_sk#22, d_date#189] +(140) CometProject +Input [2]: [d_date_sk#22, d_date#223] Arguments: [d_date_sk#22], [d_date_sk#22] -(89) ColumnarToRow [codegen id : 1] +(141) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(90) BroadcastExchange +(142) BroadcastExchange Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 @@ -544,6 +855,14 @@ Subquery:4 Hosting operator id = 24 Hosting Expression = cr_returned_date_sk#56 Subquery:5 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 44 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 + +Subquery:7 Hosting operator id = 73 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 + +Subquery:8 Hosting operator id = 76 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 + +Subquery:9 Hosting operator id = 105 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 + +Subquery:10 Hosting operator id = 108 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 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 2eaeff8cd..ff62cb5c2 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 @@ -1,17 +1,17 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (55) + WholeStageCodegen (46) HashAggregate [channel,id,sales,returns,profit] InputAdapter Exchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (54) + WholeStageCodegen (45) HashAggregate [channel,id,sales,returns,profit] InputAdapter Union - WholeStageCodegen (17) + WholeStageCodegen (14) HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id] #2 - WholeStageCodegen (16) + WholeStageCodegen (13) HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Union @@ -82,64 +82,137 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter CometFilter [cp_catalog_page_sk] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (15) + WholeStageCodegen (12) 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] InputAdapter Exchange [web_site_id] #8 - WholeStageCodegen (14) + WholeStageCodegen (11) HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,web_site_id] BroadcastHashJoin [wsr_web_site_sk,web_site_sk] Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (9) - ColumnarToRow - InputAdapter - CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk] - CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (11) - Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + ColumnarToRow + InputAdapter + CometUnion + CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] + CometBroadcastExchange #9 + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] + CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #10 - WholeStageCodegen (13) + WholeStageCodegen (10) ColumnarToRow InputAdapter CometFilter [web_site_sk] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - WholeStageCodegen (35) + WholeStageCodegen (29) HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel] #11 - WholeStageCodegen (34) + WholeStageCodegen (28) HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (53) + Exchange [channel,id] #12 + WholeStageCodegen (27) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (18) + 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] + InputAdapter + ReusedExchange [s_store_id,sum,sum,sum,sum] #3 + WholeStageCodegen (22) + 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] + InputAdapter + ReusedExchange [cp_catalog_page_id,sum,sum,sum,sum] #6 + WholeStageCodegen (26) + 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] + InputAdapter + Exchange [web_site_id] #13 + WholeStageCodegen (25) + HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,web_site_id] + BroadcastHashJoin [wsr_web_site_sk,web_site_sk] + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] + CometBroadcastExchange #14 + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] + CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [web_site_sk,web_site_id] #10 + WholeStageCodegen (44) HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange #12 - WholeStageCodegen (52) + Exchange #15 + WholeStageCodegen (43) HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + Exchange [channel,id] #16 + WholeStageCodegen (42) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (33) + 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] + InputAdapter + ReusedExchange [s_store_id,sum,sum,sum,sum] #3 + WholeStageCodegen (37) + 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] + InputAdapter + ReusedExchange [cp_catalog_page_id,sum,sum,sum,sum] #6 + WholeStageCodegen (41) + 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] + InputAdapter + Exchange [web_site_id] #17 + WholeStageCodegen (40) + HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,web_site_id] + BroadcastHashJoin [wsr_web_site_sk,web_site_sk] + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] + CometBroadcastExchange #18 + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] + CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [web_site_sk,web_site_id] #10 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 1e1b3ba45..1429e39da 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,187 +1,183 @@ == Physical Plan == -* Sort (183) -+- Exchange (182) - +- * Project (181) - +- * SortMergeJoin Inner (180) - :- * Sort (110) - : +- Exchange (109) - : +- * HashAggregate (108) - : +- * HashAggregate (107) - : +- * Project (106) - : +- * BroadcastHashJoin Inner BuildRight (105) - : :- * Project (99) - : : +- * BroadcastHashJoin Inner BuildRight (98) - : : :- * Project (96) - : : : +- * BroadcastHashJoin Inner BuildRight (95) - : : : :- * Project (90) - : : : : +- * BroadcastHashJoin Inner BuildRight (89) - : : : : :- * Project (87) - : : : : : +- * BroadcastHashJoin Inner BuildRight (86) - : : : : : :- * Project (81) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (80) - : : : : : : :- * Project (78) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (77) - : : : : : : : :- * Project (72) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (71) - : : : : : : : : :- * Project (66) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (65) - : : : : : : : : : :- * Project (63) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (62) - : : : : : : : : : : :- * Project (57) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (56) - : : : : : : : : : : : :- * Project (54) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (53) - : : : : : : : : : : : : :- * Project (48) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : : : : : : : : : : : :- * Project (42) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : : : : : : : : : : : :- * Project (36) - : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : : : : : : : : : : : : : :- * Project (33) - : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (32) - : : : : : : : : : : : : : : : : :- * Sort (12) - : : : : : : : : : : : : : : : : : +- Exchange (11) - : : : : : : : : : : : : : : : : : +- * Project (10) - : : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : : : : : : : : : : : : : : : :- BroadcastExchange (4) - : : : : : : : : : : : : : : : : : : +- * ColumnarToRow (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (8) - : : : : : : : : : : : : : : : : : +- CometProject (7) - : : : : : : : : : : : : : : : : : +- CometFilter (6) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (5) - : : : : : : : : : : : : : : : : +- * ColumnarToRow (31) - : : : : : : : : : : : : : : : : +- CometSort (30) - : : : : : : : : : : : : : : : : +- CometProject (29) - : : : : : : : : : : : : : : : : +- CometFilter (28) - : : : : : : : : : : : : : : : : +- CometHashAggregate (27) - : : : : : : : : : : : : : : : : +- CometExchange (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometProject (24) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (23) - : : : : : : : : : : : : : : : : :- CometSort (17) - : : : : : : : : : : : : : : : : : +- CometExchange (16) - : : : : : : : : : : : : : : : : : +- CometProject (15) - : : : : : : : : : : : : : : : : : +- CometFilter (14) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : : : : : : : : : : : +- CometSort (22) - : : : : : : : : : : : : : : : : +- CometExchange (21) - : : : : : : : : : : : : : : : : +- CometProject (20) - : : : : : : : : : : : : : : : : +- CometFilter (19) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (18) - : : : : : : : : : : : : : : : +- ReusedExchange (34) - : : : : : : : : : : : : : : +- BroadcastExchange (40) - : : : : : : : : : : : : : : +- * ColumnarToRow (39) - : : : : : : : : : : : : : : +- CometFilter (38) - : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (37) - : : : : : : : : : : : : : +- BroadcastExchange (46) - : : : : : : : : : : : : : +- * ColumnarToRow (45) - : : : : : : : : : : : : : +- CometFilter (44) - : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (43) - : : : : : : : : : : : : +- BroadcastExchange (52) - : : : : : : : : : : : : +- * ColumnarToRow (51) - : : : : : : : : : : : : +- CometFilter (50) - : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (49) - : : : : : : : : : : : +- ReusedExchange (55) - : : : : : : : : : : +- BroadcastExchange (61) - : : : : : : : : : : +- * ColumnarToRow (60) - : : : : : : : : : : +- CometFilter (59) - : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (58) - : : : : : : : : : +- ReusedExchange (64) - : : : : : : : : +- BroadcastExchange (70) - : : : : : : : : +- * ColumnarToRow (69) - : : : : : : : : +- CometFilter (68) - : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (67) - : : : : : : : +- BroadcastExchange (76) - : : : : : : : +- * ColumnarToRow (75) - : : : : : : : +- CometFilter (74) - : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (73) - : : : : : : +- ReusedExchange (79) - : : : : : +- BroadcastExchange (85) - : : : : : +- * ColumnarToRow (84) - : : : : : +- CometFilter (83) - : : : : : +- CometScan parquet spark_catalog.default.customer_address (82) - : : : : +- ReusedExchange (88) - : : : +- BroadcastExchange (94) - : : : +- * ColumnarToRow (93) - : : : +- CometFilter (92) - : : : +- CometScan parquet spark_catalog.default.income_band (91) - : : +- ReusedExchange (97) - : +- BroadcastExchange (104) - : +- * ColumnarToRow (103) - : +- CometProject (102) - : +- CometFilter (101) - : +- CometScan parquet spark_catalog.default.item (100) - +- * Sort (179) - +- Exchange (178) - +- * HashAggregate (177) - +- * HashAggregate (176) - +- * Project (175) - +- * BroadcastHashJoin Inner BuildRight (174) - :- * Project (172) - : +- * BroadcastHashJoin Inner BuildRight (171) - : :- * Project (169) - : : +- * BroadcastHashJoin Inner BuildRight (168) - : : :- * Project (166) - : : : +- * BroadcastHashJoin Inner BuildRight (165) - : : : :- * Project (163) - : : : : +- * BroadcastHashJoin Inner BuildRight (162) - : : : : :- * Project (160) - : : : : : +- * BroadcastHashJoin Inner BuildRight (159) - : : : : : :- * Project (157) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (156) - : : : : : : :- * Project (154) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (153) - : : : : : : : :- * Project (151) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (150) - : : : : : : : : :- * Project (148) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (147) - : : : : : : : : : :- * Project (145) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (144) - : : : : : : : : : : :- * Project (142) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (141) - : : : : : : : : : : : :- * Project (139) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (138) - : : : : : : : : : : : : :- * Project (136) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (135) - : : : : : : : : : : : : : :- * Project (133) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (132) - : : : : : : : : : : : : : : :- * Project (130) - : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (129) - : : : : : : : : : : : : : : : :- * Sort (122) - : : : : : : : : : : : : : : : : +- Exchange (121) - : : : : : : : : : : : : : : : : +- * Project (120) - : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (119) - : : : : : : : : : : : : : : : : :- BroadcastExchange (114) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (113) - : : : : : : : : : : : : : : : : : +- CometFilter (112) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (111) - : : : : : : : : : : : : : : : : +- * ColumnarToRow (118) - : : : : : : : : : : : : : : : : +- CometProject (117) - : : : : : : : : : : : : : : : : +- CometFilter (116) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (115) - : : : : : : : : : : : : : : : +- * ColumnarToRow (128) - : : : : : : : : : : : : : : : +- CometSort (127) - : : : : : : : : : : : : : : : +- CometProject (126) - : : : : : : : : : : : : : : : +- CometFilter (125) - : : : : : : : : : : : : : : : +- CometHashAggregate (124) - : : : : : : : : : : : : : : : +- ReusedExchange (123) - : : : : : : : : : : : : : : +- ReusedExchange (131) - : : : : : : : : : : : : : +- ReusedExchange (134) - : : : : : : : : : : : : +- ReusedExchange (137) - : : : : : : : : : : : +- ReusedExchange (140) - : : : : : : : : : : +- ReusedExchange (143) - : : : : : : : : : +- ReusedExchange (146) - : : : : : : : : +- ReusedExchange (149) - : : : : : : : +- ReusedExchange (152) - : : : : : : +- ReusedExchange (155) - : : : : : +- ReusedExchange (158) - : : : : +- ReusedExchange (161) - : : : +- ReusedExchange (164) - : : +- ReusedExchange (167) - : +- ReusedExchange (170) - +- ReusedExchange (173) +* Sort (179) ++- Exchange (178) + +- * Project (177) + +- * SortMergeJoin Inner (176) + :- * Sort (108) + : +- Exchange (107) + : +- * HashAggregate (106) + : +- * HashAggregate (105) + : +- * Project (104) + : +- * BroadcastHashJoin Inner BuildRight (103) + : :- * Project (97) + : : +- * BroadcastHashJoin Inner BuildRight (96) + : : :- * Project (94) + : : : +- * BroadcastHashJoin Inner BuildRight (93) + : : : :- * Project (88) + : : : : +- * BroadcastHashJoin Inner BuildRight (87) + : : : : :- * Project (85) + : : : : : +- * BroadcastHashJoin Inner BuildRight (84) + : : : : : :- * Project (79) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (78) + : : : : : : :- * Project (76) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (75) + : : : : : : : :- * Project (70) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (69) + : : : : : : : : :- * Project (64) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (63) + : : : : : : : : : :- * Project (61) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : : : : : : : : :- * Project (55) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : : : : : : : : :- * Project (52) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (51) + : : : : : : : : : : : : :- * Project (46) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (45) + : : : : : : : : : : : : : :- * Project (40) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (39) + : : : : : : : : : : : : : : :- * Project (34) + : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : : : : : : : : : : : : : :- * ColumnarToRow (31) + : : : : : : : : : : : : : : : : +- CometProject (30) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) + : : : : : : : : : : : : : : : : :- CometSort (10) + : : : : : : : : : : : : : : : : : +- CometExchange (9) + : : : : : : : : : : : : : : : : : +- CometProject (8) + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) + : : : : : : : : : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : : : : : : : : : : : : +- CometProject (6) + : : : : : : : : : : : : : : : : : +- CometFilter (5) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : : : : : : : : : : : : : : +- CometSort (28) + : : : : : : : : : : : : : : : : +- CometProject (27) + : : : : : : : : : : : : : : : : +- CometFilter (26) + : : : : : : : : : : : : : : : : +- CometHashAggregate (25) + : : : : : : : : : : : : : : : : +- CometExchange (24) + : : : : : : : : : : : : : : : : +- CometHashAggregate (23) + : : : : : : : : : : : : : : : : +- CometProject (22) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) + : : : : : : : : : : : : : : : : :- CometSort (15) + : : : : : : : : : : : : : : : : : +- CometExchange (14) + : : : : : : : : : : : : : : : : : +- CometProject (13) + : : : : : : : : : : : : : : : : : +- CometFilter (12) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (11) + : : : : : : : : : : : : : : : : +- CometSort (20) + : : : : : : : : : : : : : : : : +- CometExchange (19) + : : : : : : : : : : : : : : : : +- CometProject (18) + : : : : : : : : : : : : : : : : +- CometFilter (17) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (16) + : : : : : : : : : : : : : : : +- ReusedExchange (32) + : : : : : : : : : : : : : : +- BroadcastExchange (38) + : : : : : : : : : : : : : : +- * ColumnarToRow (37) + : : : : : : : : : : : : : : +- CometFilter (36) + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (35) + : : : : : : : : : : : : : +- BroadcastExchange (44) + : : : : : : : : : : : : : +- * ColumnarToRow (43) + : : : : : : : : : : : : : +- CometFilter (42) + : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (41) + : : : : : : : : : : : : +- BroadcastExchange (50) + : : : : : : : : : : : : +- * ColumnarToRow (49) + : : : : : : : : : : : : +- CometFilter (48) + : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (47) + : : : : : : : : : : : +- ReusedExchange (53) + : : : : : : : : : : +- BroadcastExchange (59) + : : : : : : : : : : +- * ColumnarToRow (58) + : : : : : : : : : : +- CometFilter (57) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (56) + : : : : : : : : : +- ReusedExchange (62) + : : : : : : : : +- BroadcastExchange (68) + : : : : : : : : +- * ColumnarToRow (67) + : : : : : : : : +- CometFilter (66) + : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (65) + : : : : : : : +- BroadcastExchange (74) + : : : : : : : +- * ColumnarToRow (73) + : : : : : : : +- CometFilter (72) + : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (71) + : : : : : : +- ReusedExchange (77) + : : : : : +- BroadcastExchange (83) + : : : : : +- * ColumnarToRow (82) + : : : : : +- CometFilter (81) + : : : : : +- CometScan parquet spark_catalog.default.customer_address (80) + : : : : +- ReusedExchange (86) + : : : +- BroadcastExchange (92) + : : : +- * ColumnarToRow (91) + : : : +- CometFilter (90) + : : : +- CometScan parquet spark_catalog.default.income_band (89) + : : +- ReusedExchange (95) + : +- BroadcastExchange (102) + : +- * ColumnarToRow (101) + : +- CometProject (100) + : +- CometFilter (99) + : +- CometScan parquet spark_catalog.default.item (98) + +- * Sort (175) + +- Exchange (174) + +- * HashAggregate (173) + +- * HashAggregate (172) + +- * Project (171) + +- * BroadcastHashJoin Inner BuildRight (170) + :- * Project (168) + : +- * BroadcastHashJoin Inner BuildRight (167) + : :- * Project (165) + : : +- * BroadcastHashJoin Inner BuildRight (164) + : : :- * Project (162) + : : : +- * BroadcastHashJoin Inner BuildRight (161) + : : : :- * Project (159) + : : : : +- * BroadcastHashJoin Inner BuildRight (158) + : : : : :- * Project (156) + : : : : : +- * BroadcastHashJoin Inner BuildRight (155) + : : : : : :- * Project (153) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (152) + : : : : : : :- * Project (150) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (149) + : : : : : : : :- * Project (147) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (146) + : : : : : : : : :- * Project (144) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (143) + : : : : : : : : : :- * Project (141) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (140) + : : : : : : : : : : :- * Project (138) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (137) + : : : : : : : : : : : :- * Project (135) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (134) + : : : : : : : : : : : : :- * Project (132) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (131) + : : : : : : : : : : : : : :- * Project (129) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (128) + : : : : : : : : : : : : : : :- * ColumnarToRow (126) + : : : : : : : : : : : : : : : +- CometProject (125) + : : : : : : : : : : : : : : : +- CometSortMergeJoin (124) + : : : : : : : : : : : : : : : :- CometSort (118) + : : : : : : : : : : : : : : : : +- CometExchange (117) + : : : : : : : : : : : : : : : : +- CometProject (116) + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (115) + : : : : : : : : : : : : : : : : :- CometBroadcastExchange (111) + : : : : : : : : : : : : : : : : : +- CometFilter (110) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (109) + : : : : : : : : : : : : : : : : +- CometProject (114) + : : : : : : : : : : : : : : : : +- CometFilter (113) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (112) + : : : : : : : : : : : : : : : +- CometSort (123) + : : : : : : : : : : : : : : : +- CometProject (122) + : : : : : : : : : : : : : : : +- CometFilter (121) + : : : : : : : : : : : : : : : +- CometHashAggregate (120) + : : : : : : : : : : : : : : : +- ReusedExchange (119) + : : : : : : : : : : : : : : +- ReusedExchange (127) + : : : : : : : : : : : : : +- ReusedExchange (130) + : : : : : : : : : : : : +- ReusedExchange (133) + : : : : : : : : : : : +- ReusedExchange (136) + : : : : : : : : : : +- ReusedExchange (139) + : : : : : : : : : +- ReusedExchange (142) + : : : : : : : : +- ReusedExchange (145) + : : : : : : : +- ReusedExchange (148) + : : : : : : +- ReusedExchange (151) + : : : : : +- ReusedExchange (154) + : : : : +- ReusedExchange (157) + : : : +- ReusedExchange (160) + : : +- ReusedExchange (163) + : +- ReusedExchange (166) + +- ReusedExchange (169) (unknown) Scan parquet spark_catalog.default.store_sales @@ -196,12 +192,9 @@ ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) -(7) CometProject +(6) CometProject Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] -(8) ColumnarToRow -Input [2]: [sr_item_sk#14, sr_ticket_number#15] +(7) CometBroadcastHashJoin +Left output [12]: [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_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Right output [2]: [sr_item_sk#14, sr_ticket_number#15] +Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner -(9) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [ss_item_sk#1, ss_ticket_number#8] -Right keys [2]: [sr_item_sk#14, sr_ticket_number#15] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 2] -Output [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] +(8) CometProject Input [14]: [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_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] +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, 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] -(11) Exchange +(9) CometExchange 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: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(12) Sort [codegen id : 3] +(10) CometSort 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 ASC NULLS FIRST], false, 0 +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] (unknown) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] @@ -246,19 +235,19 @@ Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] ReadSchema: struct -(14) CometFilter +(12) CometFilter Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) -(15) CometProject +(13) CometProject Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -(16) CometExchange +(14) CometExchange Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(17) CometSort +(15) CometSort Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] @@ -269,80 +258,79 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(19) CometFilter +(17) CometFilter Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) -(20) CometProject +(18) CometProject Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(21) CometExchange +(19) CometExchange Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(22) CometSort +(20) CometSort Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] -(23) CometSortMergeJoin +(21) CometSortMergeJoin Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] Right output [5]: [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_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner -(24) CometProject +(22) CometProject 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] -(25) CometHashAggregate +(23) CometHashAggregate Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Keys [1]: [cs_item_sk#17] Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] -(26) CometExchange +(24) CometExchange Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(27) CometHashAggregate +(25) CometHashAggregate Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] Keys [1]: [cs_item_sk#17] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] -(28) CometFilter +(26) CometFilter Input [3]: [cs_item_sk#17, sale#30, refund#31] Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) -(29) CometProject +(27) CometProject Input [3]: [cs_item_sk#17, sale#30, refund#31] Arguments: [cs_item_sk#17], [cs_item_sk#17] -(30) CometSort +(28) CometSort Input [1]: [cs_item_sk#17] Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] -(31) ColumnarToRow [codegen id : 4] -Input [1]: [cs_item_sk#17] +(29) CometSortMergeJoin +Left output [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] +Right output [1]: [cs_item_sk#17] +Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner -(32) SortMergeJoin [codegen id : 20] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [cs_item_sk#17] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 20] -Output [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] +(30) CometProject Input [12]: [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, cs_item_sk#17] +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, 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] + +(31) ColumnarToRow [codegen id : 16] +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] -(34) ReusedExchange [Reuses operator id: 187] +(32) ReusedExchange [Reuses operator id: 183] Output [2]: [d_date_sk#32, d_year#33] -(35) BroadcastHashJoin [codegen id : 20] +(33) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ss_sold_date_sk#12] Right keys [1]: [d_date_sk#32] Join type: Inner Join condition: None -(36) Project [codegen id : 20] +(34) Project [codegen id : 16] Output [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, d_year#33] Input [13]: [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, d_date_sk#32, d_year#33] @@ -353,24 +341,24 @@ Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_zip)] ReadSchema: struct -(38) CometFilter +(36) CometFilter Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(s_zip#36)) -(39) ColumnarToRow [codegen id : 6] +(37) ColumnarToRow [codegen id : 2] Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -(40) BroadcastExchange +(38) BroadcastExchange Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(41) BroadcastHashJoin [codegen id : 20] +(39) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ss_store_sk#6] Right keys [1]: [s_store_sk#34] Join type: Inner Join condition: None -(42) Project [codegen id : 20] +(40) Project [codegen id : 16] Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36] Input [14]: [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, d_year#33, s_store_sk#34, s_store_name#35, s_zip#36] @@ -381,24 +369,24 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(44) CometFilter +(42) CometFilter Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] Condition : (((((isnotnull(c_customer_sk#37) AND isnotnull(c_first_sales_date_sk#42)) AND isnotnull(c_first_shipto_date_sk#41)) AND isnotnull(c_current_cdemo_sk#38)) AND isnotnull(c_current_hdemo_sk#39)) AND isnotnull(c_current_addr_sk#40)) -(45) ColumnarToRow [codegen id : 7] +(43) ColumnarToRow [codegen id : 3] Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] -(46) BroadcastExchange +(44) BroadcastExchange Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(47) BroadcastHashJoin [codegen id : 20] +(45) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ss_customer_sk#2] Right keys [1]: [c_customer_sk#37] Join type: Inner Join condition: None -(48) Project [codegen id : 20] +(46) Project [codegen id : 16] Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] @@ -409,37 +397,37 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter +(48) CometFilter Input [2]: [d_date_sk#43, d_year#44] Condition : isnotnull(d_date_sk#43) -(51) ColumnarToRow [codegen id : 8] +(49) ColumnarToRow [codegen id : 4] Input [2]: [d_date_sk#43, d_year#44] -(52) BroadcastExchange +(50) BroadcastExchange Input [2]: [d_date_sk#43, d_year#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(53) BroadcastHashJoin [codegen id : 20] +(51) BroadcastHashJoin [codegen id : 16] Left keys [1]: [c_first_sales_date_sk#42] Right keys [1]: [d_date_sk#43] Join type: Inner Join condition: None -(54) Project [codegen id : 20] +(52) Project [codegen id : 16] Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44] Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42, d_date_sk#43, d_year#44] -(55) ReusedExchange [Reuses operator id: 52] +(53) ReusedExchange [Reuses operator id: 50] Output [2]: [d_date_sk#45, d_year#46] -(56) BroadcastHashJoin [codegen id : 20] +(54) BroadcastHashJoin [codegen id : 16] Left keys [1]: [c_first_shipto_date_sk#41] Right keys [1]: [d_date_sk#45] Join type: Inner Join condition: None -(57) Project [codegen id : 20] +(55) Project [codegen id : 16] Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44, d_date_sk#45, d_year#46] @@ -450,37 +438,37 @@ Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status)] ReadSchema: struct -(59) CometFilter +(57) CometFilter Input [2]: [cd_demo_sk#47, cd_marital_status#48] Condition : (isnotnull(cd_demo_sk#47) AND isnotnull(cd_marital_status#48)) -(60) ColumnarToRow [codegen id : 10] +(58) ColumnarToRow [codegen id : 6] Input [2]: [cd_demo_sk#47, cd_marital_status#48] -(61) BroadcastExchange +(59) BroadcastExchange Input [2]: [cd_demo_sk#47, cd_marital_status#48] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(62) BroadcastHashJoin [codegen id : 20] +(60) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ss_cdemo_sk#3] Right keys [1]: [cd_demo_sk#47] Join type: Inner Join condition: None -(63) Project [codegen id : 20] +(61) Project [codegen id : 16] Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48] Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_demo_sk#47, cd_marital_status#48] -(64) ReusedExchange [Reuses operator id: 61] +(62) ReusedExchange [Reuses operator id: 59] Output [2]: [cd_demo_sk#49, cd_marital_status#50] -(65) BroadcastHashJoin [codegen id : 20] +(63) BroadcastHashJoin [codegen id : 16] Left keys [1]: [c_current_cdemo_sk#38] Right keys [1]: [cd_demo_sk#49] Join type: Inner Join condition: NOT (cd_marital_status#48 = cd_marital_status#50) -(66) Project [codegen id : 20] +(64) Project [codegen id : 16] Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48, cd_demo_sk#49, cd_marital_status#50] @@ -491,24 +479,24 @@ Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(68) CometFilter +(66) CometFilter Input [1]: [p_promo_sk#51] Condition : isnotnull(p_promo_sk#51) -(69) ColumnarToRow [codegen id : 12] +(67) ColumnarToRow [codegen id : 8] Input [1]: [p_promo_sk#51] -(70) BroadcastExchange +(68) BroadcastExchange Input [1]: [p_promo_sk#51] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(71) BroadcastHashJoin [codegen id : 20] +(69) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ss_promo_sk#7] Right keys [1]: [p_promo_sk#51] Join type: Inner Join condition: None -(72) Project [codegen id : 20] +(70) Project [codegen id : 16] Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, p_promo_sk#51] @@ -519,37 +507,37 @@ Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] ReadSchema: struct -(74) CometFilter +(72) CometFilter Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] Condition : (isnotnull(hd_demo_sk#52) AND isnotnull(hd_income_band_sk#53)) -(75) ColumnarToRow [codegen id : 13] +(73) ColumnarToRow [codegen id : 9] Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] -(76) BroadcastExchange +(74) BroadcastExchange Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -(77) BroadcastHashJoin [codegen id : 20] +(75) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ss_hdemo_sk#4] Right keys [1]: [hd_demo_sk#52] Join type: Inner Join condition: None -(78) Project [codegen id : 20] +(76) Project [codegen id : 16] Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53] Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_demo_sk#52, hd_income_band_sk#53] -(79) ReusedExchange [Reuses operator id: 76] +(77) ReusedExchange [Reuses operator id: 74] Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -(80) BroadcastHashJoin [codegen id : 20] +(78) BroadcastHashJoin [codegen id : 16] Left keys [1]: [c_current_hdemo_sk#39] Right keys [1]: [hd_demo_sk#54] Join type: Inner Join condition: None -(81) Project [codegen id : 20] +(79) Project [codegen id : 16] Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55] Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_demo_sk#54, hd_income_band_sk#55] @@ -560,37 +548,37 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(83) CometFilter +(81) CometFilter Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] Condition : isnotnull(ca_address_sk#56) -(84) ColumnarToRow [codegen id : 15] +(82) ColumnarToRow [codegen id : 11] Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] -(85) BroadcastExchange +(83) BroadcastExchange Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] -(86) BroadcastHashJoin [codegen id : 20] +(84) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ss_addr_sk#5] Right keys [1]: [ca_address_sk#56] Join type: Inner Join condition: None -(87) Project [codegen id : 20] +(85) Project [codegen id : 16] Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] -(88) ReusedExchange [Reuses operator id: 85] +(86) ReusedExchange [Reuses operator id: 83] Output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(89) BroadcastHashJoin [codegen id : 20] +(87) BroadcastHashJoin [codegen id : 16] Left keys [1]: [c_current_addr_sk#40] Right keys [1]: [ca_address_sk#61] Join type: Inner Join condition: None -(90) Project [codegen id : 20] +(88) Project [codegen id : 16] Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] @@ -601,37 +589,37 @@ Location [not included in comparison]/{warehouse_dir}/income_band] PushedFilters: [IsNotNull(ib_income_band_sk)] ReadSchema: struct -(92) CometFilter +(90) CometFilter Input [1]: [ib_income_band_sk#66] Condition : isnotnull(ib_income_band_sk#66) -(93) ColumnarToRow [codegen id : 17] +(91) ColumnarToRow [codegen id : 13] Input [1]: [ib_income_band_sk#66] -(94) BroadcastExchange +(92) BroadcastExchange Input [1]: [ib_income_band_sk#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -(95) BroadcastHashJoin [codegen id : 20] +(93) BroadcastHashJoin [codegen id : 16] Left keys [1]: [hd_income_band_sk#53] Right keys [1]: [ib_income_band_sk#66] Join type: Inner Join condition: None -(96) Project [codegen id : 20] +(94) Project [codegen id : 16] Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ib_income_band_sk#66] -(97) ReusedExchange [Reuses operator id: 94] +(95) ReusedExchange [Reuses operator id: 92] Output [1]: [ib_income_band_sk#67] -(98) BroadcastHashJoin [codegen id : 20] +(96) BroadcastHashJoin [codegen id : 16] Left keys [1]: [hd_income_band_sk#55] Right keys [1]: [ib_income_band_sk#67] Join type: Inner Join condition: None -(99) Project [codegen id : 20] +(97) Project [codegen id : 16] Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ib_income_band_sk#67] @@ -642,50 +630,50 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), In(i_color, [burlywood ,floral ,indian ,medium ,purple ,spring ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct -(101) CometFilter +(99) CometFilter Input [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] Condition : ((((((isnotnull(i_current_price#69) AND i_color#70 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#69 >= 64.00)) AND (i_current_price#69 <= 74.00)) AND (i_current_price#69 >= 65.00)) AND (i_current_price#69 <= 79.00)) AND isnotnull(i_item_sk#68)) -(102) CometProject +(100) CometProject Input [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] Arguments: [i_item_sk#68, i_product_name#71], [i_item_sk#68, i_product_name#71] -(103) ColumnarToRow [codegen id : 19] +(101) ColumnarToRow [codegen id : 15] Input [2]: [i_item_sk#68, i_product_name#71] -(104) BroadcastExchange +(102) BroadcastExchange Input [2]: [i_item_sk#68, i_product_name#71] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -(105) BroadcastHashJoin [codegen id : 20] +(103) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#68] Join type: Inner Join condition: None -(106) Project [codegen id : 20] +(104) Project [codegen id : 16] Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#44, d_year#46, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] -(107) HashAggregate [codegen id : 20] +(105) HashAggregate [codegen id : 16] Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#44, d_year#46, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] Keys [15]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46] Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] Aggregate Attributes [4]: [count#72, sum#73, sum#74, sum#75] Results [19]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46, count#76, sum#77, sum#78, sum#79] -(108) HashAggregate [codegen id : 20] +(106) HashAggregate [codegen id : 16] Input [19]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46, count#76, sum#77, sum#78, sum#79] Keys [15]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46] Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] Aggregate Attributes [4]: [count(1)#80, sum(UnscaledValue(ss_wholesale_cost#9))#81, sum(UnscaledValue(ss_list_price#10))#82, sum(UnscaledValue(ss_coupon_amt#11))#83] Results [17]: [i_product_name#71 AS product_name#84, i_item_sk#68 AS item_sk#85, s_store_name#35 AS store_name#86, s_zip#36 AS store_zip#87, ca_street_number#57 AS b_street_number#88, ca_street_name#58 AS b_streen_name#89, ca_city#59 AS b_city#90, ca_zip#60 AS b_zip#91, ca_street_number#62 AS c_street_number#92, ca_street_name#63 AS c_street_name#93, ca_city#64 AS c_city#94, ca_zip#65 AS c_zip#95, d_year#33 AS syear#96, count(1)#80 AS cnt#97, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#81,17,2) AS s1#98, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#82,17,2) AS s2#99, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#83,17,2) AS s3#100] -(109) Exchange +(107) Exchange Input [17]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100] -Arguments: hashpartitioning(item_sk#85, store_name#86, store_zip#87, 5), ENSURE_REQUIREMENTS, [plan_id=15] +Arguments: hashpartitioning(item_sk#85, store_name#86, store_zip#87, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(110) Sort [codegen id : 21] +(108) Sort [codegen id : 17] Input [17]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100] Arguments: [item_sk#85 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, store_zip#87 ASC NULLS FIRST], false, 0 @@ -697,16 +685,13 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#112), dynamicpruningexpression(ss_s PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct -(112) CometFilter +(110) CometFilter Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] Condition : (((((((isnotnull(ss_item_sk#101) AND isnotnull(ss_ticket_number#108)) AND isnotnull(ss_store_sk#106)) AND isnotnull(ss_customer_sk#102)) AND isnotnull(ss_cdemo_sk#103)) AND isnotnull(ss_promo_sk#107)) AND isnotnull(ss_hdemo_sk#104)) AND isnotnull(ss_addr_sk#105)) -(113) ColumnarToRow [codegen id : 22] +(111) CometBroadcastExchange Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] - -(114) BroadcastExchange -Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=16] +Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] (unknown) Scan parquet spark_catalog.default.store_returns Output [3]: [sr_item_sk#114, sr_ticket_number#115, sr_returned_date_sk#116] @@ -715,310 +700,305 @@ Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct -(116) CometFilter +(113) CometFilter Input [3]: [sr_item_sk#114, sr_ticket_number#115, sr_returned_date_sk#116] Condition : (isnotnull(sr_item_sk#114) AND isnotnull(sr_ticket_number#115)) -(117) CometProject +(114) CometProject Input [3]: [sr_item_sk#114, sr_ticket_number#115, sr_returned_date_sk#116] Arguments: [sr_item_sk#114, sr_ticket_number#115], [sr_item_sk#114, sr_ticket_number#115] -(118) ColumnarToRow -Input [2]: [sr_item_sk#114, sr_ticket_number#115] +(115) CometBroadcastHashJoin +Left output [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +Right output [2]: [sr_item_sk#114, sr_ticket_number#115] +Arguments: [ss_item_sk#101, ss_ticket_number#108], [sr_item_sk#114, sr_ticket_number#115], Inner -(119) BroadcastHashJoin [codegen id : 23] -Left keys [2]: [ss_item_sk#101, ss_ticket_number#108] -Right keys [2]: [sr_item_sk#114, sr_ticket_number#115] -Join type: Inner -Join condition: None - -(120) Project [codegen id : 23] -Output [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +(116) CometProject Input [14]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112, sr_item_sk#114, sr_ticket_number#115] +Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112], [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -(121) Exchange +(117) CometExchange Input [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Arguments: hashpartitioning(ss_item_sk#101, 5), ENSURE_REQUIREMENTS, [plan_id=17] +Arguments: hashpartitioning(ss_item_sk#101, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] -(122) Sort [codegen id : 24] +(118) CometSort Input [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Arguments: [ss_item_sk#101 ASC NULLS FIRST], false, 0 +Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112], [ss_item_sk#101 ASC NULLS FIRST] -(123) ReusedExchange [Reuses operator id: 26] +(119) ReusedExchange [Reuses operator id: 24] Output [4]: [cs_item_sk#117, sum#118, sum#119, isEmpty#120] -(124) CometHashAggregate +(120) CometHashAggregate Input [4]: [cs_item_sk#117, sum#118, sum#119, isEmpty#120] Keys [1]: [cs_item_sk#117] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#121)), sum(((cr_refunded_cash#122 + cr_reversed_charge#123) + cr_store_credit#124))] -(125) CometFilter +(121) CometFilter Input [3]: [cs_item_sk#117, sale#30, refund#31] Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) -(126) CometProject +(122) CometProject Input [3]: [cs_item_sk#117, sale#30, refund#31] Arguments: [cs_item_sk#117], [cs_item_sk#117] -(127) CometSort +(123) CometSort Input [1]: [cs_item_sk#117] Arguments: [cs_item_sk#117], [cs_item_sk#117 ASC NULLS FIRST] -(128) ColumnarToRow [codegen id : 25] -Input [1]: [cs_item_sk#117] - -(129) SortMergeJoin [codegen id : 41] -Left keys [1]: [ss_item_sk#101] -Right keys [1]: [cs_item_sk#117] -Join type: Inner -Join condition: None +(124) CometSortMergeJoin +Left output [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +Right output [1]: [cs_item_sk#117] +Arguments: [ss_item_sk#101], [cs_item_sk#117], Inner -(130) Project [codegen id : 41] -Output [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +(125) CometProject Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112, cs_item_sk#117] +Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112], [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -(131) ReusedExchange [Reuses operator id: 191] +(126) ColumnarToRow [codegen id : 33] +Input [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] + +(127) ReusedExchange [Reuses operator id: 187] Output [2]: [d_date_sk#125, d_year#126] -(132) BroadcastHashJoin [codegen id : 41] +(128) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_sold_date_sk#112] Right keys [1]: [d_date_sk#125] Join type: Inner Join condition: None -(133) Project [codegen id : 41] +(129) Project [codegen id : 33] Output [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126] Input [13]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112, d_date_sk#125, d_year#126] -(134) ReusedExchange [Reuses operator id: 40] +(130) ReusedExchange [Reuses operator id: 38] Output [3]: [s_store_sk#127, s_store_name#128, s_zip#129] -(135) BroadcastHashJoin [codegen id : 41] +(131) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_store_sk#106] Right keys [1]: [s_store_sk#127] Join type: Inner Join condition: None -(136) Project [codegen id : 41] +(132) Project [codegen id : 33] Output [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129] Input [14]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_sk#127, s_store_name#128, s_zip#129] -(137) ReusedExchange [Reuses operator id: 46] +(133) ReusedExchange [Reuses operator id: 44] Output [6]: [c_customer_sk#130, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135] -(138) BroadcastHashJoin [codegen id : 41] +(134) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_customer_sk#102] Right keys [1]: [c_customer_sk#130] Join type: Inner Join condition: None -(139) Project [codegen id : 41] +(135) Project [codegen id : 33] Output [16]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135] Input [18]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_customer_sk#130, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135] -(140) ReusedExchange [Reuses operator id: 52] +(136) ReusedExchange [Reuses operator id: 50] Output [2]: [d_date_sk#136, d_year#137] -(141) BroadcastHashJoin [codegen id : 41] +(137) BroadcastHashJoin [codegen id : 33] Left keys [1]: [c_first_sales_date_sk#135] Right keys [1]: [d_date_sk#136] Join type: Inner Join condition: None -(142) Project [codegen id : 41] +(138) Project [codegen id : 33] Output [16]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, d_year#137] Input [18]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135, d_date_sk#136, d_year#137] -(143) ReusedExchange [Reuses operator id: 52] +(139) ReusedExchange [Reuses operator id: 50] Output [2]: [d_date_sk#138, d_year#139] -(144) BroadcastHashJoin [codegen id : 41] +(140) BroadcastHashJoin [codegen id : 33] Left keys [1]: [c_first_shipto_date_sk#134] Right keys [1]: [d_date_sk#138] Join type: Inner Join condition: None -(145) Project [codegen id : 41] +(141) Project [codegen id : 33] Output [16]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139] Input [18]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, d_year#137, d_date_sk#138, d_year#139] -(146) ReusedExchange [Reuses operator id: 61] +(142) ReusedExchange [Reuses operator id: 59] Output [2]: [cd_demo_sk#140, cd_marital_status#141] -(147) BroadcastHashJoin [codegen id : 41] +(143) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_cdemo_sk#103] Right keys [1]: [cd_demo_sk#140] Join type: Inner Join condition: None -(148) Project [codegen id : 41] +(144) Project [codegen id : 33] Output [16]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, cd_marital_status#141] Input [18]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, cd_demo_sk#140, cd_marital_status#141] -(149) ReusedExchange [Reuses operator id: 61] +(145) ReusedExchange [Reuses operator id: 59] Output [2]: [cd_demo_sk#142, cd_marital_status#143] -(150) BroadcastHashJoin [codegen id : 41] +(146) BroadcastHashJoin [codegen id : 33] Left keys [1]: [c_current_cdemo_sk#131] Right keys [1]: [cd_demo_sk#142] Join type: Inner Join condition: NOT (cd_marital_status#141 = cd_marital_status#143) -(151) Project [codegen id : 41] +(147) Project [codegen id : 33] Output [14]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139] Input [18]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, cd_marital_status#141, cd_demo_sk#142, cd_marital_status#143] -(152) ReusedExchange [Reuses operator id: 70] +(148) ReusedExchange [Reuses operator id: 68] Output [1]: [p_promo_sk#144] -(153) BroadcastHashJoin [codegen id : 41] +(149) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_promo_sk#107] Right keys [1]: [p_promo_sk#144] Join type: Inner Join condition: None -(154) Project [codegen id : 41] +(150) Project [codegen id : 33] Output [13]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139] Input [15]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, p_promo_sk#144] -(155) ReusedExchange [Reuses operator id: 76] +(151) ReusedExchange [Reuses operator id: 74] Output [2]: [hd_demo_sk#145, hd_income_band_sk#146] -(156) BroadcastHashJoin [codegen id : 41] +(152) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_hdemo_sk#104] Right keys [1]: [hd_demo_sk#145] Join type: Inner Join condition: None -(157) Project [codegen id : 41] +(153) Project [codegen id : 33] Output [13]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146] Input [15]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, hd_demo_sk#145, hd_income_band_sk#146] -(158) ReusedExchange [Reuses operator id: 76] +(154) ReusedExchange [Reuses operator id: 74] Output [2]: [hd_demo_sk#147, hd_income_band_sk#148] -(159) BroadcastHashJoin [codegen id : 41] +(155) BroadcastHashJoin [codegen id : 33] Left keys [1]: [c_current_hdemo_sk#132] Right keys [1]: [hd_demo_sk#147] Join type: Inner Join condition: None -(160) Project [codegen id : 41] +(156) Project [codegen id : 33] Output [13]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148] Input [15]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_demo_sk#147, hd_income_band_sk#148] -(161) ReusedExchange [Reuses operator id: 85] +(157) ReusedExchange [Reuses operator id: 83] Output [5]: [ca_address_sk#149, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153] -(162) BroadcastHashJoin [codegen id : 41] +(158) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_addr_sk#105] Right keys [1]: [ca_address_sk#149] Join type: Inner Join condition: None -(163) Project [codegen id : 41] +(159) Project [codegen id : 33] Output [16]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153] Input [18]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_address_sk#149, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153] -(164) ReusedExchange [Reuses operator id: 85] +(160) ReusedExchange [Reuses operator id: 83] Output [5]: [ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -(165) BroadcastHashJoin [codegen id : 41] +(161) BroadcastHashJoin [codegen id : 33] Left keys [1]: [c_current_addr_sk#133] Right keys [1]: [ca_address_sk#154] Join type: Inner Join condition: None -(166) Project [codegen id : 41] +(162) Project [codegen id : 33] Output [19]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] Input [21]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -(167) ReusedExchange [Reuses operator id: 94] +(163) ReusedExchange [Reuses operator id: 92] Output [1]: [ib_income_band_sk#159] -(168) BroadcastHashJoin [codegen id : 41] +(164) BroadcastHashJoin [codegen id : 33] Left keys [1]: [hd_income_band_sk#146] Right keys [1]: [ib_income_band_sk#159] Join type: Inner Join condition: None -(169) Project [codegen id : 41] +(165) Project [codegen id : 33] Output [18]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] Input [20]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ib_income_band_sk#159] -(170) ReusedExchange [Reuses operator id: 94] +(166) ReusedExchange [Reuses operator id: 92] Output [1]: [ib_income_band_sk#160] -(171) BroadcastHashJoin [codegen id : 41] +(167) BroadcastHashJoin [codegen id : 33] Left keys [1]: [hd_income_band_sk#148] Right keys [1]: [ib_income_band_sk#160] Join type: Inner Join condition: None -(172) Project [codegen id : 41] +(168) Project [codegen id : 33] Output [17]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] Input [19]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ib_income_band_sk#160] -(173) ReusedExchange [Reuses operator id: 104] +(169) ReusedExchange [Reuses operator id: 102] Output [2]: [i_item_sk#161, i_product_name#162] -(174) BroadcastHashJoin [codegen id : 41] +(170) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_item_sk#101] Right keys [1]: [i_item_sk#161] Join type: Inner Join condition: None -(175) Project [codegen id : 41] +(171) Project [codegen id : 33] Output [18]: [ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, d_year#137, d_year#139, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, i_item_sk#161, i_product_name#162] Input [19]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, i_item_sk#161, i_product_name#162] -(176) HashAggregate [codegen id : 41] +(172) HashAggregate [codegen id : 33] Input [18]: [ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, d_year#137, d_year#139, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, i_item_sk#161, i_product_name#162] Keys [15]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139] Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#109)), partial_sum(UnscaledValue(ss_list_price#110)), partial_sum(UnscaledValue(ss_coupon_amt#111))] Aggregate Attributes [4]: [count#72, sum#163, sum#164, sum#165] Results [19]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139, count#76, sum#166, sum#167, sum#168] -(177) HashAggregate [codegen id : 41] +(173) HashAggregate [codegen id : 33] Input [19]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139, count#76, sum#166, sum#167, sum#168] Keys [15]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139] Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#109)), sum(UnscaledValue(ss_list_price#110)), sum(UnscaledValue(ss_coupon_amt#111))] Aggregate Attributes [4]: [count(1)#80, sum(UnscaledValue(ss_wholesale_cost#109))#81, sum(UnscaledValue(ss_list_price#110))#82, sum(UnscaledValue(ss_coupon_amt#111))#83] Results [8]: [i_item_sk#161 AS item_sk#169, s_store_name#128 AS store_name#170, s_zip#129 AS store_zip#171, d_year#126 AS syear#172, count(1)#80 AS cnt#173, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#109))#81,17,2) AS s1#174, MakeDecimal(sum(UnscaledValue(ss_list_price#110))#82,17,2) AS s2#175, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#111))#83,17,2) AS s3#176] -(178) Exchange +(174) Exchange Input [8]: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] -Arguments: hashpartitioning(item_sk#169, store_name#170, store_zip#171, 5), ENSURE_REQUIREMENTS, [plan_id=18] +Arguments: hashpartitioning(item_sk#169, store_name#170, store_zip#171, 5), ENSURE_REQUIREMENTS, [plan_id=16] -(179) Sort [codegen id : 42] +(175) Sort [codegen id : 34] Input [8]: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] Arguments: [item_sk#169 ASC NULLS FIRST, store_name#170 ASC NULLS FIRST, store_zip#171 ASC NULLS FIRST], false, 0 -(180) SortMergeJoin [codegen id : 43] +(176) SortMergeJoin [codegen id : 35] Left keys [3]: [item_sk#85, store_name#86, store_zip#87] Right keys [3]: [item_sk#169, store_name#170, store_zip#171] Join type: Inner Join condition: (cnt#173 <= cnt#97) -(181) Project [codegen id : 43] +(177) Project [codegen id : 35] Output [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] Input [25]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] -(182) Exchange +(178) Exchange Input [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] -Arguments: rangepartitioning(product_name#84 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, cnt#173 ASC NULLS FIRST, s1#98 ASC NULLS FIRST, s1#174 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=19] +Arguments: rangepartitioning(product_name#84 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, cnt#173 ASC NULLS FIRST, s1#98 ASC NULLS FIRST, s1#174 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=17] -(183) Sort [codegen id : 44] +(179) Sort [codegen id : 36] Input [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] Arguments: [product_name#84 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, cnt#173 ASC NULLS FIRST, s1#98 ASC NULLS FIRST, s1#174 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (187) -+- * ColumnarToRow (186) - +- CometFilter (185) - +- CometScan parquet spark_catalog.default.date_dim (184) +BroadcastExchange (183) ++- * ColumnarToRow (182) + +- CometFilter (181) + +- CometScan parquet spark_catalog.default.date_dim (180) (unknown) Scan parquet spark_catalog.default.date_dim @@ -1028,22 +1008,22 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(185) CometFilter +(181) CometFilter 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)) -(186) ColumnarToRow [codegen id : 1] +(182) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] -(187) BroadcastExchange +(183) BroadcastExchange Input [2]: [d_date_sk#32, d_year#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] -Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#112 IN dynamicpruning#113 -BroadcastExchange (191) -+- * ColumnarToRow (190) - +- CometFilter (189) - +- CometScan parquet spark_catalog.default.date_dim (188) +Subquery:2 Hosting operator id = 109 Hosting Expression = ss_sold_date_sk#112 IN dynamicpruning#113 +BroadcastExchange (187) ++- * ColumnarToRow (186) + +- CometFilter (185) + +- CometScan parquet spark_catalog.default.date_dim (184) (unknown) Scan parquet spark_catalog.default.date_dim @@ -1053,15 +1033,15 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(189) CometFilter +(185) CometFilter Input [2]: [d_date_sk#125, d_year#126] Condition : ((isnotnull(d_year#126) AND (d_year#126 = 2000)) AND isnotnull(d_date_sk#125)) -(190) ColumnarToRow [codegen id : 1] +(186) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#125, d_year#126] -(191) BroadcastExchange +(187) BroadcastExchange Input [2]: [d_date_sk#125, d_year#126] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] 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 4aa8376d1..e2accd741 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,16 +1,16 @@ -WholeStageCodegen (44) +WholeStageCodegen (36) Sort [product_name,store_name,cnt,s1,s1] InputAdapter Exchange [product_name,store_name,cnt,s1,s1] #1 - WholeStageCodegen (43) + WholeStageCodegen (35) Project [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] SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] InputAdapter - WholeStageCodegen (21) + WholeStageCodegen (17) Sort [item_sk,store_name,store_zip] InputAdapter Exchange [item_sk,store_name,store_zip] #2 - WholeStageCodegen (20) + WholeStageCodegen (16) HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,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,count,sum,sum,sum] HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] @@ -43,76 +43,64 @@ WholeStageCodegen (44) BroadcastHashJoin [ss_store_sk,s_store_sk] Project [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,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - SortMergeJoin [ss_item_sk,cs_item_sk] - InputAdapter - WholeStageCodegen (3) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #3 - WholeStageCodegen (2) - Project [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] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [sale,refund] - CometHashAggregate [cs_item_sk,sum,sum,isEmpty] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + ColumnarToRow + InputAdapter + CometProject [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] + CometSortMergeJoin [ss_item_sk,cs_item_sk] + CometSort [ss_item_sk] + CometExchange [ss_item_sk] #3 + CometProject [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] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + CometBroadcastExchange #4 + CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [sale,refund] + CometHashAggregate [cs_item_sk,sum,sum,isEmpty] + CometExchange [cs_item_sk] #6 + CometHashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometSortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + CometSort [cs_item_sk,cs_order_number] + CometExchange [cs_item_sk,cs_order_number] #7 + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometSort [cr_item_sk,cr_order_number] + CometExchange [cr_item_sk,cr_order_number] #8 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #5 InputAdapter BroadcastExchange #9 - WholeStageCodegen (6) + WholeStageCodegen (2) ColumnarToRow 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 (7) + WholeStageCodegen (3) ColumnarToRow InputAdapter CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_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 (8) + WholeStageCodegen (4) ColumnarToRow InputAdapter CometFilter [d_date_sk] @@ -121,7 +109,7 @@ WholeStageCodegen (44) ReusedExchange [d_date_sk,d_year] #11 InputAdapter BroadcastExchange #12 - WholeStageCodegen (10) + WholeStageCodegen (6) ColumnarToRow InputAdapter CometFilter [cd_demo_sk,cd_marital_status] @@ -130,14 +118,14 @@ WholeStageCodegen (44) ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter BroadcastExchange #13 - WholeStageCodegen (12) + WholeStageCodegen (8) ColumnarToRow InputAdapter CometFilter [p_promo_sk] CometScan parquet spark_catalog.default.promotion [p_promo_sk] InputAdapter BroadcastExchange #14 - WholeStageCodegen (13) + WholeStageCodegen (9) ColumnarToRow InputAdapter CometFilter [hd_demo_sk,hd_income_band_sk] @@ -146,7 +134,7 @@ WholeStageCodegen (44) ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter BroadcastExchange #15 - WholeStageCodegen (15) + WholeStageCodegen (11) ColumnarToRow InputAdapter CometFilter [ca_address_sk] @@ -155,7 +143,7 @@ WholeStageCodegen (44) ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter BroadcastExchange #16 - WholeStageCodegen (17) + WholeStageCodegen (13) ColumnarToRow InputAdapter CometFilter [ib_income_band_sk] @@ -164,18 +152,18 @@ WholeStageCodegen (44) ReusedExchange [ib_income_band_sk] #16 InputAdapter BroadcastExchange #17 - WholeStageCodegen (19) + WholeStageCodegen (15) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] CometFilter [i_current_price,i_color,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter - WholeStageCodegen (42) + WholeStageCodegen (34) Sort [item_sk,store_name,store_zip] InputAdapter Exchange [item_sk,store_name,store_zip] #18 - WholeStageCodegen (41) + WholeStageCodegen (33) HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] @@ -208,44 +196,32 @@ WholeStageCodegen (44) BroadcastHashJoin [ss_store_sk,s_store_sk] Project [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,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - SortMergeJoin [ss_item_sk,cs_item_sk] - InputAdapter - WholeStageCodegen (24) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #19 - WholeStageCodegen (23) - Project [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] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #20 - WholeStageCodegen (22) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #21 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - WholeStageCodegen (25) - ColumnarToRow - InputAdapter - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [sale,refund] - CometHashAggregate [cs_item_sk,sum,sum,isEmpty] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 + ColumnarToRow + InputAdapter + CometProject [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] + CometSortMergeJoin [ss_item_sk,cs_item_sk] + CometSort [ss_item_sk] + CometExchange [ss_item_sk] #19 + CometProject [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] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + CometBroadcastExchange #20 + CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #21 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [sale,refund] + CometHashAggregate [cs_item_sk,sum,sum,isEmpty] + ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 InputAdapter ReusedExchange [d_date_sk,d_year] #21 InputAdapter diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index 317371fb9..e8d340f21 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -47,7 +47,7 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { castTest(generateFloats, DataTypes.StringType) } - ignore("cast string to bool") { + test("cast string to bool") { castTest( Seq("TRUE", "True", "true", "FALSE", "False", "false", "1", "0", "").toDF("a"), DataTypes.BooleanType) 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 4172c7caa..0bb21aba7 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStatistics, CatalogTable} import org.apache.spark.sql.catalyst.expressions.Hex import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateMode -import org.apache.spark.sql.comet.{CometBroadcastExchangeExec, CometCollectLimitExec, CometFilterExec, CometHashAggregateExec, CometProjectExec, CometScanExec, CometTakeOrderedAndProjectExec} +import org.apache.spark.sql.comet.{CometBroadcastExchangeExec, CometCollectLimitExec, CometFilterExec, CometHashAggregateExec, CometProjectExec, CometRowToColumnarExec, CometScanExec, CometTakeOrderedAndProjectExec} import org.apache.spark.sql.comet.execution.shuffle.{CometColumnarShuffle, CometShuffleExchangeExec} import org.apache.spark.sql.execution.{CollectLimitExec, ProjectExec, SQLExecution, UnionExec} import org.apache.spark.sql.execution.exchange.BroadcastExchangeExec @@ -73,7 +73,7 @@ class CometExecSuite extends CometTestBase { test("CometBroadcastExchangeExec") { assume(isSpark34Plus, "ChunkedByteBuffer is not serializable before Spark 3.4+") - withSQLConf(CometConf.COMET_EXEC_BROADCAST_ENABLED.key -> "true") { + withSQLConf(CometConf.COMET_EXEC_BROADCAST_FORCE_ENABLED.key -> "true") { withParquetTable((0 until 5).map(i => (i, i + 1)), "tbl_a") { withParquetTable((0 until 5).map(i => (i, i + 1)), "tbl_b") { val df = sql( @@ -99,7 +99,7 @@ class CometExecSuite extends CometTestBase { test("CometBroadcastExchangeExec: empty broadcast") { assume(isSpark34Plus, "ChunkedByteBuffer is not serializable before Spark 3.4+") - withSQLConf(CometConf.COMET_EXEC_BROADCAST_ENABLED.key -> "true") { + withSQLConf(CometConf.COMET_EXEC_BROADCAST_FORCE_ENABLED.key -> "true") { withParquetTable((0 until 5).map(i => (i, i + 1)), "tbl_a") { withParquetTable((0 until 5).map(i => (i, i + 1)), "tbl_b") { val df = sql( @@ -1118,6 +1118,58 @@ class CometExecSuite extends CometTestBase { } }) } + + test("RowToColumnar over RangeExec") { + Seq("true", "false").foreach(aqe => { + Seq(500, 900).foreach { batchSize => + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqe, + SQLConf.ARROW_EXECUTION_MAX_RECORDS_PER_BATCH.key -> batchSize.toString) { + val df = spark.range(1000).selectExpr("id", "id % 8 as k").groupBy("k").sum("id") + checkSparkAnswerAndOperator(df) + // empty record batch should also be handled + val df2 = spark.range(0).selectExpr("id", "id % 8 as k").groupBy("k").sum("id") + checkSparkAnswerAndOperator(df2, includeClasses = Seq(classOf[CometRowToColumnarExec])) + } + } + }) + } + + test("RowToColumnar over RangeExec directly is eliminated for row output") { + Seq("true", "false").foreach(aqe => { + Seq(500, 900).foreach { batchSize => + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqe, + SQLConf.ARROW_EXECUTION_MAX_RECORDS_PER_BATCH.key -> batchSize.toString) { + val df = spark.range(1000) + val qe = df.queryExecution + qe.executedPlan.collectFirst({ case r: CometRowToColumnarExec => r }) match { + case Some(_) => fail("CometRowToColumnarExec should be eliminated") + case _ => + } + } + } + }) + } + + test("RowToColumnar over InMemoryTableScanExec") { + Seq("true", "false").foreach(aqe => { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqe, + CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true", + SQLConf.CACHE_VECTORIZED_READER_ENABLED.key -> "false") { + spark + .range(1000) + .selectExpr("id as key", "id % 8 as value") + .toDF("key", "value") + .selectExpr("key", "value", "key+1") + .createOrReplaceTempView("abc") + spark.catalog.cacheTable("abc") + val df = spark.sql("SELECT * FROM abc").groupBy("key").count() + checkSparkAnswerAndOperator(df, includeClasses = Seq(classOf[CometRowToColumnarExec])) + } + }) + } } case class BucketedTableTestSpec( 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 6f479e3bb..54c0baf16 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala @@ -45,7 +45,6 @@ class CometJoinSuite extends CometTestBase { withSQLConf( CometConf.COMET_BATCH_SIZE.key -> "100", SQLConf.PREFER_SORTMERGEJOIN.key -> "false", - "spark.comet.exec.broadcast.enabled" -> "true", "spark.sql.join.forceApplyShuffledHashJoin" -> "true", SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { @@ -74,7 +73,6 @@ class CometJoinSuite extends CometTestBase { withSQLConf( CometConf.COMET_BATCH_SIZE.key -> "100", SQLConf.PREFER_SORTMERGEJOIN.key -> "false", - "spark.comet.exec.broadcast.enabled" -> "true", "spark.sql.join.forceApplyShuffledHashJoin" -> "true", SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala b/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala index 265235ffe..a3b73dfa0 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala @@ -138,9 +138,9 @@ class CometTPCDSQuerySuite "q99") // TODO: enable the 3 queries after fixing the issues #1358. - override val tpcdsQueries: Seq[String] = - tpcdsAllQueries.filterNot(excludedTpcdsQueries.contains) - + override val tpcdsQueries: Seq[String] = Seq("q4") + // tpcdsAllQueries.filterNot(excludedTpcdsQueries.contains) + // Seq("q1", "q2", "q3", "q4") } with TPCDSQueryTestSuite { override def sparkConf: SparkConf = { 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 6fb81bc43..de5866580 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala @@ -34,7 +34,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} +import org.apache.spark.sql.comet.{CometBatchScanExec, CometBroadcastExchangeExec, CometExec, CometRowToColumnarExec, CometScanExec, CometScanWrapper, CometSinkPlaceHolder} import org.apache.spark.sql.comet.execution.shuffle.{CometColumnarShuffle, CometNativeShuffle, CometShuffleExchangeExec} import org.apache.spark.sql.execution.{ColumnarToRowExec, InputAdapter, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper @@ -75,6 +75,7 @@ abstract class CometTestBase conf.set(CometConf.COMET_EXEC_ENABLED.key, "true") conf.set(CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key, "true") conf.set(CometConf.COMET_EXEC_ALL_EXPR_ENABLED.key, "true") + conf.set(CometConf.COMET_ROW_TO_COLUMNAR_ENABLED.key, "true") conf.set(CometConf.COMET_MEMORY_OVERHEAD.key, "2g") conf } @@ -155,9 +156,11 @@ abstract class CometTestBase } protected def checkCometOperators(plan: SparkPlan, excludedClasses: Class[_]*): Unit = { - plan.foreach { + val wrapped = wrapCometRowToColumnar(plan) + wrapped.foreach { case _: CometScanExec | _: CometBatchScanExec => true case _: CometSinkPlaceHolder | _: CometScanWrapper => false + case _: CometRowToColumnarExec => false case _: CometExec | _: CometShuffleExchangeExec => true case _: CometBroadcastExchangeExec => true case _: WholeStageCodegenExec | _: ColumnarToRowExec | _: InputAdapter => true @@ -184,6 +187,14 @@ abstract class CometTestBase } } + /** Wraps the CometRowToColumn as ScanWrapper, so the child operators will not be checked */ + private def wrapCometRowToColumnar(plan: SparkPlan): SparkPlan = { + plan.transformDown { + // don't care the native operators + case p: CometRowToColumnarExec => CometScanWrapper(null, p) + } + } + /** * Check the answer of a Comet SQL query with Spark result using absolute tolerance. */