From 27288a08d8f0c5e23ae1dc3c572f15517daf6cec Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 29 Jun 2024 09:27:21 -0700 Subject: [PATCH 01/16] fix: Fix range out of index error with a temporary workaround (#584) * fix: Fix range out of index error by using custom arrow-rs repo * Add custom Java Arrow classes * Add a hack * Update * Update * Update to use https://github.com/apache/arrow-rs/pull/5958 * Use tustvold's branch * Use official arrow-rs repo --- .../org/apache/arrow/c/ArrowImporter.java | 3 +- .../apache/arrow/c/CometArrayImporter.java | 152 +++++++ .../arrow/c/CometBufferImportTypeVisitor.java | 398 ++++++++++++++++++ core/Cargo.lock | 81 ++-- core/Cargo.toml | 26 +- 5 files changed, 592 insertions(+), 68 deletions(-) create mode 100644 common/src/main/java/org/apache/arrow/c/CometArrayImporter.java create mode 100644 common/src/main/java/org/apache/arrow/c/CometBufferImportTypeVisitor.java diff --git a/common/src/main/java/org/apache/arrow/c/ArrowImporter.java b/common/src/main/java/org/apache/arrow/c/ArrowImporter.java index 90398cb72..1f0cbd412 100644 --- a/common/src/main/java/org/apache/arrow/c/ArrowImporter.java +++ b/common/src/main/java/org/apache/arrow/c/ArrowImporter.java @@ -55,7 +55,8 @@ public FieldVector importVector( ArrowArray array, ArrowSchema schema, CDataDictionaryProvider provider) { Field field = importField(schema, provider); FieldVector vector = field.createVector(allocator); - Data.importIntoVector(allocator, array, vector, provider); + CometArrayImporter importer = new CometArrayImporter(allocator, vector, provider); + importer.importArray(array); return vector; } } diff --git a/common/src/main/java/org/apache/arrow/c/CometArrayImporter.java b/common/src/main/java/org/apache/arrow/c/CometArrayImporter.java new file mode 100644 index 000000000..119055b5f --- /dev/null +++ b/common/src/main/java/org/apache/arrow/c/CometArrayImporter.java @@ -0,0 +1,152 @@ +/* + * 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.arrow.c; + +import java.util.Collections; +import java.util.List; + +import org.apache.arrow.memory.ArrowBuf; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.util.Preconditions; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.dictionary.Dictionary; +import org.apache.arrow.vector.dictionary.DictionaryProvider; +import org.apache.arrow.vector.ipc.message.ArrowFieldNode; +import org.apache.arrow.vector.types.pojo.DictionaryEncoding; + +import static org.apache.arrow.c.NativeUtil.NULL; +import static org.apache.arrow.memory.util.LargeMemoryUtil.checkedCastToInt; +import static org.apache.arrow.util.Preconditions.checkNotNull; +import static org.apache.arrow.util.Preconditions.checkState; + +/** + * Importer for {@link ArrowArray}. We copy it from Arrow `ArrayImporter` because we need to use + * `CometBufferImportTypeVisitor` instead of Arrow `BufferImportTypeVisitor`. + */ +final class CometArrayImporter { + private static final int MAX_IMPORT_RECURSION_LEVEL = 64; + + private final BufferAllocator allocator; + private final FieldVector vector; + private final DictionaryProvider dictionaryProvider; + + private ReferenceCountedArrowArray underlyingAllocation; + private int recursionLevel; + + CometArrayImporter( + BufferAllocator allocator, FieldVector vector, DictionaryProvider dictionaryProvider) { + this.allocator = Preconditions.checkNotNull(allocator); + this.vector = Preconditions.checkNotNull(vector); + this.dictionaryProvider = dictionaryProvider; + } + + void importArray(ArrowArray src) { + ArrowArray.Snapshot snapshot = src.snapshot(); + checkState(snapshot.release != NULL, "Cannot import released ArrowArray"); + + // Move imported array + ArrowArray ownedArray = ArrowArray.allocateNew(allocator); + ownedArray.save(snapshot); + src.markReleased(); + src.close(); + + recursionLevel = 0; + + // This keeps the array alive as long as there are any buffers that need it + underlyingAllocation = new ReferenceCountedArrowArray(ownedArray); + try { + doImport(snapshot); + } finally { + underlyingAllocation.release(); + } + } + + private void importChild(CometArrayImporter parent, ArrowArray src) { + ArrowArray.Snapshot snapshot = src.snapshot(); + checkState(snapshot.release != NULL, "Cannot import released ArrowArray"); + recursionLevel = parent.recursionLevel + 1; + checkState( + recursionLevel <= MAX_IMPORT_RECURSION_LEVEL, + "Recursion level in ArrowArray struct exceeded"); + // Child buffers will keep the entire parent import alive. + underlyingAllocation = parent.underlyingAllocation; + doImport(snapshot); + } + + private void doImport(ArrowArray.Snapshot snapshot) { + // First import children (required for reconstituting parent array data) + long[] children = + NativeUtil.toJavaArray(snapshot.children, checkedCastToInt(snapshot.n_children)); + if (children != null && children.length > 0) { + List childVectors = vector.getChildrenFromFields(); + checkState( + children.length == childVectors.size(), + "ArrowArray struct has %s children (expected %s)", + children.length, + childVectors.size()); + for (int i = 0; i < children.length; i++) { + checkState(children[i] != NULL, "ArrowArray struct has NULL child at position %s", i); + CometArrayImporter childImporter = + new CometArrayImporter(allocator, childVectors.get(i), dictionaryProvider); + childImporter.importChild(this, ArrowArray.wrap(children[i])); + } + } + + // Handle import of a dictionary encoded vector + if (snapshot.dictionary != NULL) { + DictionaryEncoding encoding = vector.getField().getDictionary(); + checkNotNull(encoding, "Missing encoding on import of ArrowArray with dictionary"); + + Dictionary dictionary = dictionaryProvider.lookup(encoding.getId()); + checkNotNull(dictionary, "Dictionary lookup failed on import of ArrowArray with dictionary"); + + // reset the dictionary vector to the initial state + dictionary.getVector().clear(); + + CometArrayImporter dictionaryImporter = + new CometArrayImporter(allocator, dictionary.getVector(), dictionaryProvider); + dictionaryImporter.importChild(this, ArrowArray.wrap(snapshot.dictionary)); + } + + // Import main data + ArrowFieldNode fieldNode = new ArrowFieldNode(snapshot.length, snapshot.null_count); + long[] bufferPointers = + NativeUtil.toJavaArray(snapshot.buffers, checkedCastToInt(snapshot.n_buffers)); + + try (final CometBufferImportTypeVisitor visitor = + new CometBufferImportTypeVisitor( + allocator, underlyingAllocation, fieldNode, snapshot, bufferPointers)) { + final List buffers; + if (bufferPointers == null || bufferPointers.length == 0) { + buffers = Collections.emptyList(); + } else { + buffers = vector.getField().getType().accept(visitor); + } + vector.loadFieldBuffers(fieldNode, buffers); + } catch (Exception e) { + throw new IllegalArgumentException( + "Could not load buffers for field " + + vector.getField() + + ". error message: " + + e.getMessage(), + e); + } + } +} diff --git a/common/src/main/java/org/apache/arrow/c/CometBufferImportTypeVisitor.java b/common/src/main/java/org/apache/arrow/c/CometBufferImportTypeVisitor.java new file mode 100644 index 000000000..b80e6b7f2 --- /dev/null +++ b/common/src/main/java/org/apache/arrow/c/CometBufferImportTypeVisitor.java @@ -0,0 +1,398 @@ +/* + * 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.arrow.c; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import org.apache.arrow.memory.ArrowBuf; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.util.AutoCloseables; +import org.apache.arrow.util.VisibleForTesting; +import org.apache.arrow.vector.DateDayVector; +import org.apache.arrow.vector.DateMilliVector; +import org.apache.arrow.vector.DurationVector; +import org.apache.arrow.vector.Float4Vector; +import org.apache.arrow.vector.Float8Vector; +import org.apache.arrow.vector.IntervalDayVector; +import org.apache.arrow.vector.IntervalMonthDayNanoVector; +import org.apache.arrow.vector.IntervalYearVector; +import org.apache.arrow.vector.LargeVarBinaryVector; +import org.apache.arrow.vector.LargeVarCharVector; +import org.apache.arrow.vector.TimeMicroVector; +import org.apache.arrow.vector.TimeMilliVector; +import org.apache.arrow.vector.TimeNanoVector; +import org.apache.arrow.vector.TimeSecVector; +import org.apache.arrow.vector.TimeStampVector; +import org.apache.arrow.vector.VarBinaryVector; +import org.apache.arrow.vector.VarCharVector; +import org.apache.arrow.vector.complex.DenseUnionVector; +import org.apache.arrow.vector.complex.LargeListVector; +import org.apache.arrow.vector.complex.ListVector; +import org.apache.arrow.vector.complex.MapVector; +import org.apache.arrow.vector.complex.UnionVector; +import org.apache.arrow.vector.ipc.message.ArrowFieldNode; +import org.apache.arrow.vector.types.pojo.ArrowType; +import org.apache.arrow.vector.util.DataSizeRoundingUtil; + +import static org.apache.arrow.c.NativeUtil.NULL; +import static org.apache.arrow.util.Preconditions.checkState; + +/** + * Import buffers from a C Data Interface struct. We copy it from Arrow `BufferImportTypeVisitor` + * and fix the issue: https://github.com/apache/arrow/issues/42156. + */ +class CometBufferImportTypeVisitor + implements ArrowType.ArrowTypeVisitor>, AutoCloseable { + private final BufferAllocator allocator; + private final ReferenceCountedArrowArray underlyingAllocation; + private final ArrowFieldNode fieldNode; + private final ArrowArray.Snapshot snapshot; + private final long[] buffers; + private final List imported; + + CometBufferImportTypeVisitor( + BufferAllocator allocator, + ReferenceCountedArrowArray underlyingAllocation, + ArrowFieldNode fieldNode, + ArrowArray.Snapshot snapshot, + long[] buffers) { + this.allocator = allocator; + this.underlyingAllocation = underlyingAllocation; + this.fieldNode = fieldNode; + this.snapshot = snapshot; + this.buffers = buffers; + this.imported = new ArrayList<>(); + } + + @Override + public void close() throws Exception { + AutoCloseables.close(imported); + } + + @VisibleForTesting + ArrowBuf importBuffer(ArrowType type, int index, long capacity) { + return importBuffer(type, index, 0, capacity); + } + + @VisibleForTesting + ArrowBuf importBuffer(ArrowType type, int index, long offset, long capacity) { + checkState( + buffers.length > index, + "Expected at least %s buffers for type %s, but found %s", + index + 1, + type, + buffers.length); + long bufferPtr = buffers[index] + offset; + + if (bufferPtr == NULL) { + // C array may be NULL but only accept that if expected capacity is zero too + if (capacity != 0) { + throw new IllegalStateException( + String.format("Buffer %s for type %s cannot be null", index, type)); + } else { + // no data in the C array, return an empty buffer + return allocator.getEmpty(); + } + } + + ArrowBuf buf = underlyingAllocation.unsafeAssociateAllocation(allocator, capacity, bufferPtr); + imported.add(buf); + return buf; + } + + private ArrowBuf importFixedBits(ArrowType type, int index, long bitsPerSlot) { + final long capacity = DataSizeRoundingUtil.divideBy8Ceil(bitsPerSlot * fieldNode.getLength()); + return importBuffer(type, index, capacity); + } + + private ArrowBuf importFixedBytes(ArrowType type, int index, long bytesPerSlot) { + final long capacity = bytesPerSlot * fieldNode.getLength(); + return importBuffer(type, index, capacity); + } + + private ArrowBuf importOffsets(ArrowType type, long bytesPerSlot) { + final long capacity = bytesPerSlot * (fieldNode.getLength() + 1); + final long offset = snapshot.offset * bytesPerSlot; + return importBuffer(type, 1, offset, capacity); + } + + private ArrowBuf importData(ArrowType type, long capacity) { + return importBuffer(type, 2, capacity); + } + + private ArrowBuf maybeImportBitmap(ArrowType type) { + checkState( + buffers.length > 0, + "Expected at least %s buffers for type %s, but found %s", + 1, + type, + buffers.length); + if (buffers[0] == NULL) { + return null; + } + return importFixedBits(type, 0, /* bitsPerSlot= */ 1); + } + + @Override + public List visit(ArrowType.Null type) { + checkState( + buffers.length == 0, + "Expected %s buffers for type %s, but found %s", + 0, + type, + buffers.length); + return Collections.emptyList(); + } + + @Override + public List visit(ArrowType.Struct type) { + return Collections.singletonList(maybeImportBitmap(type)); + } + + @Override + public List visit(ArrowType.List type) { + return Arrays.asList(maybeImportBitmap(type), importOffsets(type, ListVector.OFFSET_WIDTH)); + } + + @Override + public List visit(ArrowType.LargeList type) { + return Arrays.asList( + maybeImportBitmap(type), importOffsets(type, LargeListVector.OFFSET_WIDTH)); + } + + @Override + public List visit(ArrowType.FixedSizeList type) { + return Collections.singletonList(maybeImportBitmap(type)); + } + + @Override + public List visit(ArrowType.Union type) { + switch (type.getMode()) { + case Sparse: + return Collections.singletonList(importFixedBytes(type, 0, UnionVector.TYPE_WIDTH)); + case Dense: + return Arrays.asList( + importFixedBytes(type, 0, DenseUnionVector.TYPE_WIDTH), + importFixedBytes(type, 1, DenseUnionVector.OFFSET_WIDTH)); + default: + throw new UnsupportedOperationException("Importing buffers for union type: " + type); + } + } + + @Override + public List visit(ArrowType.Map type) { + return Arrays.asList(maybeImportBitmap(type), importOffsets(type, MapVector.OFFSET_WIDTH)); + } + + @Override + public List visit(ArrowType.Int type) { + return Arrays.asList(maybeImportBitmap(type), importFixedBits(type, 1, type.getBitWidth())); + } + + @Override + public List visit(ArrowType.FloatingPoint type) { + switch (type.getPrecision()) { + case HALF: + return Arrays.asList( + maybeImportBitmap(type), importFixedBytes(type, 1, /* bytesPerSlot= */ 2)); + case SINGLE: + return Arrays.asList( + maybeImportBitmap(type), importFixedBytes(type, 1, Float4Vector.TYPE_WIDTH)); + case DOUBLE: + return Arrays.asList( + maybeImportBitmap(type), importFixedBytes(type, 1, Float8Vector.TYPE_WIDTH)); + default: + throw new UnsupportedOperationException("Importing buffers for type: " + type); + } + } + + @Override + public List visit(ArrowType.Utf8 type) { + try (ArrowBuf offsets = importOffsets(type, VarCharVector.OFFSET_WIDTH)) { + final int start = offsets.getInt(0); + final int end = offsets.getInt(fieldNode.getLength() * (long) VarCharVector.OFFSET_WIDTH); + checkState( + end >= start, + "Offset buffer for type %s is malformed: start: %s, end: %s", + type, + start, + end); + // HACK: For the issue https://github.com/apache/datafusion-comet/issues/540 + // As Arrow Java doesn't support `offset` in C Data interface, we cannot correctly import + // a slice of string from arrow-rs to Java Arrow and then export it to arrow-rs again. + // So we add this hack to always take full length of data buffer by assuming the first offset + // is always 0 which is true for Arrow Java and arrow-rs. + final int len = end; + offsets.getReferenceManager().retain(); + return Arrays.asList(maybeImportBitmap(type), offsets, importData(type, len)); + } + } + + @Override + public List visit(ArrowType.LargeUtf8 type) { + try (ArrowBuf offsets = importOffsets(type, LargeVarCharVector.OFFSET_WIDTH)) { + final long start = offsets.getLong(0); + final long end = + offsets.getLong(fieldNode.getLength() * (long) LargeVarCharVector.OFFSET_WIDTH); + checkState( + end >= start, + "Offset buffer for type %s is malformed: start: %s, end: %s", + type, + start, + end); + // HACK: For the issue https://github.com/apache/datafusion-comet/issues/540 + // As Arrow Java doesn't support `offset` in C Data interface, we cannot correctly import + // a slice of string from arrow-rs to Java Arrow and then export it to arrow-rs again. + // So we add this hack to always take full length of data buffer by assuming the first offset + // is always 0 which is true for Arrow Java and arrow-rs. + final long len = end; + offsets.getReferenceManager().retain(); + return Arrays.asList(maybeImportBitmap(type), offsets, importData(type, len)); + } + } + + @Override + public List visit(ArrowType.Binary type) { + try (ArrowBuf offsets = importOffsets(type, VarBinaryVector.OFFSET_WIDTH)) { + final int start = offsets.getInt(0); + final int end = offsets.getInt(fieldNode.getLength() * (long) VarBinaryVector.OFFSET_WIDTH); + checkState( + end >= start, + "Offset buffer for type %s is malformed: start: %s, end: %s", + type, + start, + end); + // HACK: For the issue https://github.com/apache/datafusion-comet/issues/540 + // As Arrow Java doesn't support `offset` in C Data interface, we cannot correctly import + // a slice of string from arrow-rs to Java Arrow and then export it to arrow-rs again. + // So we add this hack to always take full length of data buffer by assuming the first offset + // is always 0 which is true for Arrow Java and arrow-rs. + final int len = end; + offsets.getReferenceManager().retain(); + return Arrays.asList(maybeImportBitmap(type), offsets, importData(type, len)); + } + } + + @Override + public List visit(ArrowType.LargeBinary type) { + try (ArrowBuf offsets = importOffsets(type, LargeVarBinaryVector.OFFSET_WIDTH)) { + final long start = offsets.getLong(0); + // TODO: need better tests to cover the failure when I forget to multiply by offset width + final long end = + offsets.getLong(fieldNode.getLength() * (long) LargeVarBinaryVector.OFFSET_WIDTH); + checkState( + end >= start, + "Offset buffer for type %s is malformed: start: %s, end: %s", + type, + start, + end); + // HACK: For the issue https://github.com/apache/datafusion-comet/issues/540 + // As Arrow Java doesn't support `offset` in C Data interface, we cannot correctly import + // a slice of string from arrow-rs to Java Arrow and then export it to arrow-rs again. + // So we add this hack to always take full length of data buffer by assuming the first offset + // is always 0 which is true for Arrow Java and arrow-rs. + final long len = end; + offsets.getReferenceManager().retain(); + return Arrays.asList(maybeImportBitmap(type), offsets, importData(type, len)); + } + } + + @Override + public List visit(ArrowType.FixedSizeBinary type) { + return Arrays.asList(maybeImportBitmap(type), importFixedBytes(type, 1, type.getByteWidth())); + } + + @Override + public List visit(ArrowType.Bool type) { + return Arrays.asList(maybeImportBitmap(type), importFixedBits(type, 1, /* bitsPerSlot= */ 1)); + } + + @Override + public List visit(ArrowType.Decimal type) { + return Arrays.asList(maybeImportBitmap(type), importFixedBits(type, 1, type.getBitWidth())); + } + + @Override + public List visit(ArrowType.Date type) { + switch (type.getUnit()) { + case DAY: + return Arrays.asList( + maybeImportBitmap(type), importFixedBytes(type, 1, DateDayVector.TYPE_WIDTH)); + case MILLISECOND: + return Arrays.asList( + maybeImportBitmap(type), importFixedBytes(type, 1, DateMilliVector.TYPE_WIDTH)); + default: + throw new UnsupportedOperationException("Importing buffers for type: " + type); + } + } + + @Override + public List visit(ArrowType.Time type) { + switch (type.getUnit()) { + case SECOND: + return Arrays.asList( + maybeImportBitmap(type), importFixedBytes(type, 1, TimeSecVector.TYPE_WIDTH)); + case MILLISECOND: + return Arrays.asList( + maybeImportBitmap(type), importFixedBytes(type, 1, TimeMilliVector.TYPE_WIDTH)); + case MICROSECOND: + return Arrays.asList( + maybeImportBitmap(type), importFixedBytes(type, 1, TimeMicroVector.TYPE_WIDTH)); + case NANOSECOND: + return Arrays.asList( + maybeImportBitmap(type), importFixedBytes(type, 1, TimeNanoVector.TYPE_WIDTH)); + default: + throw new UnsupportedOperationException("Importing buffers for type: " + type); + } + } + + @Override + public List visit(ArrowType.Timestamp type) { + return Arrays.asList( + maybeImportBitmap(type), importFixedBytes(type, 1, TimeStampVector.TYPE_WIDTH)); + } + + @Override + public List visit(ArrowType.Interval type) { + switch (type.getUnit()) { + case YEAR_MONTH: + return Arrays.asList( + maybeImportBitmap(type), importFixedBytes(type, 1, IntervalYearVector.TYPE_WIDTH)); + case DAY_TIME: + return Arrays.asList( + maybeImportBitmap(type), importFixedBytes(type, 1, IntervalDayVector.TYPE_WIDTH)); + case MONTH_DAY_NANO: + return Arrays.asList( + maybeImportBitmap(type), + importFixedBytes(type, 1, IntervalMonthDayNanoVector.TYPE_WIDTH)); + default: + throw new UnsupportedOperationException("Importing buffers for type: " + type); + } + } + + @Override + public List visit(ArrowType.Duration type) { + return Arrays.asList( + maybeImportBitmap(type), importFixedBytes(type, 1, DurationVector.TYPE_WIDTH)); + } +} diff --git a/core/Cargo.lock b/core/Cargo.lock index eca8b97d4..71fe6eb2e 100644 --- a/core/Cargo.lock +++ b/core/Cargo.lock @@ -115,8 +115,7 @@ checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" [[package]] name = "arrow" version = "52.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ae9728f104939be6d8d9b368a354b4929b0569160ea1641f0721b55a861ce38" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" dependencies = [ "arrow-arith", "arrow-array", @@ -136,8 +135,7 @@ dependencies = [ [[package]] name = "arrow-arith" version = "52.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a7029a5b3efbeafbf4a12d12dc16b8f9e9bff20a410b8c25c5d28acc089e1043" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" dependencies = [ "arrow-array", "arrow-buffer", @@ -151,8 +149,7 @@ dependencies = [ [[package]] name = "arrow-array" version = "52.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d33238427c60271710695f17742f45b1a5dc5bcfc5c15331c25ddfe7abf70d97" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" dependencies = [ "ahash", "arrow-buffer", @@ -168,8 +165,7 @@ dependencies = [ [[package]] name = "arrow-buffer" version = "52.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fe9b95e825ae838efaf77e366c00d3fc8cca78134c9db497d6bda425f2e7b7c1" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" dependencies = [ "bytes", "half", @@ -179,8 +175,7 @@ dependencies = [ [[package]] name = "arrow-cast" version = "52.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87cf8385a9d5b5fcde771661dd07652b79b9139fea66193eda6a88664400ccab" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" dependencies = [ "arrow-array", "arrow-buffer", @@ -200,8 +195,7 @@ dependencies = [ [[package]] name = "arrow-csv" version = "52.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cea5068bef430a86690059665e40034625ec323ffa4dd21972048eebb0127adc" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" dependencies = [ "arrow-array", "arrow-buffer", @@ -219,8 +213,7 @@ dependencies = [ [[package]] name = "arrow-data" version = "52.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cb29be98f987bcf217b070512bb7afba2f65180858bca462edf4a39d84a23e10" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" dependencies = [ "arrow-buffer", "arrow-schema", @@ -231,8 +224,7 @@ dependencies = [ [[package]] name = "arrow-ipc" version = "52.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ffc68f6523970aa6f7ce1dc9a33a7d9284cfb9af77d4ad3e617dbe5d79cc6ec8" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" dependencies = [ "arrow-array", "arrow-buffer", @@ -246,8 +238,7 @@ dependencies = [ [[package]] name = "arrow-json" version = "52.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2041380f94bd6437ab648e6c2085a045e45a0c44f91a1b9a4fe3fed3d379bfb1" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" dependencies = [ "arrow-array", "arrow-buffer", @@ -266,8 +257,7 @@ dependencies = [ [[package]] name = "arrow-ord" version = "52.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fcb56ed1547004e12203652f12fe12e824161ff9d1e5cf2a7dc4ff02ba94f413" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" dependencies = [ "arrow-array", "arrow-buffer", @@ -281,8 +271,7 @@ dependencies = [ [[package]] name = "arrow-row" version = "52.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "575b42f1fc588f2da6977b94a5ca565459f5ab07b60545e17243fb9a7ed6d43e" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" dependencies = [ "ahash", "arrow-array", @@ -296,8 +285,7 @@ dependencies = [ [[package]] name = "arrow-schema" version = "52.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32aae6a60458a2389c0da89c9de0b7932427776127da1a738e2efc21d32f3393" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" dependencies = [ "bitflags 2.5.0", ] @@ -305,8 +293,7 @@ dependencies = [ [[package]] name = "arrow-select" version = "52.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "de36abaef8767b4220d7b4a8c2fe5ffc78b47db81b03d77e2136091c3ba39102" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" dependencies = [ "ahash", "arrow-array", @@ -319,8 +306,7 @@ dependencies = [ [[package]] name = "arrow-string" version = "52.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e435ada8409bcafc910bc3e0077f532a4daa20e99060a496685c0e3e53cc2597" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" dependencies = [ "arrow-array", "arrow-buffer", @@ -805,8 +791,7 @@ dependencies = [ [[package]] name = "datafusion" version = "39.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2f92d2d7a9cba4580900b32b009848d9eb35f1028ac84cdd6ddcf97612cd0068" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" dependencies = [ "ahash", "arrow", @@ -913,8 +898,7 @@ dependencies = [ [[package]] name = "datafusion-common" version = "39.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "effed030d2c1667eb1e11df5372d4981eaf5d11a521be32220b3985ae5ba6971" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" dependencies = [ "ahash", "arrow", @@ -934,8 +918,7 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" version = "39.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d0091318129dad1359f08e4c6c71f855163c35bba05d1dbf983196f727857894" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" dependencies = [ "tokio", ] @@ -943,8 +926,7 @@ dependencies = [ [[package]] name = "datafusion-execution" version = "39.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8385aba84fc4a06d3ebccfbcbf9b4f985e80c762fac634b49079f7cc14933fb1" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" dependencies = [ "arrow", "chrono", @@ -964,8 +946,7 @@ dependencies = [ [[package]] name = "datafusion-expr" version = "39.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ebb192f0055d2ce64e38ac100abc18e4e6ae9734d3c28eee522bbbd6a32108a3" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" dependencies = [ "ahash", "arrow", @@ -983,8 +964,7 @@ dependencies = [ [[package]] name = "datafusion-functions" version = "39.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "27c081ae5b7edd712b92767fb8ed5c0e32755682f8075707666cd70835807c0b" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" dependencies = [ "arrow", "base64", @@ -1010,8 +990,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" version = "39.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "feb28a4ea52c28a26990646986a27c4052829a2a2572386258679e19263f8b78" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" dependencies = [ "ahash", "arrow", @@ -1028,8 +1007,7 @@ dependencies = [ [[package]] name = "datafusion-optimizer" version = "39.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "12172f2a6c9eb4992a51e62d709eeba5dedaa3b5369cce37ff6c2260e100ba76" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" dependencies = [ "arrow", "async-trait", @@ -1047,8 +1025,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" version = "39.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a3fce531b623e94180f6cd33d620ef01530405751b6ddd2fd96250cdbd78e2e" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" dependencies = [ "ahash", "arrow", @@ -1078,8 +1055,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" version = "39.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "046400b6a2cc3ed57a7c576f5ae6aecc77804ac8e0186926b278b189305b2a77" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" dependencies = [ "arrow", "datafusion-common", @@ -1090,8 +1066,7 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" version = "39.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4aed47f5a2ad8766260befb375b201592e86a08b260256e168ae4311426a2bff" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" dependencies = [ "ahash", "arrow", @@ -1124,8 +1099,7 @@ dependencies = [ [[package]] name = "datafusion-sql" version = "39.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7fa92bb1fd15e46ce5fb6f1c85f3ac054592560f294429a28e392b5f9cd4255e" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" dependencies = [ "arrow", "arrow-array", @@ -2064,8 +2038,7 @@ dependencies = [ [[package]] name = "parquet" version = "52.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "29c3b5322cc1bbf67f11c079c42be41a55949099b78732f7dba9e15edde40eab" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" dependencies = [ "ahash", "bytes", diff --git a/core/Cargo.toml b/core/Cargo.toml index 7c22876dc..c3e924a44 100644 --- a/core/Cargo.toml +++ b/core/Cargo.toml @@ -33,13 +33,13 @@ include = [ [dependencies] parquet-format = "4.0.0" # This must be kept in sync with that from parquet crate -arrow = { version = "52.0.0", features = ["prettyprint", "ffi", "chrono-tz"] } -arrow-array = { version = "52.0.0" } -arrow-buffer = { version = "52.0.0" } -arrow-data = { version = "52.0.0" } -arrow-schema = { version = "52.0.0" } -arrow-string = { version = "52.0.0" } -parquet = { version = "52.0.0", default-features = false, features = ["experimental"] } +arrow = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1", features = ["prettyprint", "ffi", "chrono-tz"] } +arrow-array = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } +arrow-buffer = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } +arrow-data = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } +arrow-schema = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } +arrow-string = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } +parquet = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1", default-features = false, features = ["experimental"] } half = { version = "2.4.1", default-features = false } futures = "0.3.28" mimalloc = { version = "*", default-features = false, optional = true } @@ -71,12 +71,12 @@ 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 = "39.0.0" } -datafusion = { default-features = false, version = "39.0.0", features = ["unicode_expressions", "crypto_expressions"] } -datafusion-functions = { version = "39.0.0", features = ["crypto_expressions"] } -datafusion-expr = { version = "39.0.0", default-features = false } -datafusion-physical-expr-common = { version = "39.0.0", default-features = false } -datafusion-physical-expr = { version = "39.0.0", default-features = false } +datafusion-common = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1" } +datafusion = { default-features = false, git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1", features = ["unicode_expressions", "crypto_expressions"] } +datafusion-functions = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1", features = ["crypto_expressions"] } +datafusion-expr = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1", default-features = false } +datafusion-physical-expr-common = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1", default-features = false } +datafusion-physical-expr = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1", default-features = false } unicode-segmentation = "^1.10.1" once_cell = "1.18.0" regex = "1.9.6" From 42ed636bb189089809037c76e617e854923283c0 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 1 Jul 2024 11:52:53 -0700 Subject: [PATCH 02/16] chore: Use proper constructor of IndexShuffleBlockResolver (#610) --- .../shuffle/CometShuffleManager.scala | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleManager.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleManager.scala index 335fb065f..ef67167c4 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleManager.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleManager.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.comet.execution.shuffle +import java.util.Collections import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ @@ -61,7 +62,23 @@ class CometShuffleManager(conf: SparkConf) extends ShuffleManager with Logging { private lazy val shuffleExecutorComponents = loadShuffleExecutorComponents(conf) - override val shuffleBlockResolver = new IndexShuffleBlockResolver(conf) + override val shuffleBlockResolver: IndexShuffleBlockResolver = { + // The patch versions of Spark 3.4 have different constructor signatures: + // See https://github.com/apache/spark/commit/5180694705be3508bd21dd9b863a59b8cb8ba193 + // We look for proper constructor by reflection. + classOf[IndexShuffleBlockResolver].getDeclaredConstructors + .filter(c => List(2, 3).contains(c.getParameterCount())) + .map { c => + c.getParameterCount match { + case 2 => + c.newInstance(conf, null).asInstanceOf[IndexShuffleBlockResolver] + case 3 => + c.newInstance(conf, null, Collections.emptyMap()) + .asInstanceOf[IndexShuffleBlockResolver] + } + } + .head + } /** * (override) Obtains a [[ShuffleHandle]] to pass to tasks. From 106b2120a29461c616186c5152098f9e144daf7b Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 1 Jul 2024 17:18:48 -0600 Subject: [PATCH 03/16] chore: Update benchmark results (#614) * Update benchmark script and results * update diagrams * update docs * revert cbo change --- README.md | 5 +- .../source/_static/images/tpch_allqueries.png | Bin 23677 -> 24465 bytes .../_static/images/tpch_queries_compare.png | Bin 27388 -> 28097 bytes .../_static/images/tpch_queries_speedup.png | Bin 38821 -> 37231 bytes .../comet-8-exec-5-runs.json | 247 +++++++++--------- .../datafusion-python-8-cores.json | 0 .../spark-8-exec-5-runs.json | 0 docs/source/contributor-guide/benchmarking.md | 11 +- .../apache/comet/serde/QueryPlanSerde.scala | 2 +- 9 files changed, 134 insertions(+), 131 deletions(-) rename docs/source/contributor-guide/benchmark-results/{2024-05-30 => 2024-06-29}/comet-8-exec-5-runs.json (52%) rename docs/source/contributor-guide/benchmark-results/{2024-05-30 => 2024-06-29}/datafusion-python-8-cores.json (100%) rename docs/source/contributor-guide/benchmark-results/{2024-05-30 => 2024-06-29}/spark-8-exec-5-runs.json (100%) diff --git a/README.md b/README.md index b0b72fdb6..f128d17d3 100644 --- a/README.md +++ b/README.md @@ -34,13 +34,14 @@ The following chart shows the time it takes to run the 22 TPC-H queries against using a single executor with 8 cores. See the [Comet Benchmarking Guide](https://datafusion.apache.org/comet/contributor-guide/benchmarking.html) for details of the environment used for these benchmarks. -When using Comet, the overall run time is reduced from 649 seconds to 440 seconds, a 1.5x speedup. +When using Comet, the overall run time is reduced from 649 seconds to 433 seconds, a 1.5x speedup, with some queries +showing a 2x-3x speedup. Running the same queries with DataFusion standalone (without Spark) using the same number of cores results in a 3.9x speedup compared to Spark. Comet is not yet achieving full DataFusion speeds in all cases, but with future work we aim to provide a 2x-4x speedup -for many use cases. +for a broader set of queries. ![](docs/source/_static/images/tpch_allqueries.png) diff --git a/docs/source/_static/images/tpch_allqueries.png b/docs/source/_static/images/tpch_allqueries.png index a6788d5a402647856d59eab220c68b80b1a84e81..c6fa4e06548b2f657855470608e626fbba14a21b 100644 GIT binary patch literal 24465 zcmdsfc|4X|+xBJ37$F%NM5YjuN-`yL6QU%Aq>zxvJVr={l#~h?G8Ac0M8<|)8W0gO zE6JQPB)(&5*WT~@KJWMa|M@-7-n($$_jO%so#!!~E8Jka&U_XD7K)*;D5Q50)ywUFgxq+IZja=R^Kwt^W>Qum>W3L7zaxw-~2Qldd#ntI^MzF?pvDpN!9b?%d3-1Hgfn}Ieti6NH3ON z7+>7C->~$=6QZSBqA=pLRonUGc&X4orB?Gl|m`v$p#XM zy?6a*{mctpv~srY-p#vx`}W?pQnl@g%6qO4Fmf!je1FHrwjp=vd416{XUGx^IM#on`c`DJdy0a=gcS4PE={n8n1z^6uWPd>qbw_T0IhjvuT zzj$46csTRcty|kWIs(he)Te)rx7oXWdU+}J`|J)K9men9zk75=t{?uC+aZ+`mxnR&;K9SavNvX@^X>_yw=_amoi^&sP^vz&Am0LjGdjG!%r7GwdXu& z4Z3uR|8nwvHS#vJjfoBaCew!?e*>F>=>1t$gssIsR|=RNZJRxvTs zRg-OUL)`JmKvRBLM8xLi`$vy`dVTfTyQLdkMD9DaMd_UrE3dETxxDUZ(5+jtc*D@` zSCyWB!y{mpm6a_hDzg4un@(O+^+*32^8#lY>@EAFr$R%mhuh1m-@dKPv#P8v_B`I( z+uPn!c)YRjxH!2C+{L;ZyKil6Idtfd#LAVM?CmAr-Li;&eRX43<8ABFSv6s=a$VYm z3m59{D>@Mu8ylFh?kpU?6~l$Mqrw6{0w@T80^E!+L?@3fzpo{UXQOsq@MVztA6*S=Ak zmHs(eC0kpYz9k*kr}yyT!`e_mzlp~K>1k=YPo6x9711;5dK#H7v+v>gy|U}p-GABE z{PV|n`n>r9x_BjfH}CKLE8jOaZ+ZXzz03QlZ(l^I+QAP+J$E8_W%aLLzaCpuROE^m zkoR4*Xpz3uuA4DNMn>Its;do{16;qnTb2JXV@J{jO1VCt+whg}L9Vcp62D729GYXD zeNFjN0in?=XqK;Ax2W{z_r*`o@YAWOmCApcKY#wn`@>1hk;gv1yo6oDTrx9tIsb>$}+Z##Er1c!#+5cS`%VMEN!C0_UO!F{+T^J{8qOm831N-yD# zkBJF5HGgros2er>Lh2S3Fg??qwKK!O*f@CI(WcD$HF9$55sOyE#KbI*4+#yeeDZ_= z%h|inLw?PgV?VwhJ$Ufoav>oa>!)V|&z|LM_4)cL!!aVvexB8Av6Q4_1ztMOeQ5tZ z#}5K{Qud$XW%fx*E)>o_=;`TM;QM3U>4=D9gDo5A85n3NJa78;4kH7q8g=W_pw#;H zyzkz zQMdzY?$ea>*N?AUbUy1>IkprCHF)2fZgb;UszgtriWVH}nunFN`}+P}yB3{3eY)Fv z=gyscix=Ohx2wxM+jKiUUDR)8^00zq%f!mu(_!{pR?O`vVmg$|~s@sF$x^B^Al8Sg~Sy>dUR3#4qo1n*IG|-{tI8 z^cYT_))dB-`T6-dA3Ihqp5zmB^QQF6gte@8c6Oug_)MW_8LIC6>4R;h7K5?7wY9ZJ zCv|4`8h)t#n(klP^D4#u&%@QQ9jDf?dk;&A0DU(W=6&IPS~;{ML=U>aS5;N{>NDC z)rH3o;~{O16^Wv_>Yeev(c50Mw6svlJ{be`yDfYj_M*sEW*Vl}y-3qHzcquJ=j`kp z6Mf6PFi1O6AnQ@|?kSo5S=f-1zE59RIypsuX}A@?Qr5S7uYkqp)VA7D|AtjL8OD-U z_!fF@eAe!%?d@?=yAu^v*Q{A{???lC>GUY8%kkq?#dkJqu1(6*TkbyC5*Q!PE3nRi z$-=^7lCcSGW3}h#K|`wrhCIC=iq=!@MMEV&-aJ%QRSi3R+USR|7%MAlKBxA^jT_U_ z(lQ^3yHAeVTNHFZDNgW%d_`SwD+1(La;!FUYe>ahvAa%X##25)w>PQ&YNndV%5L;pVCy z!JoWQG&D2?&K;Ny?(Xit&N)z)P512NRo*QOi_VaBmtL8xeV8e_j3*>s(9ylk2}`o?A@Kyf}Xki)_xwId7LI^ zUtM0GiWlaSv!uMn2j`pg_4~MQj z@~#~hP>ChO5TX&da_9AS-~*4LN8_XjwwDFc**4~N_}}*(Zr`*-*<(Rj<#VyZ!orJc z{;C0?L+{-NTlmW=pWQh*Z@z}EE|a{xe0yhSkWT}`Ip3QzF`da*HAuABipRM*kTc=^V%z2jjp z+BY`zddHSBZ1%_^_n`wH9=iExh4yNR+CQVc9pTG!P&_ZiRx}J$jNl9VZ)$H`T za8+G0Ha=e5_xt<9M%WXg_`&BuQZ8<8aj2!s_Z54LsFhmm*%*jNWq9Y#omd%*66yF4 zCBA!=P@{@>KE=zW*VWaD7i7=wet1{gGYS1P{~?f5{PnGugwcxiPMtbMUON4+zy5M* zR&ni(sk^qB@%dkW>E5}2{{k%?ooJlCiQ@;-q_KcvBO)TuD@0Hiwm3RE)}jEXofF?N zbGVpv@QjR%u}TiLUoAQ%CQaX`Xlk{OuW!vmw0-?oLuIZ~tQ=`Nax1Q=n1$1Do0v3>vhS2aadB~V zwuuy8OR)EC2FL3w8&1A_xpm0~7y9`E3JW;radHNPo?Cp1jX&s-_ju=3_ts6bGe0}< zR69Elh=_>%#o9208tXeXmeuDR9wr%=N56o&bNB9T`+CV~^y9ytKi_m?SIz=HKGQ+T zaZSyHU4RByn_YL`s8(KDbGX|@MMY&|V#0E;rO+0Skw;rsGmXE3tL*&wg}??EerO!i z*JrtM<%%Y*$v78PX@#gL14^%LT_)#weH)WdP0h5`6?abI-jy0ls*wQip+o5VBRi_{-DoT1KI$EKrBqyigd?7t9fGe^tNqV0#ejp2++Ze+I4qTzw&2UggtDJjC7 zR>qQX`V8qt;r;y%VgdNrYw7*{S@~#;vy4HW{90#v&$QvX2rjdU)gY{@c1dLEq^+GD zGqwb-UBRJoLB4%`teRV2-CV)Jb#4pYw6^ivmxd7__tX;uO>*njX}`U`?O5mIaF;Jp zi`TQWgwR&S%kGwnj*ixO;P`vTvZJ9D3mROK}I;moEF5S8gZ6%K? zN_bb91*1cCUUn)8ynA;I3NnMBpx_l1?+ZtEb8#9*ULT>)Lmi?5zzY;NY&cf9&`UDO zZ))X|rAu!g>(VZK`Tl+3)<;-z*66y8kG$7olck@K0bxWBF@5Bful&IuE7tbiO@6c+ zCSKln`2oGy;GU{@YhdHv<_EHrC4g35e*P0+{WC(FPZ@u-;o{_E7Ihf8jMwTI?x-Yf zz6RC7ywqT<~_OS9<2zyfaWo&53InwrKo z!J+}=o`Y(GKXVESMx!TD>L*UDTO@6|T#_x@3f1lm7HS0wF$$};x;iDc{Yn@rxOVPW z+1J2=p8r5{A1bPMBcP{p)2ZK4@$eH6jJ<`6}pv+NmkudS`6sQdTt+xOrfqZ{zx z6ZB1K75!&?Zti`+jtdC|!8C8ZZLRb8*|P|illB*5W4Z3x*R!xLl2)&}u!=B+GyH4j z0U***kDolTdUBel@t)nzkXyV!o_uXSU*}(t-b2gH&E5F^efj(M{CFF)B3fl^?DbZq zKM5HH!iitGTQcsi7caPY=GDuQMm{`jAa&f`T?Z9~YDbSjfetu#jvGCezz9&o10Zji z7NsTAS0A)`5#U+i_frX#>EwpvAKL-oJSKbMs|FfgzhN@&ccm7 zapFXQ$H;+8qm`i?yC=3>%me_zB^R6RF*4%ntG~fW@u~VK&dmI9x}xl9G^41fXo;tY zh2?(fyTj3=5DrcqZN6U)euPye^}qu323IGfq+}+uGs`$EGBPA1h9FZKJWX|9A%_LON0b@=7QKs-+mPfvn`uWocdb?urE`CyP= zzz^&%-ap#hnUUm2AOgxMBQ8|hqGUS!zm8I++Clh zC%^vn`t^nAjBS@swX~R@S-Nr2o;`bd>$13oSF9+Ux15u+-lO6>t&9mY7c5F)adF0F z%a+NlS;PI_H@$Oz5UGtEsEtc#q-`uMspH3wqsTV&c1M$fe4T3@b6RF5)7?X_g0N?P zPJJ^h_4Ad8(|>to19wPBNJ_3=@%C-^1ZyYh7m2N92HP2 z#K>BDYr(#HAp@R-nmgM~Le(`jWhFq7F09&{=k{gSSpWN-GDlD_P-9lR_8QEUY|S7h zZ+Zt?+sylNPAhRQ`1tq;i-J!01>^)uQgpQ1jA<(!ABDX>@?1R0a(ZIq_a}nBqik75 z<#(Ywr#OCiXxm!6foi{1I-`MIiQcZjj}FessiKj}oNSb3yz%?juRAxMaCNOah-Qq6 zz#b_Uuz0;=-NCQqk7I1XkLYyCf7R3dFPfxD0(Hp7#>8gs*Uj`yX(vz?4gl_qj*vx$ zyU>*$zrVu5|EH)V8)-N>;?gwc4;-%+4_X)-FNDmsfRi&2u=M~I|DUVa$2e?=?dC$u zw`^I|`Zgc#4@K@ecnmeU`!)i&k&dlmwITqWN&~75wOn4F3k8m%^kNOn&82_8kpsPR zSpbd4mz)RfeSN1lx(_BFev?QA1P9X>6cqgHp2n}*dkTbd#oDzzpzuV#qUK=k?9Xwn zkiQ{=XaK7GMR^D~YkUakQ*%&a?(T8mX5vF{ zhT!Pn;814k6~xHMClEg=&Y`itRDDcL)%$7xGYURwQr9IeaUxA^JP zr?yS^R&KwdD3la0!hHAen^TV-sh~`E3=BlF^2%i96TQ=_@>F?61?`0k7x+}X6kJ?f zG%Z;0!Oj}2m8U889+y!%(a(e(jgQmb|Nbr!Xzc5=MH}|qv*p~CYaZ_Q<(&{D9YD+B zkr4ZT$VZQ&6ECYRa1o2eZgruzD}#cTo!OzYOIvweR@6l$Gp z(vI!hwav^Dikq1zY*4~HSFc`81z;zdnwhFF&2Z6-cmcA}J9-1#UJxtX#jEE438yaip)qMyYGZBDeECx^vtxJ^`=X6yFtx%cij-9H)u zjK+)`aB;oUGTfTmK9eqNi`Q+<0|bS>c%hN8{ptdKer9M*g|Elk{G&nJ zDyynkq16t5{aX1VPD*OkcCruqGPQH*O!ql2Rj(DvMeR%DH)k{$RtkVPzKU!n;Cb^$ zUOZGD-e(##Ubu3u71)EK04E76D)M20;|*imChy+6_ZUm-UnL~Te`W)bn4?vvn7~E-vzB_Z{|JZ>g?c04DvrB7)HzKK#jrHz?}NnPBL+nQ~Y)VfPaA zY%H<&IME_tEU3qdEZKO1ZT3go>;kwO&@S|i`N(Q8+kv_~STt18*vPl$@N3Qe``m3)HEQkaB1Sa7753DVge%GB5TpG@fe*V;pF8A*{SHv9XYUl*bbVvIaNE- z;DO`5;-BA}e_x)-9bRXAC`bXpvp@BaNobKUmrF76A|_rVW~b)uF8>Zfaf z1Q6bB_44BC-j>2OZH#2|aOs-T_I~_$YP{8(0}oE}yncG7BXI+JEwHPAU>4Q$kWZy@ z3Q`*7++IdAhF6r4k%2n8DM{7Wxwl&A>|9quXJl@Nj>HHqgj-H(8T4^s(m8s1LLi;| z>3q#BuVM0_ylWX4#F$M&AGz)K(@R85>@-uV(J?8-S`kME9qfP>mhJ6K?I1EW-~jSH z=y03he*9U2^wl*roTwZ&FUAKyu~UJ+kd^rUAY$Ac^7+7$>pwdK=dJWFnZIuf*`D|^ z*^xSj1cEfKd89nNbm>w8TL{@4GuLY9ohL_o5*!lA9}@$$xz+vs{kN@~WmY@y^ek~?(DevA)2BW49CK(go8 zkA#$=msuV>7>t|Ki6$Es9v+yOxJ*MsW4P6O$PU&TzJgA1mqA#ydKR@51-%>yE)0@$ z2TELl|EwyY&q+YS0{{!X@9&Dk8(GUw+BoOW&A%FM9e6#9emWQAB$y(ZW8ZnjT5`wM zWqrK(H>@1fs2v*`N(f-x#EoCGpjU`0!ARhrF33WO8iwqnKy7J-E- zEqH7HBY{Fp|Ie~Khyh%9EiO4RI|uhj{hC3hFQn)0uP{E!H?1qqh@VXD##%3>NtP7+60&(G_hV8ONvQr#z31FJOgE6h%S%skyOR7F~4ij(dI; z(>}vT16(9+n%C|ra}JL-vk9B@1iaSr@^Ydz5Z(d$MMJ^&L$&&l;Rvgg1Dww0#ofDi ziM3=6V%iI1jIxBCf+F+`)jb?)oAsMS6-Dpw(%A8=d-v|WXIn!n=s&&s`i`V1+>f%d zvi3=t3*;RIxW6-b)MuLzhZ4Vc`^11*@mTE^DggfKSZ_^NR&x>5WGt>>^cm~MJgYe< zKpHVxBH{D<)~{a=r9wzdY#s$tfAada%N6h5?aGA#K^LbVf=X}Pc-YTR6^o~dtl!M* zoa<uD7_V`HzFOk54f@lxg*ksk~?bs3D#((eb zU8yxv1!D!p<1AbFfmA9gG=77%U#t|C^a@Ey{iCs3Krc$+VSIaum zkZXm4TX6gn`^}u3nyj7T#IZ+9nm6oxAUrCH9np%Iq$Ca~U^n;N)t)KI&6R;3%(-9z zd3!*`incaEG%D{IR#9A8_Q!>g&V`i{j^zOq5m{_gT$DaJ!PQ#WP`%g)PntwuV^6=n!1LDP$)%2Q~v#z4ok}j ztV912g5|5_rbm9ZXTg28mrzlhM>@5DwvrJKki+NNh}$j;dlz3h(B1nRZY5>rl6UM~ zjb^|0*ewfuCx!PcU<%E0I7}A4-+0hZy4`88wU7Oray$Er1Fr=K_MiQ^9-8VkW65JZ zFL;Pbwrkfe5^;dJPeLt**9cX$BR`5|#)n9SjClb?aBxdQ120hH!rwUN0<}!dZwb7e z{g+1L_j@sefd76HK!%$)Z_YH%mhS;&CCCQ9RC9a}y5&L^2E=Sc;$ahc+g2yj0B2Yo zJsOGL-HEKl=p>=tI?>BuF9tyNAwq}W^n{DQn5rbS7?91ASoPAYR;~6LvjyO^?W+yb zjeuzMW9;)uv{zEkbgv2ALeDh`pX+CfIks6ukjF7Fqxo1uOD5N-R6k*-1TOUIz z26BApJ|r{bmzJ7(5})G$HaB2h1I8j|y({Os4FXECvEhy2eb$g*e#IX590 z0)HZ^^|pn9`|Pd-7br0i5fuL{W3unsY=lzmt3W!@A&jIJueDzYTqNf^<=WfVcVRM0 zL=P_JjtYrmaF1+g7?V0rG2Mg&y3fn2 z8+oxrpTaA+A-_;K#pTlF%c;<0?QJ0J5M?((!7d06jWuMs-quGO(GiXfG_933FIWJU zkJZIM5l2XO#||C*1QDJA9f%lBfKXqqQq`ny;dOX_M@y40?(`QbR_xeP1S(&(D+SOB(#Y<;zjeSAf)Fw@ z!{g&RfF1uZ8MzG`_89Lbm3xfOM2R^#EPH*)spJn60<<*|TTE#iOa1*N7M>Q*E;{~c z`+OZD{G>e!enq=WGA=AC>M4j$_nm?QMPi=Lh`iDX=g1}XEk`p}Ax9z8k&Rf*Ws zqZL`m0eC=Rw6d}a1bm5LA*4n32J>Q?mOLvu$m1#Z?MYV#y}xZ$Iqwgrn$xb}X6(K` zP3r4~ISvHwzKkYXU@Z%YQXv2`7g3_nsY$ZwjKJD|>i~#rl|$;;azVNr#w!g4C*-1% znhYXN-p6#AUW44K|5Y0nNTR>v z((^(*dp*3u!-Lka{g{}Uv9)~yKql#^Wy&4_$Y@x@*7@trn^3q=?{4m)1Js=KUb%85 z6BAPzs<-8#L-Rn}fr2O9HawqUeRkz0)#Id_(k2qLJ;PN z$8}S3srIyy;sK}~_Rqi)6`k9u{eF`@pBy&=g%F7pdhu%bSB1(OH!cPoU3=f*eGn8% zT56qR3lp*-a{)=z5E!2)q1_$;Pl5A?$jYi=tPbm`vCkLR9A*c;PTZ(Unmu%cf7($mw;3ZcvbMs_Pji}|O2+VUn&Tw0n9nWCKFUc#xfw?{6a*}84p z6Y_+HN@tfLQ&llFpP6~ni4*syVN&zFTaUYMXj3!6)LtrhPI4g&TK4&s{s{ves8TU! z&j#+wF?()tdPYXorx(_?woIEh zSBgi?@i3)Y4_e|!hiC{1Zix*BlFnW#wU7m6TkY6o?0>M)^Jec6U22C~LLdRe2T1eS zf2Wd?1gUmZ3=-%;eupAxJ;`U38EUDewKW}uR1v}0%F4=#lU{Qdyb;g%$M=C7W9weM zdR3FA$DJ>ad>iEVy)uH()3NugY-|X?-+EDw2JRl|3KWT_fI*d^+}^gW5e6@^0%&Do zW9yb6ve=_)g+%Ja=UT0|`_3Q1B855(NGO&BZoIv{Yxcpi{M*p)&+rNz(OU&S>GM&eSF2NPu4c0bjwfHZ z!9KqHte7nU)&`IysM?@Cm1>st;K{B{n+)~Mlj z;UdieMJm1`z*g6!#W(kGf}V1hkudJpH)^+rry^n}4VMQcW@YefJ*KR>ZuqXz~CtQ;JW zKxf^vzwm{$OSUda*7wh$^0vkvu}bz(bMyB6-PdMxo22UY?#mKh2b)I07HX$VGBO@tU#+*Yw`V~MdYP!Q z7=a2RlRf zH-UX_!7aNsLSOy-IWuyK_BLopjdu@oAov;$$@z%V+*pZMFm?pk4bSfYj zDRP8^C(tyHp|&97LWdcSm{JRK^D}6b*U(%5MW4VUK8emqaz=05uaVWgHbA3xSl9=2 zl|WvQVX(+*=pYX0jetB<0K7C%^2}B2Kc#QpyqSkOM(7?;AqoZSJoY6_l%o6K5{yA` zLRy24gR7u*Dju0fVn$dXR2eoTLGLXO-L~vc8ZHG+f}8`)TUk-jS=cH2rXsjGu2rml z$r`hFS$zvUIP44ioH@Acg0B1fH_+pDUs3f{BFG$jmc;V#RDrjx(~r*X!nI*Nkll`+ zLq;O3tc1YvdzAhs7Kl5iM!~y`75Uk41qDXDcBP7L8uJwf>2}UMG2}!oABS-g-4@Ll+f+>4pUT6G4F+AxFzf^}No;+5 zqkHH}U!O3dYDkz4kBtRqWv#??z-p(~;_QdSeG;~U!*1zT&Z%CT)+aYr;cZZ466zhJ z%dJQ&VJp6X{6x0uZ}Z@9Z$UftoH~g-Ja$WRQvtt1n^1y?_T$@^am{Z~g&>aoaWUqi z4&h>##qm=qhs)f5&AFY?)}asto_y>EktMTSmz43$^N{f1n5!77glLri8h~2>Zw5&G zw~<5B_V>7)z*3s&2Sn*40*{^^rbGgp3c-F(&ly0vM-Z1uO--!~qzNqXej}9M# z2vG|9?LT^38LoqbmMA&*L35yNOE7aZ`W`k*a;F2)!XU5QnLZ4-3JSj6cn}5Kfm?k2etX8KxmQ4pL^JCSXu4 zWezu{=s=%?C32d0b-0Fbn?pYvkaSQ5 zwn0UOC&5JWdw}oupOBea`Hv)e@7_I42(emjbCN3$loXU*!{5~0oGAfIMYoD+F^%hq z6Mp?o{y?dVc;=EX#BAN3pzurkf!Z5(=FDTbN`GAVL3*Ra`Ww5uFiiLRZkx=oTf(%t zjgpbgKIb0r3Gm&wdZbMs# zzCsM%@>j36{a#|aw5G;dLh``%u zJrM>W!$jE|$@9(*$fGA3$TRq;M6H4c-;PSM>xXhAjP-j*n-`&b05NSs@Xi?}D_`k9 zyKUS5V}~9*GN#anpB-*Uq5amCLW%-X`baK(fu2+ zw+z3fAfC)sQ-cR1=v{&Z_on4LIv`lvNjgUMWKdM(TTbQY&(Z8aI!0Ow#DEbr8#f{u z`v!NdY_U^uAT-XuxObvGgo4&|;oo4s6L0)XBzaIEK@iXiM_!KXKKNgy+G56hbGHJ< zaD`28OG}T?LMeZ9uKi~`6f7!+_k7g<=)bYtWspo2jy4$|-E{}51uZSD^T(I`2;~}C z|7tO!R?NxQv8OC+)AWdlhPp#b>2BF_GBA(^Vt0LCQ&SV-vfFaM4vE_7M{oHuy%!`N zy{E?-!vK5Y37-<#8t23?UW@^OV^d>xfGs9nz7Pc9uDlQ=7TGv)7>wWByV4S#55>&7 zt;9ET?}J=($A4L2aC1VhUcGAmsqJpimbo`Vi^84g*s|scd`*i-FCMl+6yc zpTQ*IgBX|nC$y~jBs`|0Q)Tn%O%ln;Dyyi_Pyi*w69%pBRx&g+gwz!f8mjXv4fHRM z&7V;Y8U_$6#;rLxP;mRQn!%DEV;2o0cv^-YOmo9vF)$lz1} z?nGf3RWvnCg*Tf4%2Mt54f}UWwG|v&p&hk^nKHfaz>o2+c1(dG`Ap^xUL~w`!R)|3 z+|Lz=KjBhg*o2gpszF*DH*c7`=O388>6NW2=t+ww9s*`@$dn(R5-?!?lWR%AB6>Z3 z`aouK^70jk`jVs`t`>^>-Iqna2wYVX7=VZkup?`d?|1&OG5hJqw&^Z966#bXG*^rk zRidg%eyzJYX;U`)6xM)kLVDm}d zfsB-Z&LWGfn@ipnsFH{vgmuFOh=Q`VStp~@nDi(nc6Ks{h#9CFOx)s*WfArni$;RUbrwAlIj5j z=Un!{Tv*mT1YenqRzcBZDXUz1m-zk2J8#Nan(Y}JNdb-Ukfue(7~}ULLJveRJUTkp zziU`AMN3a+SIGbx7$+D?GF)i;=W=5{gGV;kSiG*0nNPHEXeyde z%kvJuSvEG*CJ6e1xhY19LeH8Lv zJ5;>6R7&PkfWUHS((LiXP3r;7b(F(Np)$kOM{^Cc{SqTB~5$yZ}zz-CKbUMa# zg)xB&v0I18udmj(%}s%q_!hO~VjvM2c9I69qoDS63=KsC^0ebnfpSFm&R)J;6?mC9 z0L&UiOXHdW(kx|FRa=f;>mlP#5P85o69{C;e5wTIN(NLWK`QyEcDxW^{xxKlOdmKd zMf5=KSf{3feH|0#Ez|)syN9+WC@_;Qjob>V2iYx}T3T7Fc#nUs6>Y(MnJVOQIieQ4 zzk8TWa3G|kjumw&T*&yVgos|O!uxU<06nu9gXaeh9o+!&q3)K2ASPzf^{Nl@30(W1 z2IXw7rtHmM;oAS!^&nSZP>t4OVy3*k{AB$uQG(TlM-Tw-9U79wJS@Jv*sO?@QuDU+ z2vbw!Eg{<|DNwhxQuPEO@#yW@`&`=mqlZrie?(c!;j?`$?e?f2zkMC_&o`MPcEP2yO za^6Dm*Vh|~D*Ojz(^tQHw+KZ=>)5f?SPV5-*pxcrg^*wWz`z_jMf*All1C+TNx(V} z9;~OxcsnTeZ(N97PEuaLzPb>_@oqSb0{g4J{t0aMxw1)y$p-1=5yz9aX%UVhS0W9v z0*c|&`4EFKb(SL7o|?w`sm$$nY^v&_obWtR_YqB|r7%rsdEfvo>dr5e9%Spf2+R4- z^knzB9b^|Hi6;z`9F`)SZB5KH5$oWH{G0%a45N)DNk(9nXbH%5Y?Z)T`(W&GqDBAf z0^pjaWM*b=^~AlXef##$^@|2TODBVz(12v)p>zo?U#`BNn@Cq-3W$pqz;hyVAXvoc zvca(K_j~s6kiq2}w@Uszj{EG%0A+r&Kgld?2bLp6A(Mn5G6Q3Tx1oiWvu>hOheUd5 z3_yd^LTCBLqs zfzO4piD{ru*nm9sbF+JKn;WGqvGz%(AJhK$l3XrLgYqDckRgp;tmf}1#&;$pX(}5Z zI0<4YR~Hk81rO4G&55IXsHWT12OmhxBC|2h8WHl zao`Gr38S%j^KGP1AVktoM6g6+tm5t4uUSof2s%P4*)zC7FP4O<38Vv(XP_L=lF)=F zkXKrL@UI1A6JWgKPac9~s4?+MPMyGw?AVZd9``HLi+dey!GR|%WKb2^+z`}tOuMvW z3?C)>07A}j?{*+s4f6|`h*j|`{~wbAaf?{VMYxGL(C1ffF2hOU^!`;Ze+5nk?e>HrhSanj~~;c z=X1WdNoQeYt;DvuF=s!dd=l>GbjDcKI&7fpKG-%BV{dmz?LVAkC7A~Oyr)ym~sW85r~3cRIuav_$G;C_wL;oEU*SmM*N2ig;CV6TtDOA z!w@9i^LMykk&d_o`#lgf0=ps4k#LY6P)N_dv@c-mb4Mq1A>Bm<0FILqV)ypC5cV7i zFe4`cS1A|()pL@cBKdK`K{0Z+3BU~<(X`N|=eRj$0t=w%NVYg{XXIxPo{jO7b-04wArcht+`6>1_-7SM2v@!%K%`4q#qRpmg5Pw9N=?a6nBw9NvJOPoADFuH> zkX&DwWD=eqI4$t@gy~8c zEOJZs5lTe%KYIwgrx=bWZfge~hy;#UKt_O2--Cw^bt2jQr|)P}m!Y(!rHPQ!i?FM; zFwacVxEQ@su&q}At%@`VUKH z=heB&0O@c{3=Of-La-txrly29Y~V*d`13Q%Nb0l_S~itu{fwPt<;g55;4^B+zixrR zzp}-IW>UYh#YCw(`w>};ehd@>7uH~4hKyE#;LuTMb#d>P29w96pwr0Ij(i`P7X?+d z2I2DfoQ}h0@cU$Zcd5?SmA)Fq0d@=_6i8NmYs{Yo?){fUOc0W#qa!z)h>B2N>sblAOr^$>D&@L<}X2l*&q9ElDh=mVPD_aG@&;Poy0&$l^= zs*!)T)CgM|x8YyeN{q(bfZcW1hxn|R+6K?m-C zgjA2L**zu_djP#aogsmoD;ti_!^n%eF(!txbP(}5gM}ZGVKAu?qDN(yAJz-P2eCC`FwxRW; zalL{Y$Ay{JcOUb^Lc~aDp^S(Ozm@1OZ!@C*GH}MZNl*_fp_k^g0Mg@7Fsbp4(C{v9 zba%YuLPW5~x=lI#SG7eJ4QoF6_DUw6@l0rw|tf z3w5SFRq}~CZxsCzY3gz4Z9lSjVRA? zmlUoUNMN|ehVxet86Ywl13ctx08ECWfj6EzOg_aFuhk_}Odi9fd--d$`ZROZQ&>;8 zk2bISQ+Kw}t^{QyKok2`0eRV%A~D3qVF}r9N!qd5J=;L6>)mb=@Y$Z?42Smq{xfJL z7_CUjuDoWijv|bXP>p~GIpS^X^IJ?~#i+r0!2V-)#?fY&=`4eLj!T0KeG&jRYRLwp z`fD|;GU&eW;mV*#vLWG&BSumXSGE7*yDJ3LZ#dp@J~==F2V>xWpbJ=vivXlhK^DJo zu$WRrLf7m1HribCLgDe*-cJYw`1ReNd4)*QdZ$)R%r+(#t@2upozwEb2@&ti)3D@- zWlc^jK%UHoJbWc30eBxZrb!T+@ zc>8dxWJ7sqgp3K7?QCLV6o8M+I!n)_u89P$jh zXqo8?XKUgb<0xG$rCWLIlpQorBM4L?&M#YCS>Lzy^15y06dZ&{M4@cifmRy9 zVDnK$akpB7%5!t*3=^BIyA`4(if7t^!s^XJFmst6Fucl=<6 zMs*GhDIG_S=_AGFq6Q_3i&`lpbZskJ&sC|Po<*4ZLuz~_=5f6+Z=MD)stZI&PV>2k zK?#=9QVc9+K?aM3Y>kY7__id~eW)_$-?$?wmLO*pqyRvfg(OmJPEL-`D&wzqV@dKv zD#WuVdN>W;Jmvd?K!j+R%mhj^$x9MWPS5gP{lnwMVXRt$xX{GanQq zoGh===i*?0tB$vVA|pQ2ivvwwzI;h05f@?*8EFu-%3aOr5Kp(D;XgCVYpypep?`lL z;D_C?qTvU^yd-vx$QgVn%kIC<60ZlHbJxn%J9%mkp4foep_gv-+~2PybpI04hRF0i zxSQTmCD$0V&NN6}TU0sJu}!Zux#NzAGtko;H-7AV5|NHhQ1uMQ=EcHdJ%9G>wOzMd z>75Z)h&9Uc>DLu`q=r4h7mq z=o2_7(6HzGw;_!6c7z(?_NT!KD0~fDgdDatxgVl_1QcBH{1cewmcqXHsdg(eTP-q* zSiYPP1S^rC&w~oH?Sxi+7&4xS7=@7GNhn{yCG^A+0%>8vOaMK}Rc8A>d;50hQTb{d zD?*eD1Phr_dJsiI_%o!*ogBS#;#rVDFy49Z=LVhn=rQ^)@5uxPDmX@y zTrOcS34Yz#ix(x@q@cWnVkF!A8JYlgi~W&{#Kl{&f{r4m3<^Q%$8Fj5We7PKOpw(c z-^Xhr7%TF8qv^nhB3_^t5_^hM6)ix-(-@#XI;1hyv=CkSlK z2Lq=9NCevVO>67^lJTrVZzrVNp}Zhap!4>F!CGq|O~6+x5Ob1$h7}f$aXRrSM}&;g z+U5Z$5dRCuT?zV*aRPSM^~H^6a!{B|XFhV|2rQS5a4x#+Nt~Df93W|v@~kvIQ}Qb3 z)X<$;i&orhs07adtLs&<{hs0BCT(1ny$m*12O2lTQBqsf5sW0%j2!0!M~pJ~c(^VV z2L^T^*$k-UgJ?WCHUde_442YRI75q=PVuH7qsYrSUZb_Y=T~R?nGUzEqL7F7t<>P|J`6Lf}1&J#N5+v0H$AgSKBMFg; z9s!~8l-bYG8UG}d8FGRQ&e|b~*NRuK2C}qk^^;C0D{+~&a@qeHTzE!~3IraC%pe?L&Ir zH^1o-9ddXOC>rqJ6Z8c+NGKGC&T%qHhN4C`5+2_9+4Si&m}p_(^<M(7ju|1R41*t}LSZ4t3&0yZ*5A0AqyR8~ve#y} z7Nve9+6X`uL|O%;{|Z=uIJffV-wIPF?iPjxZr!Z9u73gQ+P!-U=5YE@ZV)G+nfg8u zvA)%VZiV)lfQSfIsB0sT5xdZU$XLj2#}7*R>oCv~jg10VyNkS5Rnj(XX17WBJ^`n&WaZ0 zIMl*#k^zMqR_MnIIZYaGW%-eme-0V|rQn^l+Smb-^-!>W%9aHa5#{_mPH;}cbhsYI zZq7q17SZ!kWIl*ZVT6Y$0$56LRxQd|UDG8cw`}906GxxpP(A@=k33`!F({70PWkF=sb!t^&As-ckbQmmY1u)2rO0X8UaiE;L)S2JiWcQ zU^L6%GW|W*&qmKtXyn(eyMTR?2BU2Y)!lOn_z74-ydG)#qx=^EDRfam;~{d1Vm;U) zvt`e{X4wO_Ga@R}o*zGdUP+ETgvk|y3}@;(=5F*E(s4vFAoTpj3;n+42e(`?&d<{9 z7i(j=?9m&vG(2vp$sa$|sw`wp*0AW+_OCGmD3C_7Q7v8t!#xcL`HUlh+@}E_dvW5c zNhrciX>q~*^zu-a;Q)vI&Ky4#9Q*oH4U8qFkEE&v{3^DtT`GG!`MhEzf* zlu{W|B*Q*$ty=5-9q+sMaqR#0?pWW|cNw1N^W67!U)On^=Xu>x20OORV-;kjC~BUz zmWB~U(S}eIO%F38zM|0F*@wS2cxjq??R2yE^0D#UPwClsxgT`%I_P98;=SMVkdvFM zob(21St$`mFE95)8)al%{{Dcpo2P?}JiXRkyaO@g{K51)g zHuk;r<&BT2@wN`eVgHYTTv=NUZ0T}M(@ir{S9nL=)NB^eV>aPUY*XN>G0iw#U46EC zw_VH{8a8k7)YRIemzgCs#S3V4l)Hs%zZKKOgbP)a2EQtNHTdkwGo|H2mcz0io}5&V z7h}=DpG*a>`i+z*{%}XLh;VUnojz+rt3|%C{~X1Q|8`m^5kh{@JmUZF-)`nvqZJUK zr0CKfma#4N-TS8(AJ^4|78WXWzIpTZ%a?gpR#pOnf+v{6BO@cV2A#YZWu>L(TUuJW z`}jP%Y4@n9#ZLiWqADb>5F~qHH_uz z=L>BNE1#Lv&D*|vH~)?uI|kl7SoiGGhMl=X+hUj4)WwTCc9m>6qa%LuZrhHD316?ToeS;l?F(+*s_c1q%wzoP8}5!PSy?vqXI5rg zm8(2?^2B=IzOV}l2W;C4Y}R{!K4;!>cEk4IO@acW&p4 z7knJTD)bb)fGnk=qOw9tN&_oyl4l-1JnTL-@<`aRHJgj-=_>Uf=q#4AwX2gRQJ**pZROntdW(K zjq9kbu1-Dh>W1%d56$HGP^`YnPx^?62rVtGif5NM-E@7kaj{B(fJ?{SUD5loUGCoA z5n|d}dU`eY4-fYA_8zkep5}XV|M03evsxO)BU*xIl9H}A|Ijclqo=3WYMi_o9~UP= z6SqLxBHN_wz$Rp~WBjR8r|w{>ZKsv|#0oGhy7h|Qq8#TJ*NirA4z>6? zG$gZd(IVY-Ze3>@u=2}k;>v!$UsNomti1PDQBl(8&!00VCMIm{7VzNn_u?XNA%+&3sVcX}+kRQ55q1`*#}Z=g*(RV`J4XuJ5+W zR_1bwMJQ7gH#av$H8(d;d@1Ud4VZABn4Da2z|GMyQl@;$$TVR7{P{9tw?8IX1#J!r zQc;;6+gZFl&7h*YTe!&ejk>Y%8H+NP4n?oQm-BEIJkKX3@jJh|5t9-;VB+;CT7P{a4fVkL3oXUIMCqRKWVz3fWzN_)s~ssS^LFmsdGG%HYl-FGo^A3B z9DJGQ@#}|6W>!|3)AK7elfEA=`wagYDsLDHp8mdDV9}xs{S*}$A$@y$iLY-SFloof zojn_oUpC2VW@a|gbBx3CzKp2fH|AeEhmUcnRE&2A$Nv2GAzCm#vOew7rNx~Od^p0x z!|!>0Ly92?YO+L#$H5N$v5?`8V$7F>|JHKl&e%bOgw~vfx+~){k$JT-LVpS7kGc!6#9rU1(P9DUN-dN0Ac}t2z@It1`*_{{8!W z`3x-$w*@vVJy@RRdmeVwio4-e<+HXD>FZuSKE15D$jxeCV4$Lb+b=G~?pB;+6qp)t>HWL$* zo8E`sKRI1gaKuFP*RxGS*>CKZ$XZiK=4+f=S0LK-T(Z`1ISJ!I9v)-ToTsVLZ<=Rb zn0)b!ve{?%ubmt2I6Z$~9O%+jval**-p*ACVjsUB3t;&1Q;9z z-}6|h`1`NSDW|%+I%L*(9$S8X{!7aK3vg7Ni$WL4I}W#ud>j}^4gB@P^zOlhH!Mp} zAYqhw32ThK=M3WQ#JMYQXk7ABQOj()Kq7rK$AA1Q6(SrwHKN+#=kIUVT^8t^f2$?i za9Lkp-{VGE6)qv6)vxF(_PCWY95`^`(#FHrOofrcJU3ATFY^{HQ3;3?+T?vk=OlKV zGI*CO%H_n54>|p7{O!k&QSF5XsvbSsHZ$lmG5q$Ntj*DE*~sim--d_PK0LcT^2Vpz z27lLl`ZhW$>+w#5igkESBFb`qmP2E5)2mmHURedNPSTDKD=3gH`TlZ`}C~^@Y8Lx`dLG zlVueYYLI1V9<}(4sEygv$Yqw#iE9bu3E*$#xJcgJ^B5G ztJ5AcGn+@Jga*1wH{k^%XQqE;psbLGH8SG9d+;?a<@}39g=@)@Q#g$@z5!=CYhxC9 zeE+m-`}wt+LC@kN@0`mPy6WQMk|pQxOe`qExFI~Cw`nSQiyUrfAZP*9Mw3ld}2 zk#Fy{@k0sh?ChC{C>_CixCSo9d6|@JFo6|*wL|Qe5jkBi;;lGmpnh%tuHFN6xdWT z{w!5bQ!_N=Qs1(rxh&A%(1w%1No53JCj%j`>PVPPTx3n$KpL7d>lE@U8d%A$9B4ttR9PRMRuQVg#FVIYkuE%dT0&h4Lw$ zJbC)uIn#HNiUiC`23B&b0TLlZs%mSu=1pPq3U1!qY-5uaDGU4-c_+C*Q>3N*)HXlt zxvmOznS4=Rzjx7QCPqeD-;z&0lVcz8(x(Sm+1PO4^1R2$8*(|Ft>vkBepR2&SBcIS zNkTwC;PGL^vtwI9T0rhj&7H=^VViuuR&;ijRKM0EPZBJqU3rX&qo=2b5)!gmx21v4 zrjeSzNRhqa>}vmvQzF>81&f7*8aA@yr~MeTwY9Uc%4Ww6CBiN<{;zV zbH+{kr-FuQX9N!&Iz+{mtGOB3>S}0=N$kuqcK7l+6C8UWW33fKz)+WR+w0e9si|rj z0$Dm)Mw@etMdjoY2zobUc-Y-dYftTx~ZUwgW>Z_QW zn^)G?v$BTMA~kKz(n;*wohZ3$`0a6FQ4tZf=Qaim-l&rS!w;FVEsEEkjEONU4+EOp)hI{SwR1hY&_>=p zMXrJG+tSj~Ov?f`yv(=a094+dq9WAQ)pgHzM0EUym}okyUN4I^Py?DY78Z9;&v)J{ z$BY(8nNr)fZOh&Jz(o3xx3~1mufSFU%6{@cKEFP8WAB4&FTQk@E}ER2G%fbbD(O^} zl474HC|`j^G`Ma5)OH$KGfOwgw*8LN!0X$~DQlFKg2KWYV9Vz6;Mtav?FfLhFqfd6 zdFBd>H+i!Q2Mp7g-#r-BQRGHP1x>tV!WXw23m8BA++N70cTrIjmBGf&j?Tx&C)cvn zJWmgYjS+P?HC3b)u-sI*xVRX&)}Ujs4(NbN{`p2%OG`~hhZPyUZ*7)Q6k42Rl9Jct zrDGx^v!5{QQbkyCH#aw6on`24@at5_lPBAsTwJG5-nR3e2OZX39aWMPtchR0?4F!k zV_F_m+G&p-r|J3gs&i}XbUtTnIX+)YTl*o&yGcpM5nhXSUfE7V;V>pmoIi7hmkPTcVb9TOFegyye z`{BNP{_KtG9+xj)wsCMc4j5LvJeP)*9_mx_he$ExB1q7U2H4|4wDMt z|1h5a9a1_2WqiY8A)`e0f3M*D|Z?g8`JCS>jTTA zqLq@Bm#?ml;#uQ2=1BJXd9IlpJ|f-2hY!!mIn2Lw`SMM#k4Ef*@&j3uQBhG3pPZLN z5@7(s3i$SPoq&>Wf%(OTB$;IZg+Z!LEn@o|9PIiYpML!Ku^Qk!^7TVyDk?I&lE-_& zFb}%Mdq4>&BR%imGs?-yovuofOcn>*VLUQCUR1HLVmV5wwX5s=Lx;YxUY;WPewT6O zf>2&wUTc&C1Iff}%hHY4cIVH>;X#kER8`f=_@7eM#@<6xbl@-Vu!xBI>nk$S(`665 zlej!>euF^EXaNNTZ*QM-6Wvy1I&*m^~wD+tZ(jM$8F$?-56(lj+3hQU!1x15 zK<^Inh3s*Bd^{mh5F?Zt`cKlc{`=6WI2aN;E}7;A0K2BEljRn5Tu%ca&wtM3w$XHD^Vix)2je?fnw-vf>EK^GUon}B~29vK;NKXj<7E@64|9jBdC z?7Hk9BO?z{r+|!xQ#=rZD-w0~+1poER;G_h4-0?O?_Tv?CFk9_ca~YXTSP?tmN`w1 z`)C?0b)bB{w1?x&q`P&O0n~*)c(BPN-*WY<Gs=~ASX(T3(`!;D^% zaGZ`7?5P+k`8J;rf|7x+J36YKKGn;60_qG2kfKP1S)tf_?!pDhSFc~cUb=dD-Ons3{S(VnQ! z&P-c5GRR`F3^m22Xtrxm~9e!Q9@i!|!%=cm4H8&g?X88G3IoEmKehw+kKj{~U&wy$(r7penM zMdXkDqFB3_hK~yesbSd=wjmeSxzSSOeF*ij?{_88L*jFOAxY23So!D?6W(xCF8J)( zv!?gGa{kRT!`Ji6DuH{~fFdT1Xg}awP4S@*8sQ`6I+7ud# z+x9U5f{{^CED5Su%Jj9OQZ(!=ESo_&z3uA@1D(?QV5qDLz*l3QK0c4eEazFuveb`6 zFf$8FD1MPNyRA7pnUItSTP`9=x*yTJ-4YHoANu-8DJ8V^aRe?)yclU_yae)N9B2|G zMx?AoMW6PVRnYQ>5-#ERe4)y;Eyvv_$DF|BrSE;<-3AJ_{jSTZe_nFA6wPOl$spI( zAT3Ry6alvID#(+=rmWc}7ZS}cL)r=Cd&0}uVm(Mudi2-x87?I)D;K>)2%!;>kv-5{ z;p3$4Z#avkP%HKOZcymF>(ADtE!wObABN_zw<&E4A&JoaM5L#Sn(o;X%|2I%t^?^J zm7eN-6uTwMD9xrS!gu_Q4}r5Frq4Fgk*1bKOp3;7^pS)AG~59*FjAZ6d$+|0LUO zHKi5c;tI8UbgFI$%}CVN7(p9y0>FEtQ}`1^n>TMZ$T6lupl#VyhOrEfjBHj{r$t5i z&(Dy2b?6Wsb+F^^Jg6X5B}cv$c=qc6)0G=rVTq2ary2Mc-@Ettd+ZWrbu~3ga`z2( z04#_Fq6!MT@?xJZ*TAQ z9`ER}y=(S8<{TItB+X_1q4Q+L`T+wAz&tavvulvFz{7Lw>BV())D4T`qt|v_FY^jU zaYu}R9t!OmXH0~!OrVgDg>e!B)gwkzaIE~mjtdF z#;3AtO5HryrB>J0ilQJwfb`sherWxM4NKAGQJj2y;gjD7N!#n%3)M1FCmf8X(xE4v zofg1Yh5QI!L(OFe8Jp_rSdfdPP0IGw#V38ejxugq?mJ-PAAWv)+iIgFIX zGYT~SeynScQKd+i>Xpf$uI(PK4R+#s9E1 zTP!V?tX;ddwLSC7mHI~Z_SdiZ_V7*h1!_0)sX#no26_5t9i(V30l%t%eZU#lcR8(z zjT}bmEivhataC4DdgJFWUz#(uc@QMa)~+?WQ4SfBn}_G^+qX2-@bGX2iZ*m0iK5p< zZo)W_eR`c`NQ;OKDHB)7rwE2pV=FY_8YEqV;pr?s#Ds1??8XggMs@)T z9p&4q`9{xnXZrwU4SOVw<3AX}qf|HYz0^~9({=M!x zrnHV&1?(L<#p{VFDuL#fTvO?;Z|;ZF&@uKPAs5`eyA>H1GHY-JNVAIGaB<}8;qTuy zU0tOK>i_=Asv+goix)M}x88!h#~ z!-Ccudlz|lco3ZjT(a!G$8oQM<`+o)MfA>nM=E|^wQ{vT(?%o-YQVtTVP zH<-~>u6OHF0*6r1+{}d)M+(`}ao2^g6WcE;>it1C1BUL`cjwolU-OumI)dir;hQ%? zKdk9m>})k*(87*~(J^I;MVB#cLo=;y6CDXga^xK0bGdmxK($18`N5cSQ-- zrl5)v%!9=8_lq4&?l?sgpKmwDB_~TeYmrxw3@vc6Hpw*&MUDWuK|}b=oSYogSOoeRi2=;qQR=HBrGAd^Y;?I4o=x-=@2Y3M^vo@5UITwtC*|7;4w$>`{+FC9gi zb6tGq3qZEunV-vvaLT~8pr=8mTx082M#}ovk1zQ71*p)?2hO9Sft&@GwmWx6yciB4 z12|sLL&n#2w{O>Ybz`p$XtPC&7J=RYXLD3eI zO704<5rj|0Gl!o(ecIwHByi{mk?6Sx5-%$q;qDIj2L990z;?PNOIKIvbvC$W;4@8} z&K;N5he8RlL!kg=SbOLZWHL@^i+dpu$!u^y)_e9bZ#}ixO!AewMmrF-?`WTP01N^( z_1%7ht3kj0eisCS4W5rwOUYf=jB{te51HO^H^b3Hkyu5svPL`xe5C9=T za&sRK8bm5~a(Hx9(j+M%;TVK+TPV$QWkLAg3@rWa$Bzh|ED)jmI*VnHp3w+6w%^II zc;Da8AS5KDv29yYB_G#mML?$?1I=_eW<(;V)Y1$@AZtL2kAT2W+6K#ZG`yT()Z^mf zxCbA_3df#Zy(b+yjQGo|nVH$*!BpRP<9#~t7Gwhp*D5P3o0^%8++g7q4g^DFb<8K6fSTU}k9{o?ACE3|4+MJH3vT+PbLN#x_@6;W26KA*Bw)Xxwh zw%okPH3l^?ddXP?7233S*g@z0KwHP6quEF)#lG|I=`n!X&0nCJP0iL$^XwQXO^>-Ct{?v>HHxe|U2K zTj%6#`~1+zY^ie>FJ_=`ao!JU9a_gN3(J<1mr%vPOVUuVn;gA(aS>tM!J!}~t-HYin+e-b{O-)aevIEz`#Lu5O(>8dLV<0~p*!roSt0_U1 z*@hWCA3w4nu7Ig*e~$L=_h8W&i!da>@kNhhA}e@#O&p3fFu2@e?M=-2FXhtuD5(*EClZq{}lHR|z`CqSb1X2<8C zq5{(S#Lx#5F>Q5_W<;?dg&yHG#yuf$)VtS?S;I{l(HJkT+uAhe+ASQyyNA{T`#}+JF z6b(VKqOy_}!dDQ<(Wl=tU!=JMl?8w{7MeAhZCc-e)Jd!tggrJXi+cvfD@egG*8_tK z5`n1R8-CI)5su|W`A|6Ul4P1W4pEn4V`X*!*pg-2pCr9k9j2SV{rsK^L}b9?7$@TDnh^qtBsVFs%yEZ*>DTRZJ$^DB(_%|t;B)7g6b16{rwg6s6^GIbLW_YgM$|<`4ZpN z@V9SUw6vJV#_IREu#*+#6m0Z5D`l4cLj@@Vb_i$fv+8NS{q)0qbqN#iFS;Qi(-n@6 z^gik>>JF+xno2d6B$f*lZzd!zrs0Xj@1T}dK6^HQ`HoA3UX5U1NLW+I5krZ#q({2O zET2pQA-fc)=Miwx2#{~NH-TgVC={eOh3ao1-T355H8I-6ELxY2;HE-AIaGovgLi_- z{ElNNn#6bSYNFf9gH^#qHp4;z!kiyzwKmu6wzakOaQg@elEiI%lM5?CLMYN308l&k zBSSoHY}~!d$HOBO=N;ZQiZV1bgiEfheYFi13p-b8p$Q1*XO__eBehc@M2SJ!f*1uyD+fma zBndp%v6z^7l=a^<+_B0^25WA3ka&DZwF~n}`iY!L8=(go9}am)$|EtxC?_l{sa zzPu1oQMjhJK+J}}CZN?^`m)*c!q!oVCg4_oLwN)2X#K&fcv&G*n_=_pdBPJHIarR3KJ#PJUFOY-SStfh}` zewy1pCdlgm3sfB7Pv!I&c zVs+<}MKGN{+w4nwE-sFO^-ParF1ls3dK)Q|>n3{7w~(sbtjl?egILMA5{3UqQCcQN zBPA*6XMY6I_@{Ux<$$9gasWUU@k78If{e4`Rd=aB|FdV${yhM@i1FqT6wDFWiV2P* za>2WI?+PHgqSvE(EJMjkh&lx@y2I#2YSAzP^le70N@dQ@tM21p_YxBsu{HRbGn3Dv zg)j#ox_+168V;8N5H)0@;7o?|yV(nEDwdiTF#}&}WGskLIaDK3Dk>mO_zjj~+d0U|=xp$v?MoG%T-~Cs9LT=oSDeX`1^L2^FzZrTcw1*lU7PP(T2n zcsF_tgrI1=L%%|WkOB^~UkwVl`^2yfp(_F>$4;wSnVLqTe2dD+aG-owfRZ5!A9&^7 z@6u-wHQBTVT7q3+LrCNUtvNj4O41BgG~GXZfHYQs3seY75qQ#Um-*9@$nXWAv`faj z{J~DtA{~&FLPOYEdN9%&nwn`ifZ&d{!77YR6#|h)O94X59vR)&S?a&LD`mL+FVTRn zzbkwvQ#bW|T&Gn3+WIT^cCQ@doDgdD?0>sSE0g8I%=BI^)KB-(zPjw)`Ag8R5b=wk zeCQBa-cx&QY?cy_IoM#b;EIaG#EB*^p12$;K$rmoP$6J-Tl1}~cFa%}I6pYn$(mAd z)kzymCbmbD6-dVs@qJg#5*8Mgs*@-QiCH2D7Ght2{|x9nut?Od8uy2JjO<`j6DKiE zd4F!B0oFkm@dqz_fpL&fZxld-q3%p_t7?w_SwIG;q(A!y1}5r92$`M8hwwz=4@>Tv zl~FWV!E0LR%!jNo_a~&N55MUyqgLI$^7Ayc!;h}*GdKp@$5p)2AZ}^8>qfrW4!sT1yo{2^b zO_& zRwN{6f0A(-0p01#iA-X4t1rBLB`@pmsh^{GS?H@xe@q|~?7wG(ng`f>%CVSipb6A% zZAC{%M_WHFkg=3nJ6FTaMr!xf5>^8nH-eSiYI0+!lKMIRSy`zYoCPp zL9nIl+O?ICMPTFzL-?ja-v47}LE{54iYz&mdiCmYV64g~PuM}1z<0*>ANdF+Wb{xl zvJ9Bp`xZWs>;A|_FkRjSClB;-3>1ApKmai*kl_zPCc|(1pBEv5AfcD7k+h+GRziR> zyl%4k21SY~x0Wd_(FYr56%`aTVPDBkn^Tcib~0ZeAL;+pkNt7WFz;coKn8`tKqPk9 z%s@aGT~~GQ3Hg@>bcY3saPjxOznt*<{%LcXL8Ksg@yKNGQJr_3c!FkrI$(e!62k@f zqa*Tpp&A%^2!K0V-RFuHI~I(EN5cB`t^L5Yt#Raw#y#;hN_Afn&Y%G7^B|AzwQLOSUOXsWvLl`b-qS&sJ$1;W#lCaIc5`8> z0-7XPSqj5d3jvA45T#(Yc`4cdmvUE@Z~xp|M`ILFyIKtiFTgbrl)7c{5Ei(yww94- z%V+{nCRhOW4tCz-8Y&%MLiOM@yB|KR311EAv5829(H&5xr}?iC3nz-}J^yiOSm|aC zSzwR^$_7lN@Vk_FPBpHD3C?IJLGskfue<6u(g!gWS{#{}&+T;dN^tOQdPJ4g@`t}< z&`(ezZYzYq=U`+29^pC{3k!>$zCN+#A%EKNYx~fK!{lyh2=vjcYYQ@8AHOZAKE3k%W3aiOv&!Y zT*e?P%oOO*$f!X~EFU*FZI;o>^2I_z3Mvu^3`3{uO)Nd&+zR~1T?tnL2$9$TtLLAd z`v;I2$ex6bDgRx=3FG?hQ+v=gVMd{UH=bvuT2%qZ2l;Mg(&8Qv={PF#NjoFmkye`tu9VFwD{ zYA#{S84*DRhtlePEcgwwhvieFY;g1UJNzIMhA}Z|U}s<$BYu_odRA0c;%LVEG^J~@ zQDpc6X2}RxI+n@H^FsPNr|2RG0BY^%XzItm9V>4946NyMFu&!Xj5c~{SzZDi7mA0o z0}%pH7xcCaW>5x-I2~TpqpgdET?4KeDKpyNOr_|$b^KjjT@#ao`TckQcT}ehtWf!f zFV98mWw@hYaH&NOBi^*kt5=EljSLik@#|PTsi_rj{BBTtl!?MH1AN%xm?}7Z`Em@T z5`O=?`G|-KG{>jGoRTa8nItUPGTcf4eVf}kh!?4l>3{}8;ADj@JGw9`HWtZ284MP> zzY{otOV_M1%+o{Y11nQtViDHF#_Ee#Kdaek13^H+bgd;IC0`cYJR>| zu7#DAFpkDtLcZn>f5B1UY^B>lP^*Zf@93CqVUn?RUZEmfAdjHS$nLM_EzZT5S)-88 z$h#WCUgJ3#dq$$n%BO!Ug36Ndl4F+=PVFXi4w#>Lheh*zjkR8bqy)Bp&VM-&V+03x zchIkSfGiP3Me7b8JVrXr3(%R zoLuCI6D@t>rzsRRjF+jwJ%f{t)5nMwp9a|-O+di+7rW;^n_mVF$WqMmeEZP24V~ak zm{riPEkghD5XqE=TIbqHQBN)?un{Moh6X(uA%!K@qSUXjI%w$4gX8EVxb>gna?;9{EiY{A5#y8LroY5OLwns=1(_wkgbkj|{ZE$rPN;ft%RDq0ze9DQbJSZ+} zz}3(yE0@|H7r^_~7x{u{ph>M(oV8pzJnKS?5Nf zxq%x7FNJ?FcWT_>SPX3mjsxNdyix`0?X?2LXE@%xVF(b2hC+Wspp)PD*BH?KSZ`u^ z%Kpc11Jb4c4snZger!ekk9|%ZKohzIK&HFO;)~p_Bo; zO3=~G@TJBhD=;`Czo7!ggZ|mevNp6Z#P@B)8)Nob2w)217ESvE1i=U+Wst(j#ujZf z6g>0MfA$T8oZssA)O_dS(P1I67Wz3299+`+L+`g!u_>xp%aryTeI(O~B^NkQ(^FF# z7)->U#-&n33h^@wVBh>;WDWgOuDkOH8_$)<@ejwc{k%Al3h+|wGcdW`VhshJ(OLz^ z*`t_6$B9p+qPH&}`YSP`qKfs$ND|;W=k~)$j3#mh4Mlbj)4WhQ#4)zF1k@SdzbLEg zfsek5(j=&wZD4Z^F_|7P@%H$7pRbqxa5)9^O)!Y?%UEilkwyrT;HK1e?3nKbyzs|P z^o4Hy!8X|vn8G1{99K`9^J|O1)&b7$4_3q}hesw2+MAY+jym3r*fyvpSQ9Wo@E5AX z69l$^Z&fds2U#OqC;9(lQ={=^8tgCfJ{>>+r3P($`}XbDc6R^3a?Y3=v^%u4;;gI! zMi>MYoWsy<{8L-tTb{G;KD&Iq?l;%Evhr@|d6mG8L@kDgahvpYgcs@=p`xSyG&Z_a zLPCNeXqem*L@c1dzaLEvu}{QoWa@+ljIh9JAv46HMLxm>=s$2*&I3!|1Unh1`AQH3 z#10Jof)LYxU%!K?3F=w-?93)&OdHj;Nu-<-ibh+IhLcYEHD1>6Kiv&B1jqC3hm_!m*x1^()&X!2it zDstN=VV%xpPMYT%6`S^0*p}B3OtA-g()-iO3CMZuEiR@6fX@q{u`HQQ{%d zB7U#4?4gfG1?5nk;zZ-;*l~9?(h(aI6BE0z%3|18|6O?ESXhh^eiG&I2N5$fjJpiL zxwQcIf5S>V1gN))=t?M$L`(!ILVbJ)jp8O&C+X85W{}8KC|cK4gB1nf^$PuP-Gf~% zdmHT9n6*N^y65-(n)$`CF)w1G#S6f6S`C4ajKJMbn)4Tb#eyI5^t3*3U{7vmZSDIk zH&VwkUyxp0${OMbk^cZ%bzLrzIkFYr4_>3CyMYN2@SZW|rGFgx;&o4- z!qs$R$}-C+(WH&6OC|Fgrl#s@dX#+54f}Fff4kOxk6os}!k~>iRL(i$NC22F(*o@)9`#tf0Uyg}Q8AtRKVD7`$uNM;FcHowpjU%*uS8=eP2d7v z^>6-Arn?8g35Y=4?Ymax@4CLgwX0<1AG7sAM%hi9_PD$sib4!Y(~~+$NToL1;Bm06 zU~>>rToDTJloaY+o-5I}!`{%&pEoZYu5AK#N=lTrUyx_vm$p#Dg%Fs%k^c9k3s5re zdG@ms-wddH;HWv&3*QEK&Ut^yFu4$lsPdRTIK72@?&fmXBu+*~Rsh8Jjbj!>1Hg=2 zo-&1Q=|2fHHB|&an_XQL*lEp}0f2kowx&fH;Q2F^+S>4(fmCcDT!U)6fz03~ypSj|XZ2_9fFK7(+sH2%XDG0cMtM!hvB35Ibnuzwr#G!auVGuvX)4 z0Mhs%`nVP5%<741m^gsia}+1|KNK6eGzipBTtdRKzkb0z6`&<6;gYjKPDbt70%AE& zMOu2tAHxb_=K5}#zvBS96iDTj&MU|o`ptKxCt76Ye))}bj{EnAlkpxRMHuI_92%d| z)z^RZAA(r-tW+p8c*+{)mJ5sLyf>C{TY1z}Rm-_&=OoSj#~c6eZ-WY39)i}s3L!^a z>{JMfKW4VDna)5Cm_Q_>YNY*(jpcyrnufxPJ^Ej3E_%-6NZ-(l;6fBl8i%@oo&Fnp z&u9TZWyqv8OxV{FUtQb92Fhx#J0TpPBHg*mm-7_AjcedYKy>(vHpN4TjTx1#Zybag zx?g5|6Rt?jaKk4cBMs3Fk$aM00K`1Y%G+~_f(9C@^`JYD1c)pKa+{cO;P;@xlobly zy=>W55|U)#1{c)8K1QZgNqNSA0I0x~2_&UP%EMALSH$2PyyN;t5N)Ccu6e;E=`mzx z95BK3N`u?Dh2^Bc8d|{bV{nt9+LKWY_>RtKE0MA#ii1f{+`&*VPb?likB8v|Ee5rt zc>CYZSaL)TtO72XO6+#9x2L6`L~XSVIP+a?*|bqjs?kDNZqamX2x7{~b^5evqHp}J z;*h44JWT-9L~>sax$`DFTPyDd$PUph4`G{u0?UkkywDT>vcTw4bt7q|u>3XkU~Lq* zmpK)^>y{V2IU$sJ>&nNW9`G?pC<;nCKQX>S5(%8Z?tpW)hkw(U>FP)J!}khP{h4ll zKE9>Myzl^EVt7T><1Qt5({P0Yp#IVDa80x0Tg)T7H%lJCBcBq}CTb3VR{h2^Yat(F zHd}u3#~0F?@m@Ta96W_C_aIXRpxK)npX!-nVI z`NY)=mLI#2mM0gikJ*{{3l~lR5PF`Tq6rBNrMV#Qbd(HK=R%1>RfAN6C!5NV&ClexF-kNExZ<0V##Z9b3+knbNbTDKhX zh~!;Ij8XJJK@qK-J6?V$+wf8%cQ9hcKJ?V7f=!JiY{q0N*WN^-fJhfxCrm2aLY}iR zP%~{1t$)G#UFJ+QCjgplw;ec$Rres*WB8?@2DlaYbO-ViJ_BrT1Dpxzp1t|yGe?1A zaC1@&VmI;ddgJ*BP9>kOi{WBO%g8u}MA-|M*XSUMHx4chdT2Hn$uOEgWKIlS4FC91 zi?+LAqr6`gtWnkw7kz(yDI(1sum{BXy)NaAx5(V|OiLa`ZaWKCr@=p1Kr%rAV+O7; z(!1HKo8kcRDc?)00uq~e?CjVeu3Yl{ST^$%xD%ornwPEvgf>(hzaSf5> z`ulBQbs=wg8oQ6}#(1`sogLh`4=YkXbKJOk>l-BRG?Z-u*8#D~%@%zhKG;DOIW%WG3Zj>CKxX2gY-*U;p)F z=M61gZH=q}G~#C|Ele5s*)O76$Za}@zjm6T8;OUhGz*ha*@frjo#H{&tj4Vkrk>v3 zXMw&Y`uqFqzGGbA47u7S54SyBvn;)zX+Y?ROi)eQc!b0bTtBhrRzX1=9M6whN)04^ zK=Fx|VwA*)T!(ZYakK|q|NSH1uC!)s*!HhCrE7ZxT;~Fi_7Z3nV%nOCWTc3?ipw4@YR7hcLq500w1IuFXjYWF*jBC z3NsiAKe|ILFGB`uz3tFw+mG|aO*!Fe2fTX=tjL)_q}rJCF@g(T90BqX^fLJt{)(QS zRmLG8FNZT_lpvueh8dwcfP6t&>yR^;dW}t(yG9LBxK}D7{!{W)iL&o-D%Rl)%=YsZ zuQ#}6lyzbs>h%lAVdfY-l2Z*HXj(dVvkuNKZU|Vem#kz~-`JQm*im?3dCU@Jb4(h= z!$>Uly1l*rl{#O_*4Z;aOdQx#!muNxKNP#5TfI?dpv8&Dfo@N35^`)#uaR3Pdl^&! zaaUp}5dNkhHH@cvfTIUzM@xZdC+-OFZ9UL834w~QtU)m;jzy1cgFdsZEN~Opk?5G4 z9=%)1DQ~;yu?%yFSI)Ct@g!*yXLsUVw2Za0vw3b8?ovSsBiDz3cp|n8TumS%B0?dm zcW%lWfEj~$CD66quqrTw6G|>>I3`E;V^ETnlC_Qk69Y;EV13VO_ab9dy zCBiudei2*X2++rjARgI!MjACKcqt#@^BbaZC(JB*3AiA%96~9{=_=~`AS+4;Dz+xIStd)^m%Lv>DGgSA5w8EBoE@o$!6?=cg49iw&PbvX0RxU!0O@nWgV%Kj!dm_!1x=FTZjT_weR zLqoGqQ~;}2D1a9YQYCSStf8Z;yYs;KK1O$pXvG`l1bp2tR=NSLsyNkx)kt}5k1`hz z!^D|$=T;=`z30A}R-~r7S}T5R!hb)I%xot@d-TVTvrq=4`4%jw-+%a1>uGg$b#ine zW$1t$Xu;pDwbuwY|qzJ0hbKS6%bOb%u@ZR~D>#ooQTxgEFmM;n72_&JS3 zCZ1*VGo=@U7DlupjZ0L{XUQolo`nyif47B2C)-mB)WBY33)3jO>0jT-m5jz%{v4Ef z*V`C@sDmaqbED|Ms~yg7?iZS?qUTM3pMEOk)z>;y1DEYVQUl9i(|iGXWM%v?%9<-Efo$UnT-I0&})6zf8Gx`14WW{C+#V*JBW z17_3l@FY0=-~y7u+rLGxMkXHn!^Pn^fnt`>fe&C-wup9WmiE-_JB&gh4ytQv(m;sf zLKA+d6=ricYJyP7zFdzqwhE&t#FU6jV`wQD5Y@gM0vR>xlOm}IcM4#!BHX(2%It}9 z(4~kY<%)jFDNKV|Xaa=|U zhT@xg6wR&xNVP`+pAy5D%yy-JsyopXoTU7-|1zdLa5)5~Y`11 zJ*omDfSU~Ykz67e=LuRP`9NW*0SKl^ZEMWFy5GT}3Z@l=;{=*(c%-ypL&&IZ+^}(D zIvBpu26Ut|bbTLFoB$)7QwDK^2HM&v;uC}HLvjZ$aIvMRBoa)+Gwgds z9L;XfEaD~m#Vo>R)T)HIzVYr zK+l&iF-S|-0vnTIHVT1#1EwXIM8uLXs0^>rLg(8uGS@*9z$q09%-$~Z1xEgKv^x|9 zY97|gvQYu!yYm;XCj+d+GY7vLY!{U%MMUK!^J|c=4G&ng%yyGJ1Abvm@Js-BF-CCG zG71Sl`dlA9mkSklNwarxnWw6%I)yu?FwwuV3>TYDXB2pVD-N1g2(vp%MOd5hYut6%DfVILM=C$pYH}XMg z#0?O4Ju&PH4VKI>E?u{-=#KP79W(gdpklk@A~2i_8}#_3D3Q>@U@jqqNy^`C6&Q{I z$R1J^;gm5K#sx}ba1#VHnWP5+KsqmH&0_!R>7m#_!-LG zHW&!v(5*@=6cEq?J#m(xuAfVP^1oqx?Rhbr*C6JO3@*S_S6zF%IeNf&BEH^bc7!Y&Us_ta57Spi{QY$yD#lTd zJLUmDk-BDtuy}Dd^z}WD3}ZC9bvVJJ!^3e{?5w-8y+=AKv|4&o96%P63jv`eFLOcw zI7h3!e|C9Q@%{Tc*o$}&6+m(NAF*DIpMPoI4LvgHFRdLBWsDloVi~EK8BcY?i*3blJ`>pv<86Zb(V95lxj3$|D&77rZnklAx;KR3I z(zNtP*E&!N0msiviUci3TU39x_P1XxeKh<)DWSstj<@`=Q~kMo=zkcE|3C2UDdC*r Xlh?PuG~J4;z$tCb9U3>(t&jd+Y>naM diff --git a/docs/source/_static/images/tpch_queries_compare.png b/docs/source/_static/images/tpch_queries_compare.png index 927680612fda7033541b09f9b158d2cc2e98e292..a74c1acca1ab88454fbf4c2514d6511aac0de047 100644 GIT binary patch literal 28097 zcmeIb2UL~k)-}3~i6$EL#70EWB!Yl6r70j_iDiSLf>bFgO{oe3A~i8+Y^a+i-G+2g z5Ghg&ibz*!f(SvH(go>d-??5ifxkJPEFggY$RFFHdrZ^L{Fw6rdo$L~1ILb7 z+a0q$eDFIbGkb@_cDB+I@)E1XzB_d6*bxT>NlBYOULav-Zy_o3`S$zxkgtwx*K=U8 z<{hB_O^a5EI?Q6(NU(q0xc7Ky=WFLHewB^N-4^d01Al$^^NYRz>NI@*`0V~Og7bHt zKO-9~EVo_k@xNrw+;x-XHMy`y?D-)XCzCB}7F+8sxI5#=$Qsr(k=*gg-ZqbiR%Cuy_){)F^TXKc z;(>DypZkaBE)xUQIa05Fo4Gtx&ZgGw@ZgP$${AG-5h|V@t)*f6I@%hJH(eNMoacGP zDC5PdA z*YBFJs)*jB;M^T*=Q;MurLRE9YGMrg$~cl8gYCJ&J#DboD(-boYp_+{!|C(HoHFei zvPV9Y@A4HG>&fwSdS#O;W{|eJ>B)s6!9W9!Yj?WPF69wZx!OZF1PW#@ce`ZhR3m7T z?^BYRGto8v`%IPXA#yf-wN{BOrW5z3Wc%_Rc{r`+$fG&s(R*%Wj<(0)A?~v;5DYXOAAbL?K5Oh? z`m1|~eYc!j+!E$oU(gb09Bz~PT5gxh#IedToXNbKIGMj&HJ(-(Kaes;i?QtZ??2`F z2o?PD)dG${h;B|o*sJft(dEj+!Mqh?=Y7AQE9x!+@Qq|KqfWUa-ss~amz`$HPW zS4M@}t6sL~t5jE+_z-%cJx0$iYoM<@Vp66fA=Rj`u&}RcpB=8wwqWr->&w3C3L;Sv z9w#o8I(Ma1o2OK2tKHaoA*udEhqz;1`7V{PrM79`b9=ZGIpgDPD!Vv*xGefW`!{EV zTF(fDNAr%IGamn;Vfe{fsS_XG`UM6GvGL!_Sm*Ua4W6DezW(--fD694Z=%Oj<@E#S zGAz)GbcZ&}M3*W(i=6q;&wMli5iP98a}_)9 zjKAT@d5W{>d)x4Vd24Zy%kMwUlrnvKDK<%IBr|lmlR zi(1B)yY>ENuf&buj$4$j?=P-%Zg8(T92FMpQx+^Mny@cbT~AL>d2+a^Gqb;RiLB+` z`?LpAs*P$a;}Qbap5Qc(v_V3-zrM{KZr2?1TEE44u? zVm2;4cZ=pU)_xHzgYP`$;qNq~)Js-eT+cZfBqEaqpM%}aDtk`P$_}6c= zhKh=c9I*GyBHRbreJQ)6bq+_$i}s|;O62q1-F)jNIGR=NwqF= zhdcx22G8Z^3{TjU-zquSbicu4EOYqHIjteCYs0YQIL^f`7Z(?#du;nG0|bpFGKY3| z=Wr*jvj(bCf_`AJ+(hUAh~<5=jqkGi(Cf7qcx6=|ICUj;XLM+bp2>Ns(BivGnT@~6 z{$LQ{F?!g%_{yHHmpra#h1l5InHP6OpAkM zt?nKTG<4Ea9%=U75vJgHh9`R`2NCUl?G{a({pc+pRl!iJOBkz1lG2SWJe>f!tr> zvj%G`2Q|3|B?lz&fE2aHp7U&OD-G+%iitaopA$FDemaz=EoIEdZMbWEJ`UNFew*ky zfy*@Re5Z`h;BB)izqRvzoqbWA$5>a8lB<(`s;yyD+h?o|J6}G+wOJm%s5H2pGu|TU zIZ}9c`pmhMD|dHk?`UahsnmX^qpfZJ=JA=4&g8OKpNTdVZp>h4kz|frx^^Wu@q%2f zaAMBniAwF&#>d-ZqGFfr;tW*nliV3BQ;=Myyw9d?e5AiWcedo-3u+w7PIlQN=HVWr z@2;DUzZo)N$F<^wxrhMLT@4r==)lU>2zpNTM>rr;iVqIs&2fW^dg{_0Mu&QgaY>Yv zIudij-<3xRu_?qNV+d?p=3-ipQ?-7m&Ov^>_mQTyc0jeUr;^G@!7lqUr6JSd-o`CO zy{~&b*3RaUD5f+TYtr9ZQuOrliew~&TUOP#*~-T0_1G`83I#$DZW^-4{NMc9655JZ478Yw* z64`3ghhEca5Q~!Xm${log*mBqR3-FN8l`OBT;Pky-B#h^BU^1Cjnkz~$(Rx|0h2r) z$_W)xinw?(EWN$v}~fxb5fd%-}}0G9IdKJ zwrB%-u;G*qu^Rw5eBH83in7PLLk>JTWmbFdFmsLvDs?P!fB8y_$6+G(T~tIkVzdHQ z+|Xy;n_roHkCl|(|In*5sYpVr>BK<0KNek?AWRj2$ETF9DZEh1V|>^E@$WETk+!k> zK%RYrTYt%6Y&ab2l{t&J96u_N_2Gq|zklWE5Y9xvR>I{)*EXM?>wM+-)kdz77G>e`y$`1+1C+G+>lm>w zE>ohz-I?0pfn_r|xG_s+p3;Zi>WW?WEc3HS%dl%abwSAOePYV0&aE3!_h4J3g@B4d>+MZHr9*5eb|U$bkMf( zJMJZPbbh&tGMyvajCZ#>GdgaMlsaeoo?dn$y5cC_9ce-_Gv0KuxW1p&?evv^Zt9bW zwL1KE$SX7E@?9xcR}>Gi%{ctkg3nkF3)%E6MDGH7NG=dk-+{Q9jIAGa{rdHa%Pf3c zni!Shrj8@$nT$=|#eqajSGV5zE*&wpIyfuAk+c zEl00Ik`7Rynw2%@F+q(=9pkKuHM(bn>qcr1AyIP5xZCuNjEvsM&};k(kpRo_h-0(A zJiDc_wE79ptEWI)qD&chr)%{Vi<8I{`123--@|1(Y+KnDojL_@sN(!cI0T^xF}koK>+W+M=sAMa|4Ca`p|4#Hk`P{$955D@6co?F4-h z z@ubi={}wHH^gg6&51;jYPmgs}iI;?|-372lJ0gF%XJ*B7ugD|M_o{ZsCzlpdbrP4D ztw2~J+uc1JH6ILUE$qLgjdA`{ggub+r~$ybsqfs7Wg}ujw0nKTK`B1Yl@eed-hz$ zQr+%eIY)Gwx1wMwa-ao5BGs?H0+QLo2bZ~a-len3=Dcspwji#DS6dvR-LNFAbh&|15Cjr=*wmDkfco|DYJGJWp4?;5z1 zL(;Z&X|ee@HX(7hGgi?fnCv~R(hP`E=F%3X)5S-TVsdgQr;REf6nA%CT2aKV0mgH9 z?O5I7HGQsdE#CpNn@6IW)d6eVxMK~wB9z@<+&f%uo%#NWJV1Edr#^cTR3RKTD@3tOgi!W4+)Dbia1c$b}LoRbfdLR+xa z?E)l;P{h`&wI)1<$M*Orv`5$BCPz?`7&l&k3m?gN!n zzuIHxNmjd@p0hMyiM09N*Y$Yk4}#3GaHU1Xg;RlI3Jg;4_W6xuR(ejMAsH{paeZQF)0MiFu z_eHy#_X|eS>aJ`2C045&snmmtAhV~JzL#%0FU=9~_xERWfIzif>v1gNmhIyJM@MC5 zX4cNZn(p87;`)|zOdSN=9e(`H89r12OV%9x@j|#Fb`Fz{vFq)t4V_z16_tHVqdH5g zGaNevl|4~tsGpw8Z{Omtb8EM-*pR{R9UB_e3rd2dg)Z=Fac{g#JK7|se@~eG0y`Qv zW{#b!b+If|UV4|(K*d4sSrkKH;PzRBIZC+>cE&PgdP;tTTVKJ4u2*Z>eUBIHx`~`E z-{Pm4gtdtT)^$P9vKrT?*tE=JxRIbI!Q!*h({u}DMI2c#O)e@?IT3|)sptIKv46b3 ztZgWV>poZM!!v;htLla7DXkDg*3wA_>cX zc(ytDw;4-sqMmUXtg&#w|11y}VtJj*cdj+V7A#tuGg4xJ?{UF*I^-?V5nvOl!INtx zEQTi*PL%;_?+&S!x2V?XqOEf#22pvjOI5hz2h|s7E6j3M?Y`(aZbC@nhL6H3z#O$e z3Bx|ZWI3aiiLp;Y@pK4c6OckSQEI>@cj(C)x`mQRaEH+eCH3qFTbovfrn~YB5si5^ z2AinHzDV2w+pz^OtnlYAW)q;{!}rs<*(2M1uUx%o&f2<5^o?PP&ZWlO(Q3qK1vW=% zs7`wGsjmy3Z(APb(%(w4vJ8pA0YF?rGXTIvk+2ZeT{5)OIyZP!H267^5|dCryL z;4AVD<_JGhCQr`$o2xi8+^N`WKy7JYx(}T-4tKY>%1(r5=(fLc?FZngJJFHgglbaK zm8zP0_B|PGaiQ+kc))YoKR!rzls^?-aLee#Af#+~%K<;eUaVJ#-bdCdb4_ zex+6X-L1rUAg-8GgaD(W8r)H#xvOOBQk%UmzJB&E+FJA{+Br(QqQ9d=w>4_VmDS(B zvQFL!)+3NR#GN#*GWMJh)4lT@R?B$Cyk(&}#_lO|fBvlK(bNqc@}jsx*bB(G^y%(j!{WTQv8Ze#rRC4u9T=<5#h^0&AfYdZF6B!QF4=yVk;2;4i1G*`L28AeV~T` zro@n+)7xX?+Pj9n;FX$!JXe1jd3)ocJvLPiU2zz z#Sk2~fFM9AxoRN26-hi01E|(CLhf+uacwBdnH)&0L*ecMM#5KM(V~Iz1=F8xpi&Ox z&rdF(=1?EJz@0|=5PxIjG15Z76<4~GW8hFC%bfaR=`vHiaurx@PBSeltGMrFUQ|VpzU)kgH+J0T(L)0k|ln7{5 z6LvG=H%vjfXAc+7@=oNVw0R6TFV=b&hrSO#79kAA&Kg4cp*W&tqTtx>2ZASR&yB5> zK6upPY8Cj43hMkLG{0gsHkY6_a1z|ewT-yUd{ZA z;uKw_tg7OVb=RfW-j^{xRDJukecMm|K~v{u!}R3BRdJ@jKl~~E?e%jMWb>9G1)mrl z>myhmwW~~_BVMzky57`xCxKR(!!HD=dQ5qRY!xhU7}p21tJ_iS z?Fy2&ybtx#Q`AUF$Dvs>=(ODUVrm%2H#NsCc*;9dGm| zk_w2$=u5(8Ec$l8;A&v2y?ghXJ~_Vx$5zrY^!nSQeXnJC388kZLw&}Q zSr5vm1X2Kltq(oFUU4+q!gz2p@x0PEb51T|ii!WL1*fa^6rphQE{)zis{CJ^RiBEW zbBam!K;T6vibSRH4$!~}S;D}j#h=dlg))aqEuBc5r=syuW4||C*D(koX~k`$#`wDu za0SUkW)G!by>?h*N=Dn=P?d9H;)Huv;JgzqK@rI~)27W&FYBcpIdDroEEtD*EfA0a z2?Qv;`H}m&s0J#!y0E=Ek<+L$) zWO|ms*h2)OR5{~7pk<>Si4j!^`=Y?N$hSw|paUU5Sj*w*_owNpXo*%3;t(i6>IwmW zD~5Bp6D#A0rGafojuyV9>_x;43MwDKD#VO0tzTiE&7@Z zg4Vm(d!R@VVlNicy+km;1uO^AT4o?#Kyc8a$4#QEQA|-hv3DI8SXeS;; zAkLmRiV+thxA%F(j;6mkE7}Kid%re~N;7cerJB2x{1NOGDHitwZY~s;UU*X6j56T< zFDUkAet4nA<}_VY$!eUMaT;)6DSP5u)v|~|1 z1j!uMpgL;2T_-2@!~~HdV8*v|2+{Ev9X~(^AEMBRX#WQHJ6LsbOH`8S%3TpsKZeGvIhtZmED2ZvlE-A$t)b4 z%0i1CgQxThH1=SsZc+dd%0dx)NjxDqvm;n0rgoULk;Yu_N}ntgYa_jn<~g(mS+s(f z*2b|)-XWT&2z5cAQagh?^u=zjYu(0~{C+tgNaeZX{ryoY6Nk^Vb-YTo!@<=|cnIDy zzZ?`$JWhpd=@)Z8YVf>Q37DeZbpc?Z2hak!%aFKLH&kk+Kh0crC8bJVboOGgrv!+= zVse?&56$&ntvm^8cqV2jNJf4?IZw)>G?Za3f@R}Q)Z=7crh_G0N{qcVVvl@(u{0%i z5F9J0GSjmuPV^XvqvC_`NI&FEMbrQ~DG7%sU+vk(?@zfkpU+{35o$zw6%|#d=JM;? zaF6SLGrWN69@d;^1e%CX*>-Um#7DzEtW?|2vn8HFC<+HaumJm4OiF1eHXkDqQdxao zx-{C!hlo`=ZwwTpXO5slDiH<6!kueh`w{lA2zP0fvKyfKtd*eKtq|K_99d;(z;Vnkab$)sgZZPJcA$2RkXf4f4TPO zX5QRMy*avPN>IolTn4Q%*$S+w#vdiz`RGds6TkHE4`&49P(8W0JR)4lm1>kopUp~R zNkg7yoo5bx8~=&<`-^EpB%Sb4e7yDDS3sD)Zj((akjT}?2D?xUg?!B`BMfpt{HP_Z zoLBq8ZofL|vh}_NzCt>$Y4-{g5EU)31VSS5pLNH|cY?>>CWTZ3eL*?${MYoxJ6Bdl z&Shb_qMf)vh{Y)VgMjQLq|J+;k|afBv|@Lf35T-RJ8VIM<+K4Jf1P^&x88~j+teSJ zGcE|0c_+Zb6CeH%P=Gf*xszHZ*x#}@&0Rz6q0j&5& zcv_EQs%Karq-G#J~B!a4aRm{^wA zjCdSRHb>r}^(K(Cp!#O_5NQi_4tLCfHL$T!{iHn#QBzdyJ5fXij2T#j9^FdB^$$0R zBGmqotP;?+q-<*M39#d|6Ew87wcqTmNCC5okEDc1*TB2h)}Vd6+Xdme!>b7ktw(Ml z5kA&O3mJKGd?W}!GDWK!&jq=(B{19$>=)HCl@meI7NmV_DI=Oe#=!W)Ok=-pH}`v1 z)!m|FD}?2+icg@Cl;1VU@5~M093~%^7f{472aNzIo#*z zDJHsVmHo>Dr_J6zyQXL6uccvR6i!tiP^ZIhd+~u+pWt5Oy}oD<#oHT!j*1aM9o}9$ zEiw2KWKJORi5ZkiLY@?6!IGtYF2b0F0@ShEt08Cr`wJ>pfle_v_IwLr?RNP^_+-|G z64_N#l_N`c$^R=HMu$FPHsi?r0_W;Ob-zeECpHPN?i`!KT>sxQLj6oJf!&AtqvfWr zdLg(EUpfWInT4d|fp-#QgO}fW{@crtC)ZPwm^tB%^^F@s8LYJOlPW1AcF zp!(?i{br^hYlpzCTZGu$XCs0!fE5Z)0XZq`fOND>N|nbtsZL4l#SdiIq)9eK0aEVQwl}CAp{Qr&dVqqQvzA8U+>sN0s?5oiJ9O9FF7#fWHQHjenmOdhTFN56LC)lVV-)O;aGHKjY+_mBNIi==FM`p z?1Cy?M7%I6Tp_xjU}9e1BS%JVZDYh05M^dSs-@()sXZLkLNzWZ6Hh8ki7*9kq0=S* z2bRzL8b7# z-*oKAu~y`sO{@jdmMW}Y$$tUpl}(gRSArDrm>6|`=R(R!Utb@iF0;Hc4&rv?z%=tY z6t78#VL1tDOn_{24)A=olFS9elh%!X#(9KqyWIWyfdvS4s*!=iLnnqBR8UsRgU^)$ z{x9%9KuCf%;3ly?YV)>9_e?G-Fb5%+}*KZp3O5HOiKP ziGx&6qV%=j9AiFcKn))hb|m*JhE%H9v&{^foXvqFshObrodb{lmId)3=?(~%k)akR zh*Xrr^_ea)H<%4nq3D{HTt_P4MdeXp(v)d0Z@RX-ef!k+p}b3Q2R0=KG4WeuX(tAj z(+sg8dCIrcn_{B2#L@<77)9t|Iu^tb@^kBlvIA*vA`yrmP+&!e{hbS7-MLO{MwV`D z&upT%Dt(5)U}9T2ozY;f&+nADS#%1VcTs37ILFV0K4-9wsVj$T>CLcurrN^~xc9Xr z$ZBi{dlINY4hkQTq8soUwZNry5s*Rs?}uq01UOkls${h!fXwqpwxfmxbv#@({&~Z}N3W^j^;atvc8difmWSFGibVP_SG?Pvk_w^M z^@A@>w1y-z-d?x9_k8QfZ}U7Z;qj;aH47>>{<464DFOv3tU|F?=BclpC~-^U+|Q4i zijKzj{jOCJlSGUbyAM*n5+c+#6b}t8D5i>RA9Ve(J&Nc&A!>ntQ0FvLA@4P9I!7Qb zK1sE*kZI=eN}NqGgFM+pRgmL?aD6%|B3}Wj56Eoz$5+$gCqg+`<}ioASD=ACTO0XC zpn$4GcuFH5zoV67wFe>t!v4y5C19Wh1tq!>hD2a-g_P?vOw23X=kE7LWGpfTU+Xxv z-ut%@)`UCg3MM=$`$wfJCuV9!5)t40Wz3V+p8J zfGp$=K!_H^9U)??@$=BTuhs+E_=DDJp?K;sm>l7+FpGY%;pBdD6ylHYXN-vKRTied z=h0E=cDb3X_^bcK^D(=GdaT3(>DwtylMf~nHbFLrT$Gf9&iGCpPf4VbLAqd=hj$bO zh?AMa0>_ulfzzg$3bQlie|aU+q#W!(hb3~x`(!|FwGUIF4xRb-Uw0iAp_<~ngi|I8 zUI3{??gr)_1mRSwf(XJUqO>O|THzou zhZ1F-DD|w=lVKiqCcWGJ;BtdVk|M*Nqc!ASBI|2Y?IK_ZFeP>h8OFYpxzwx2vAn6o z!iPClG4d2%BUYVGA7S8BV^#xf5z9|a7yrbS77ge#pOR-vVGUn20;&88GQ~RZ7!+h; zl_n{sm5)$ppv(e6z{LZsyEcCWxZvAqZ_YACFerV#0xP!p?0~k`59Du7Rng=W3KkQ> zqjdr)*Fx4;>Qu45o3|MEx9@iUoMr3Rkpq@UiG(A|0ih`=OiqmH@OR*GkqKgr$&)EE z&1(ku&IIt{h9mhZlb2EJo2QpKf;exc*Z_(+`K1Y^C0GDo0Ag9@D}p2e(JD3&`d2LG zknX@BP_YY?1S_@oi9;bY?CvLTynJ_BlRds662!%p?Stu*B7sk|Qt^3qVy$ROhzFqHkh!iadZ!i#Zx6 z;+*2aP?1HU@;$TwG0@*7q(=ez5!8d+A2FNppt9zi0_Y*5r4(FbR8Tlh>Z24i;+3`Z z+jw$@8BQ<}T@u;DZxlQYnJ*8f0t~fW0V+0qp*Cm%$5O`NZg{$d$IuThgJ^XlvStC{ zryp9C;zW89#iFejmxWPd!=FUeu)vyg>+E8XY&O7u$Vq!1Dnb; zZ0EEJrfg&JevPe20*Jo?vj;YuyoRt$0w-xMFR1+tKN~AJotz^V9YuUGs?;dFgz_4Z zXVf4^-c1O~2;b)}e8L5eH}E2|rjXz;D$Bevf;a-=ph(|=oQqC2*O(^eS2Eu+g=-n- z{v{?V3}f zo)kxq@!{1n-{Q43zkajg6EE}I-AIO<`)3nYR231XaL+%eSi^i5@8@*lfuJn_Sc)kk zlG=hpDC#IcFYw|JqCw~8CJIyJi=u<4fg%|;BB`?V{X6~uZAhbL)a3L9kHIC1z-L8o zKSjU*)mpQX)GRWplI$Pp133yXTXJFd!Y!aoA$&@CPL6}CC#m_VH;>U3oZ(i8&0>!0 znQKtMb4{n(keVGyVnEeuN?(Q5#o4{s3<3BlH8#*As93{1CsGac_W>~_BW{?1P-XOV zR29u6F`<&xl4Xhy94+HqKl&QhzzwFZucZnCG6UHg_O)L@u`Y{j5u5*Uj~&2*CGG$1 zXYy_Z!BiJ>gL$*emq0mW+@N6a5xmpGv9#&^{WtlJ_~bs+)t?e{LCqr^MI0A`({;!u z)KpfE5Ndg`0q#ISRmLyBn~Tk&Pm&Oewk)6_{bmGHP1TCHG ze_6TvC_B*;d*p5t7${rVzxMq#kyDEg7}%V4_^9+Kk5Mlh(5I;3YPx%``FA3}PH{qO zhB>@bdSp;_?yjT8*{4hGGB4AU0`Y1LFifw$YDAnH+6|P*k!_2BM-hXTf?6NRMq4Ry zodHa}j#sh4$XP|eM~$B#t}>mn{F3Nbd@MI{`XJOHt!m3v(2dd|!o>d#@93{V;@_^F zfYNA+3X<*{W%S1$ifnCOf5ICCuxoAbE0WLp(01z(BoTaqnr0EZwxcHwIxV3sq;IAF zb5bWa?0P#|d5Oo&)6Ve>q8o&yY)*-dN(oX*NpK&BTS}etHJU;}mswEr2<164X`mX3 z$OxW_stH7aT49R-o$hUVPZ&OBjQBYT*i+WSH=~DYl^nGH1C}S5){9jUfCERGM(k_J zg|ny}BX0;9!c!F5h{q&vE?_np(EdAd)>uES3<3wjPh5ryN)UZB<|?Dn7@Kk*SXAeM zF|r2u0aF;4sQvZ9ju(W!`Aj5zz3y9IDqJ&}=`u74rz-TRP03o;|NUBAx%;m`v zi)Q5IeYYrlvRElZjiOZDd(R;O3?sR+$Rk^az9#v%S5H!S5v!Ib8#yW=?bi`F@fl8C z8JT;SC=;-~vDAJHUWFHs?coKg9_1?umMx;DQ-K<&-Eb(BHP7=L-kd#@)k9_Re^qCb zar&tAO^{|jPwd=QBDoKPEg>MOZRl7@h=kgXDdtjB7q(fE`y|OVcpH63Ih_0Fd1YhU z@BS4?j7{mA#Fz!tAa44BrC}Ohr_JQMgi?4V8+?L4HKS7h-zBoINM?P|AZeWhB;ym= z^kr0WQ2Rc56w#pi=<1k%+vEF zUjpR)qfJk!LA(_53h@Sh3LljV4O|&jz93pxQa9Ks>_WU;fC>&|vQV^ltOi3WdQ>%5 zKIQtUCi4m#Ms3hNgUI9$k4Gz+fWhB{68FST0)Y#mU0dYn6bY=|mwD8h_vMF2lfA+n9ak?;1=|7q>LTzv?W4r(A?e}W=)BXo`ce?cX zQ#NkaRpI|=fQfoR7$n*nL?3cjVVkW%26ZNDw518S$dKPo&Wu7=EO5q7IuWFE2s23T z;Tv+_khdEzX~TJCjy2^aSm3Bfq4ElG{UlK_9K$jf0cxLu+O3&e!K`s5x#`moWiinL zw!{~(fe_{buWEyW;W9~K)MVoXuc$x)`9xCf8Yn}M%S`o~rSQMp{+oA!#;!qG!A597F#`L}!?SPyqGDeCbz!%RFqP<}!4ZBW`io=w7y91D z8f6gqp(Des5C_Qw!Wdkt*yeOIUXPb`%dDnp9(3eD%FKb}@V652`|s&>4>=<6(fB}s zqV`Z4i;)cFT!%jqJ-KLKqZWM2CVW#cEU=M`m464=!4#473WJn)N z(%HM0j6~Fk2ahx5A9I{uHUNVUqAff)Whe)51zs~Yqrb-xtT&qjlUjD|dyHvlMN(4# zZ=wjz9k}c?a#I8{C(1e^Csvz zvd{m|7`&fBYh_uI)b10qCzYgjjEbNWEmk?IlW2V7D1|Jh$>-28bB%bxmi@TnXW!}R z!Eg2O?2|tN6B7}lrT-p4qFw*}X8)sz!9;x4|7he9v0fOiVDo3SP_g@>Z_n-LX|#a( zY!>gzPeJNSh8zrtUK#go!v4=$Ufe%p+EjDFKRs$%So}HL@yF`42QBKB`I4H{lL0Jl zqHBs$7PV@2y+;Bwo@m#3ht?zNmIZ;-hVo5#r!IYlm*5@piUB6lkQp+2nM2go>wJ&0 zMv=$h$VPHuqgzRuRAJ=Y{zB2jo4`VX+mXE%`d8jzJyb=r+sQj%YRyS(d1Hd{Qh#k8 zRb{$S$XySL`aX3(CYD9hSF_Z>tnp_4wO{itSacdw1X{CLBLY2{nZ+>sF9}TrV!-51*GusH1Si_Id%CeM(uHetrsh!8VDnLC8iG2Ni8zrkZ z&-Z$B=im0-elnBEs-l?|RH)6QAe3{!K6?yP4y#v>Yy74-A zWasp=c@ctMivp$`_n$5~dfKZWrxnG0Onr+T`9Akj`1e!uFn}hNmJ$viFGy5+!s5;;O#a>_W z0PDohcjx@^60GahpN@V{b+y+b-~S*KKE#u{`qRf2nrnIUES+*SEyT8woU&g}r0ysG z!v7KoS-*EMgC8p7xx#oL)r>{_cdJ|#Hx!HFbk{ORR5kE;*2w&6#+>RW9C*4b0kaC! z8s~0!UfCS7ys%;x28T zTY$DKU)a`VsAU{h9lejQqvo643y4C75ntN+<(*g)8e;=T1A@pEL1hw3rqBwaiU1qB zvtH_fj_22194uvQ&GoKrZCdP+amTz(^i9t-d9m&esnZ9!hBBw~65cup=QVgg=9|>n z9KLz;B-irbTy954-$=lfM_j4Xc|n`IhV>60ZQZSC9GK|OIi+-3W9*=^_okA`_)Rfg#1NNTnYX!2~PzMIcS$~CEDS13} z2saV*Djzp1!16)EKR5*b0(aF5C@zJ zz~k*{*ENn^8DT-ad1yK)2U!7zV86XvpoMJRD0|jys&9T67^IO?FzoOZF7z!+hrtnbP7&_%?@I(t}v;^)QwYy@* z4U1-ZliTMLWU*V4qq?*84mnf?_1?p3dT;WP#KxTsCn0wc548h5xtcJq1K!BdRH}(n zhG!FlA7P+fz-W*=STFyTAGpy4W2;fEjKAHcqKfN=(ALel#`-Vsy{?QD>$H=*Es zY%{LjuiRicI#EmX4Jmrq@YDi8^d&}9#r+r)7x#{jjFvP(i^dp|u_^eD+8$qd>QN$w zhEM|>7Usn6LnR=heg4YR{^xS}vscmxGm_kiomS^yZEkseI#I@6}lXofXN((#^|6N|MbfXKv@>~HQ-8v1U${Eg+Y+s{S;$F!|vujAkK%;R16_L!C3tWf;etR056S7Btade&p@jw;Zlu`sN_I) zxbz~*iOgh5aO+DutNHG1TE!Ce2IaSinfyn?$D+~UNA23)koRvN0ezsgbS*fvO`KeW z7h_JSF_vaH7al9a?Hb2X{&uW=8>m|)k7k`4mu~i=j;*qLCW$>wqgTA4$t3hbh>v@Q z?U;)vs*{wd)~DsL4rJ{t6xlj#X%Bzg`w$q{TrjVXUYr`ARWv$u06TeAXIhibz|uZ7 zH?2vO`>DMh#@vir2O6K5izyl!Ki^XuG;=tPRh`A21*Do2`$Q7seUJ@K=VFQLzmw*s zM!JXZ#N41~2uZmhQuaDpV^4Yucq158Jo@_VSP-7^JRZ(M8fK_aGM>}^rsFL6FTEjW zEP`Ubd+jyuHpAjZ^#6+qh!8Y)Trldn&}uF{w*!!zMCr_2x_Jbrsmf(ga-qpwy!I8H%)#6z{+Jf1vcq0PmE)nEHl;MH#(4k6MtqIW9B$r?04OlnzXOV}e! zV9_}QI`aTk9{IO+MHpc~t0>6U=>w}zws1YV09`}`L?9+eYneb;3Eu?er4dG!o{#+-oH-AO zO`RGeXcz##q8Re9?seuF=^|}+qqmG^_K*vOUWFtm7cKr7$82nDHHxV^-1;0#v=6j2 zzf2E0MuQ5qPB`9i`6+S{|O=ORz5aV%99J%_AJwHHMhz~Y+gPdzcERL=&>pWO|0DGmBnqVpS5p71MU)aHElvHBKGp>nC+iDcAKda=ylKL_D=ih-Ol!X+VWHkYoBC=!08^H*O?!nWuoNFAv+ZxHQk><%Xq5(&ex?H?r+{q=Y{*`gASVzM<;EuRn z%o@1S3!%8mq%bm$J&7e+Ov1w^&L(i7vj7Z5X?&2NYGxWD2>8F*I8vfCa>g4g8%=AN z8)Ol#w_W2~s2C@U#~T6$U7TP-Sp=c!{>${Wn1+{&x>%RzZk))Vg}sP&T6JFaB5Go| zhQT-9)FG;vS&_~z?d+ES_&pv=~b33)ND=+vFT21z>gTXeoJ8GFu8jsanAR@g+ z7(c{Gx%8%H+^P77Y06PPt)OTXA4R5IIPhYXQ!ycQHnk1$M{y0XMv ztl+KSLqDBojuW064}9)fFJD56TgVtj3MLxPPAFM0cRW;Lm-&9{H4HTBMcZ(U#1D_E zH73L)r?poQ0H6$nNEWYfpJrg-<|Piqnk9=l=-%1C5f+^sBvERAA6U|_R(Xt|&gM@u zCep|Iiju)5XeGFYz@9g7} zNJhqFNUJL_EC`W$r0i?hp#Z33?q_7|#U6z1rv!6KIAp=0$3d29P+N>;58L@tkk03= zchca1G{<_=NDp5$^QUYV$uU^d1|@o?DHM3duZ~7(PfsRzVw!6}{#TlEYzA~EreDw1IzMjgX3lLEQYoO383JS+q^WB(LxM(6 zon4AdGuG1&eK(kF9>89b+K&_W#>ppUZ7}7Q=BmPHq0^~jmjZZmk4WhDdAQa0Uo}&w2D2JjHz4F zodfh+`AUx0vGLsX!&`>yVEA+3#LGm~AjUu#5Avk}=Tw=|ks=+Cwk&OP zhPnWaLD8JJ`bZxg6B?vPuY+pTf+zfx=4}AUCT?`M^{g4PLQ4{jEKGS%R4Y~WGX|Dq zNx7Bjp^ONY5uqbUBO%7oZoX=xsxIcY9uT~P76NJwt{lvw(Ov!M0SR1PnG{pNmpy#F zk^DHcl?JkQ90;N3A@s$Fmh1KVfgt)4?lwbq|jF;fE<>GIDonlFJCyeC^fSX1!V?Jg_7} z^Afv%JszT}R@@Rb8@>9IpmAwR2t_5DZb);b2bQF$u2C+Ndwr}No>UoRyjTTtO_*W! zEjbkS^h*>}w@9b2Bw7PS4z+-%!`#q5Af(&g?v50Px?`X(-SE78C?xZ9?zTb&)Rjhf zNwM!BUVr7 z!|MYVsIOxN!(dWhKhR?{O%$ZL4fB!gcGjPyH^Bn9o$L)Xp&3z@rX3k3&CXfe)K6Wv z|Bb_eCJ%@{=40eYG8$ISs;7#WhS%V;+Q+Y(=UsWkV>T8l?*OnqE(jN7F>oEGPl|7X zdwc0P9|py6o8fOGA3TjNVrK9HffGSOy$p`QIl2zIwbd>Z_b_o&=U4em^t_Thm`rUh z5P)6YMFvt1vxbO4h6!G`NpdQq34=IH#)b|bB((ukH)1P-ioMwp!og8WxpUl256u(6W=dx*TK;~H&$+hL%YV~XLl-d9KJy@SZiaSA-?9KXg7Qb=_Xbp zl*avR;^f(4%8c;gZZ1l<;WuwKs%0pl1dH@BYAI?Zthbm*m4$>i4`I_?!#?$ zCS~_TS+rW$Pyl-a zXWu!3rSg|Rd1T7KwQEppy+LpUO1#xTnTb&tJad(Yf8T@%FpK7b#lvLe4R$59{pZ{{ zzv+}UnZSO%l2YPY2;*407NZ(lK~1k#fq|EfFM|X}mj~T7whsB54u_ z4Nj+Wq}0fj>9ay6xrs(3$3dvPcak)qWo?))b-ed?6lh{pj}x$^suiWC6}5g+uAtUr z4l{xcoJ_1zD*Bpf@L+6r4vI9Un-~^cD*9-SffQ^{vH56T{+*_AQpBe-O&giLxIu%4 za*Y5XhthR`jV1pf&9I8iH^%>^@o&_xL}L$;n_6*v#YcM}%E=sKmghUd8`LgG6T)5~ z{#jnc9FtZ|seMWX?WSw+hv}m-pa~x^#I&LsDIp$<+AB~U#poA=3m|zH;xYs9&tQCq zc|X+{l>>9BIoiI!XSmX6e(R)ymCQdkq5Wz(&^D3LM75dXXwYmha?rI5hmjJd>8nA# zBFaNpc^XcI!Om)K%s=ZwCXG^**#^llfLz)UqL?zSrib_8aS;+!S9-D)yXKmEa zPG#Zae(Jd*c264nn%D)lJwSdl9!n8{z!g60exS}D%5}=_qs*kzlwyaO*bR(g#(O1E z|F`UW_TxEO@V&X9kCdE2mig%V3ISk@Ye!Z{W93c1?SMIg1r)B`O zb~e#JL_rgXiN~Xw$T$ezUTwLlCqQkzD9O#hW6@~wB%Dys?cIJLs%!1lh*y&@b`T`M^+0B$Z2E+(OoOqmE^?#d=YZUDWSj-IGA3TR^> zuG~U_SrtU&9h{wl_$LTxI(&e_ZFJkdLq?(5?$or3Ru}3)?@ejC4m;m7FY2KCG&6>t zAdaUqjn`y)>=E$$b)B6hzhPT?Wyfx~lW zhRT5`*w?h47!1)pQd^NLShIWjSQ6;#FLQ^8}{JiWUi0w-VCFR zs5GODeuubNNVAk^BqC@W;|z2s(8vQK;XfV#G|3{9VXUPrbsYAcTt5CCRi@=g5wZQ$ zi$Ysx$onq(re>e4yg#ADE-+xJ3 X&%EcEy6+iX6?=>NkGD7N|M~v_=(cZN literal 27388 zcmeIb2UJz();78mPxPo!PZAALK@tT;5KvH%CPrh~pnxbZ<_nmS7Z`^VJF>b~aP|E&9iH+gluS+3#S^ z+PVMO5nKCXwpItecQ$u$w6eF8l~Ryez3Tfz$BrFwRFsxJ{P6=)_70ZPYyY-A6JIj( z$aVuq7Hi&q`rl^>s_|AVmbDf8ryutmkLYQ14GA=FQJoyVmOAa1yFS0X_~k{eL}$T* z&s+l520JX1d^ks~vP7%WsABDcj2%C!SKerF+4cAfc_Rs<&AaLwd{_p*42@zhwia04_4&-II!WhQhD~sv zXP^7%t1IV%);mADVWrS>qcW}Kx5aTf$-4A&H74#og05}nUHc1HB%cvdaA-S_#}`s^ zE-6#$isa0XyZ`91lFs!qxjWK03QN_|DKX2|q!H z?S>iFf$~kqm)&e|Q1=?X-E2NNKI$Y>5N@BVaI?wHM((+7`mxs)Jc)v+qmLG5z4&Fe z!@H;3EQ=*?mKeJ<25z3KAYyKAuJG#O3Go6?+(7+)KcR}ao#EsC!FeTIgLH?t4+T0h zYE6nA%Tf-ntKh+(TaXV%A-~YA2wIgz&f`i$k zGYbQRoa-*B1v!=}tWa@#xBZgVdw|jXVY;hUY|FA?CS#0kruzK zHjcW?@i#AwTzP?P4*h=p(E=gIqLuM&KGWuI&;M1A*>KxmQYTf$7 zHydSe`+|-oYuKFf4P&dudTyF7T4#MB*wjnmj@w{Te;A(1&Mu1=wAQ}KUK9rBy{!uO`NuR%jib-iqx9FEIW;-+-EznsaAt6zWl?}BA#PIJIkX~20B#vS^ahEYK;O416FjT6-({q+VcC~&k*)FlG)%; zoLX9&dH6YQSNZHB1NFbLHohKBnooD8)Rz+=V%41M$!Y7o-KW9v=yPwDC>m6q9BI95 z)X|lL187yBX6`oJYs?$b-FsNa)eFl}?kDUOVpSQZW8gX2e6TN%KUvf<+~BAgCT|}# zG2HAOI#3d_W|^?}_)$b{?6M$Ta+5U)D*_yFy4)*z45D(P}vpqnQx_xLTXLD zRW|?jC7UC7|`B(m)-^i zwv1IpOwm!k=eT!)*ZbG;q4M@7VRDDV=-axmiU%aR`4fHKmQ6XXf#u=2xgg2Cx(h3B z+Bb`rhvRcysd*c2_hjy!WBGR$>){n-1D~be$=T}vHfOQ!XiiR!vVh6){XFsBP+806 zY{Q#&cN$WwaabB~NYs_}x?heCl&fmu4F-p_6c6wxno}1)zhPBb5}_cQVwhoQ+c2Cv ze5b*w);O@7@7_EjJx6$hvWolA4$h89#dY4}t-=GiIn|WCH?KZD<;r4-EvHO1|K6Gc z!GLLV*MC1}(c10LGY_XvzVFgo8C_gl9N=;N)a}F1GfiDnD3Zuplq5KCFJHdgUpr^9 zoRMv;%V8^eQk@RdFeMi|-Y9?4-sw@MU85o`y={X+M??Tc&%z3R%f954%%rr#rT1nC zN9>euTw5UQ?Jl|8Rt|dkK=U*9yZTy*c7K8EEU)~IX2=QuI%c7=&Cn5?;kikMG~nw zoG)g7yQf~);>uY$!*&lm`1M85=^%q@GM0WQ0ru)ut6Pu$t z`SzTtQl>Y55>a?tlCqjPf@NWZf}{BGMSewk`G&o&8`#*Q8OL5d`9V78&HV$l#}UKC z3q%VMx&vv&TpEsU+a)uJ!@}l>UJb+R&W@p%uJ|NgGy2Q#R)3#|?c&%G=_2Lqh4?!9 z_MCS_+v)0;IAtmQVu3if@@E&zm+9yk>$q~Y#M;7at`=G+rvxEnWcH6BHaYg)8E(f3 zNL7}zu1d%mZ4VEWKev-}@SKWVq@q*N?>}trZx6SZ8VV~;3-iAB%NMSQQk606fOox( z?gsGyBU{YX7qVCh%Mj`hz8~xg#&@Z@_uX-EZgi@>VOuX_e`ipaBF~N}Wd#9KkE4-h zu7lNjj>C;EE7(mqnsn|cI&sWe&&pplI-mExGr6NJK(j2>YcP40{uU6ujdiDMYAC*rQ7rC% zd0M(1iIMBjUEG1FbKJZoN<_fu$d9@P=7HlwH}gs#pI?SO+SPW>d;H*Bg|^?tg_|>M zYR@c`UB20GZmCs#WDwGsS-Hw^iDQ{qA9Lz+4K$uR@K2X&znf&aW8X@90BW$%j7m)v7N{>+B9!*Q_bL_k8Z|V{&4j)3^OQ5ZuiG<-2P3E%$v5uI>9V@u1eW zMe=qLh>t;v)0;O=L2O+^-0beS9fj{+SE>v7D%K#??Af#D)*5jqwGUs=Rh)JQ$;;~L zW#9h6#TT{2%*rAZLgfw{a$+1i%0x9D$e(0x$S1G@s7RuCXsG9Tu;N^&1lNIfW*^1k zr!#{umEE!ur<;|Cl^(XWe|mY#$pEccrd~}wjhe>VOet_pJN;~ld_cKU><4h4REIs_ z0MHh!IGuZBe(!|_T*0wD)H03$A>w6?p-sy8OMyHSjKin5h8;+pn zNlSW0?xE~xXVb{srrQGDStY`I zx-|>uS{C0x*e-U?JbK>QahlOlE254sr99}|BAx@1ttCJjyZ_-5o}Q_v zlXJ7@Fu@C(t63IyI78yKqlKDYoG2CdxGkr@HE=9lM@z1ob@G*gWn|z^xq1-_wUt-4 zor{&S&FRY>BQV-Jx;K!b9Y^=i3?t6sDh(KAQg6ZPhBMN4tP%yuLJ-fv9(NjpyW%z^{Z2J`qk#D zyn7;e?8%P{RlPiTqj;s3ZI927B0GdqGRi_ebbK8ftef@dxqV(_-Z)Qsw9AwqMsMzV z?JdN1M*!)LxNy6oOMXiXZz$X^+TNm380q}Xig0#CvR>+1yN31QcG*@M8%EoT_&;lg z%80Tn5_iXo3)kY`MULtYlcOC}+ytIOfURkwTAg(&Vb_(d1g>d&+&KJP?)l-H#$N9$ zquYUqQ)GGpT^|o)_1<;Y%jS(dlump2?cTOi5?2$H`RUOUtDf84KAi2+7loXuL@ASj zhDGZrv{Erbw?ie6c*k$^gri#hm81x`QR;sdzM3mS=Odaw-a)S-3y(NJ@%OjIc8?zl zm_{tNyR|>-syn@V0RO;qKze!GPg31&-HhbNY#!uDDqXgoTM}6+_k7i|ZT?)rN@Q1Y zVdmcw4ai_Mwl^D{6B81)C+nv4(>;H)LWA4C#)f-+tKzX&*Xp{Ok;xXym~CHzOy7O0 z)&F6nAUB3?ZDo9fc2<#R{v7!d!d71{lo9EO_V(h$T)w=NE#uT#B`z$Fe`94U0K_f? z2kQY>;V5knTW1r5!a|acS`?t7;L!(PEdf;*5LTFJ!l0{dCRhhwTH9@&mBN5k&Zwv z!alb|MSIt-^LT|iQPJ3lQJzEf4<{E0;4b!=XEwQYnGbc>2QtNvcn6h0Nc~Fe{&$s{ zmS@-1JYHxSab7aBZfO@cT?03B$2yO~Qy}1(vP+|rp2qr?UtjKE={lTM6JibF0}(Dw zt`v-zKm)KoI+*GmdBtB*yimZU0lO~15fI2C(y4}S%z>gFfLXnL-h|rXCAg)F-+CZ2 zD!lNS=1{e3n>i4fT31cV!2Q|MZI@3=73Gfg?p5f`d~R3#?OxmSJvW;-$TuF_9p&De zWrVw7bDU}nR^Z*0NZxGV;D0hWou<8bU*<(m0o#>PMARerR$xOLIoF#%j9AV+MsIO} zp3!;Pa%I^9Ar~X64p8~D%}^cr(TftUzrtYEZdp`iRDgwU@N~<>ZcrY1ae^Y_%z$8% z;mY3Rnp-Z?b?HF?^l~3ofdjC`t|GQz08r8lz>2b@Kce+GfK7^^%TT>#2jEH}AWU#s zq*8~b_xs&7Kpi558^*c=e%Sn5+equV@*LLzrs4(OC_$LY*kMqE;#qm9?jZ9RKuSgq zg`y85#JPKlX36b)x%fbPNhmlNJ{R|y69#D)WWFW(XJ`w|tmRhfrUvzvDd zs2}#QH__D%QtG{xKbNKcRwtStczwlXuWM6T&tC9!rO&e$af$RipssY8RCagPV~wX; zeEt6kwyHC6L;Ab?XIq_*kHjq{Y4UN&iQ5A+Pl-s z%c#`AEr4k7dQILzR#n3b_PRwfuOfI~cH_Lu?^2EDJh0cU46~GL% z9oZZz;!4Bi%csp#{$1|yvo`$1VFL107mSv<$IHx>_%d1^mjgPQ=o}8 zSVC!M)1}`{?zl-hmEaKdBTAd!|8*v9Zcf{y#r7pwsAvSh4TS1BX23&P0N$0^PF-A( zLiI@G1NcmS54qSMW@aS0E=;8J9PYU2}w5IP^uYDGI0)pwPWK0<#c3M)#mIok7q!%j~f%N z0EKtioq2ZaXsdX6aZypwaa%kcn*%T&3&g0C89S(@GF!#nO33onl(YrV5h; zOaJvR|B3S-%jK>@;jRI0E)DpIhhtL?S~$rw60sr<`?h%ey=l-({@6&bOWy60+V`fY zm`d9U1JdtJU%YBK<6^GWo+-&et5z;&f4-vUo!YtHheK5@Kp!t$bKobUuz={=L8p|k zS43XG$yEWT>9S~D(u8&2(`~|~xTNVQ5=?usXDOYC%a;#irA`|>{~;jFTX*=`+64<1 z7#z;1CLWXr3egJZ^{O%gBY{Cq8rzJDDBc1OC9w7o4iE8hv&0|A7_>EnohbrSrzrOh^W!R#kA{j98B87nTL7V5P z5Y*X@>WAuTvj_kDiXD^_3u575TD9)pSme9c4OxyFNbQ0V31Tlf719M zC0Oi3Ws1o}qv)m>>O?(vetx*};ywP}^f4@zS)h*Y=(95folri60q|Xxd0t`URINjy z{VML3HmzQ&N$6TDZLj`-=r*jhE3eBQWg4MYju{YLVW6H#C~&4Z!1UNNWi7p_;C$`# zmGfueEU2S2>mqP$;yp1)Bmk4rz?u^6DrtB_zjGbw3wX!Kll^MILFq+{uWLlCUxQ#_ z!n;MB%pLdPs~(fQIhuK8L_z;NZ3cz59eG|8X#i#BRG2U-3WC~(Ku0xBPv&zeaE(Qv zoqxA$%oZKJe1vb0QrhID+rj_$uq=;?oSYboM~G*bt4o{xcHnj=#)iwsM+cJ6;3sTI zr4cX4H9Be{HedXlr2e&wMIqQND&XZzdh_^`BskO~LdudXfNZxe$_2VXva+;!hu7J^Pl7IdE&tQ{y^ z5#9F7Y^ld!g8B6Jif?G=0PwK2&11*Bn^}W-1wv4$Mq7pXAvhDOpn~)T9lz6b$K4V& zh|a9>_$W7V&_HFK$*CqiMq0vCC+Ol5(CQ{}J3=|MqbdB84#b)$DM&TRQl5^A^7mNKo|v!0z+Ql_D7FWi1K=S+H{YS9vyc)8gu_R zM+9ibyDe3dW!h=$Q6$Ql7Z&6{+iPkp&q zZWW$6CCyuz`}gsaU*H(|xV=JL+|f}MS&0orG%HcXQhDS3l&h@juI-m=I^H^t`z|By zWR6_j_f%$-sq)9gYwK`u2J(d`S2qJORp6By9ecXv^^;4RxB3f)B4S9=Zk4+A;urI# zPA0AsiDCLT||vug5Z64(El^}6#R80_)u6B>yls{I4nk0Es8zgcK3shnR__suTV ze+S)nJ2c*Q(KIlqKXC5ei5Xz)3cRP!TM_}nD;{J8VdXfDNW+&^6o6PdE5U>izzbNR zdDVk9?-Z(J%a}d>on(kC961@hD`F7wD!f-NVgTt%Xxxzjr8Eu-yacPg8^sHq0xGSZ-?Zn2 zATq}8y=k{l$(aju5_ECQYFNjn0aTSiF&ikrZ__2M&c9#qVR*QB69Sd8p6}mL+0Evl z6u~Q5ZUq1rc$9eY3oCc$_8Vdg79&H7N0&z^q}LqAP<2w$A|;a)oj_fNY!5ufJu>79vi781)Qz8&IDB z348DcHB=|weJP}Pqkj~^v=Vgl{$yIokd3Y>i;;Z@LxW|LYps(w$ z9~^2J;PKF z>WCDQjvGDRwt@;<8RP~o>;|->s9+IcL(2!};n0&&Yw`ZwYpVYp2wnkutUT&~rQNrs zV&cYF(z@Qp*+X2O$lapHvs~1Ap~Mv4M4;;f{Ls^a9Yyg?q@TIHmC=s>4<01jf#d6Px&<0 zb4+)M4e%TBa%bS>NUV)Ss%eM)8 z9=n>=9>&Y=MI9KmP}XA4h9+Nb5HClJ zm4&&%7B6;pyaDsHL6kJ^LZeJWQfY_FzqQaTw!_mIAd)YkYHybIQbFnQD>W>=UICQ?hzn@re;=Rc5puLUxe8TQoE&v&~Lr(iK8Sn=R1KVcQuj#0CV?jhNRB=0I zmCtO#sx`h!>zu(qSt2iLa0ug&afXT*V*5!YK3O^SNlAQi2$(jW$bScLJRRif0UPHC zE}I8FN8%`x0erG!kHkMBDvGcnk%_d;xalQ+%Kk` z$C=MMB_Z})d)W!#LS-5;q^>Bb0z}{$A%ays;^Tc8*nzj&lg8lG=HqxATd31Sk!(G? zC=h{=Dm9%lRnQ8eYD^f_8t!hi=_3fz`z zatC2CQc3>AjCmShNh-lU6BwdEdv@j2mJ3;cmriO2ElWLY9YJG;FcEnz@qG8RsN?LS zi9FRX*q92*;YGy>kX#{lXf)Y4AQD$OJ3F(GP&-uK;saPE>X`XOhc;eNz(NcS_0MevtA%fnlKpdvf zK_Ql~ok$@V)U4j=;#Ayl0*KE$My z8Yd<@3M$CJQDwJWGb#cJUWtrA!9-sZeHymbii16>I~t-9sb~%~Z|d3zxNZ(DHn1G1 z!r%@g60)j%upw-K`eE|4mmFq9<04ZcHAuisf3(P^t=xOkk;>iHt|;%xQ3a3^!Ah`< zVac3m$*>w#MNasJaEf^lq{@)zf<^~AERn}_bh9*3O`LSk<=?~+P@yu^V*`Lbk<_{} z%!aAUkK_%3kK>_MFClv2+=lmuvQp)db8v!`h}n{B_81`H7SS|GnGgQl?B#=#r~u0K zOyd79c~7z%4V*J_?^Vn)89&}aZpaI_7if}zk2KCsTq(WpD6_m=KwQ{8J?G@ zJ1V)!>!SMp={eAuWD+FLAL&+t@Zk{49)-O_mMFFZ#B7FQfmWMN>@bMBXYyuKaY2GE z;0#<+18wzh8 zUru}u7?7y5@=fa*xq!-QSJ3Ke5sYp7q8-I1in@#$D*JWZl-uaUybbI9mbe|F?ep>& zeJRUHy`{_nrjsos1!@;CzbB^PjwdBz>}uZR`-lE;Xbfh-qM`oDz1C5*e%D!TbG?;L zH$k+oMVRm`CPdGPJr2j4E1naYFlgdWe^wvJZ=h%QExrO| zlqvyvQ;8~?jC)q7dtpnV$F9Otr#BE_)l{8WYm<_pOZF8i(zi3K;-rBP>52Oya;y@2 zn%uj@z;euSBay(#yo?aIw(=O7J}3+}2l;wAb3zi_GcR0vL@Hr7j)W^#gXjywm<(q{PtCY=TBnv{ zC0=7D$o;lb`Q{QLQpx>+<*f($CT>KckNtP@*_5r-cG5<|StZx;pEeRs+&+q) zMIHHm^U5g$6V(oD47BR0OR3;e&hQ+YA(gW^ElbM)!hmG zhbS#ho3!_+I!XLY#c*0WHw4-B%7fR;>*@?bBGM07zLSbJk-{tIHiUUiyccDI0oBXu zXXM)#;rA&wl3N!^H61Du4{q#Y=zd`$t^=j=>7c`d*E1j8*qBC{8XS2gYJ&!tFNiBo ze`cL5%7(H1XHa$*|8!w!vkqWXrJTp%%a;6c5#5td1-`^w2Ex6e5SfQq*Dh<{O5Z_* ztkIEsUtllwYZpK_WEfcFM@NVNwfm<4S_KY&u^FsTqcG-&5haaidX$P#c-lpYY#?QV z&>Q$BTgsLc-wAxE2{9GlMGZntl%OVIXs3Wp#8^M5cE)2!(FU5pIx@dpMJdq@V*~My ziLK1$t6GRxM0t_X)XsUmyP~~~-kn(P$NHM9^??{>{i|u@S}b9bH@=;=m(JI1oyH2n zg42VuK5aRPA;-a8%iPOPY}d35Er&W>#ZkBdV{1_Bn62I@j&@d+2`?tPq69)Ul1s z?TNRdvHnwd zIerQ+LjjV+xh;a_It*N$8PeZ&9F_qv-RoCG9NCok$ElM*-PS0D9bqWjkRmgB-yII7$}m108{2pH^j4 z7A`N19hJ-gAzs-`dgNLwDFFZ0F5HpoGw7JTJS@?08y#nBIEj> z61{1>n11esH)URRsVosyxcmAMnHLyj=|L9O;y^fR@%71d+P^J&4$4LbG8Iz*BXz@XYQ!5yIb)d8#e{ae zg(N{GP~nwu*nzuJWh3KO0_L7sv`&25jJZ2FAo3TJzXS&^y{Ze&i?ygcj2}(0993ti z5C!rYXz>$fSPpEcJ+0w!WR$1Qp}>QZdoy>S0FXF0@81Hmqf^ww#U^?uNG!)5`3r>u zt=y_H?ZUy`2?ADz%$itv!ZPF*?@FD`Qg_k~w>}fPYfJ&pnjlY*2yncAv?IElqj$RE_b@XptFnbs8%J{CfP0v{Dsot6#|*Q%6HTL>qq}U2UheUP)*69NCg>)xXkEzcEUnn)`VPFtC^+O|DLEkTf?SGS^bGLrycg&*={-)Ak zV1ag#SbygKYxN;bSom(bRK?V>tWKc0Vqnt!rtVYk)W?1ia8e1rTep*euB<`-|H`v~ zm4_TlM!gT9AVGkj4z*lm)EG+K?#;HLQjUxqRp*&oP=m5^_6`V8S6Dx(wSx|x8gvFK zo|Qo##`;ectsPj+VDcT(Nur_w*-c8=96#CDtyigfn3 zs~$rSg_Ak0NIba>ew!UQAdUbPL5k%0V{J^?eJkCF&MkF=K%Oll&JVo+7ULst@r66U zhNh6QkeE(&A?9petYJX6e8Xx|iAe9F+acP6v^S!S0}syKF!I6!nu6#KCdz2ZPbCMq zpH^=Za@s@jCf45oA~!iD6?aU@28&R4ETdb7k6#wG+EnIcG3&S;(yS~(!5?c&XQmEr zGe^R@sK7N7uh8?bE<~sxE|E+|SekI=9NZ{QG1p--)qBF`BaSI;VScpp z3!DzQ0}uZB>0t=WpZ zGl)$vEVJ*f4{6efxW!bN0#$a9;|hS>0fhI*S4ADH+*{P*V0-=Hx1*qzD)13N>Gw4^ zW#GvB2x_S+13UB+c~;5ZUGx^-st!Ejs$C2^_NlmrOB1;a3(Bp!w^RTE$VmOj>FbS&p(~a>GxI9qUE-M#9Wh3A=Jh=T>BP^~OTl zMgVtjCCY^XPMj`Gaba|->tLMO!y8588-ZH#V8NyEp)t2v4C<24f?B{pUEUfh))C_C z0aSxu$0d^goGi^yyW-KaMWr+LoR+KCzXcE0ho`pZ#Ygjd=U2n{7%*Gv*1qf2eWvE9 zam&GF{tohk2POJ3DWpKE^8AqkRgMVpIx52r0mSPd{})2SOhfXjtwAFl{;_@#Bs1R_ z8<6mY^|@wjhh$|{P`0F*Ft~t~3I5UgxwSU1Kr56+pkgDa3aH7UW1 zo>?^Vk>V*u^P4JO$dC+=S%3tlroz#_Y-|ILx0DEAg0IxJiBuc4=HR1Mwd;^C|16#I z&Q1{kKOs{*htD|z2Q3D@caLfhq1%PZ2a!U^)lqc3^APj;Bl|p7z?)qT!8Doc5IqGc za{I$Ru!}YkO!V1juvW~8rMm;Tw;UlOy)!8}4on`2P~^`h{DEj=(Ay=AT9S?{T>~-E z5*s$K1Ac_FbdIr?40>ys=ppv41ae2T*L$l!a6FL>q{)adEd%7eK&S6^Dh6{! z{7G1vHds&D@UeZXREuT~YS%2LfQQ8%w@H-;-zbP(g_ONN4s8xkbRLF=yxDKuX*jww# z8wE@>A9>S4UdywLR?VuCqtri5>EF`%bJq^yGV3KNL7oW;5P^T=Afg+3;$c5#SiBB?#@ne+9Uf@E2UUYC-uc z&W`lCyJ}qU^vqA(E&4B{*{Rm1YyIn!cdjcaO=uKkc`P&d-|~J_gBe{n!DeyOwU%0c z*Pb`Eo!`%7NqqBB0_VbW+u3ArIBAW=)}9W0)@NzjM-+zI<08|91cT>uZqNz*@y40| z&eOs?-T&AOhPC2OtQX-ae|D5WZ65u2>3zORcOBByh#2;H@JG=->pzGMptSk(*0x*^Fm7!LC{NLMh4oqWd zl{OO%~%tgKcx>f3#>KE=cg786{5$+5yi?upJz4HDgv`Qf+i67VF61^ zIa$tm{x1uSWnwL2pcTx?$;x!c zEbG&f=3-fJS}{SFfH{u2UT<*|cz0@8BZUU87ve8*w56yOq2ehzs!F+rid^i{@1Jh2>^Zxq=Nk`fOsBBek^4BrdD35y1IclHw&tGMfryWZ&e zqHh}O+Se!?-1egxjf7A#a*0Oey}Epw@n#db2z4eL;}YV$`IA%+2(4i1HGLUScmmYz zNq`1J5X9y?eA-AVQV1f1WI!IeI;go|Pw*}#_yoZl)Om*(Z-g7hLUwnvm@sCzx2e+rsiSR8&kk~xSHNAg3;Q7`3apUE{#$rYbX z2HzL*=~ViNNvedK7Qf)X9sZ1!`<4zfw<^g|HuJ%pdRD@=@zyzaXB98O@8Tb(i!G4L zq5{t+Ed5hZq5n(g_yfP_p%+~vg+Jg=?E$Ts_w~v2rpycYorLMnfBLLA`2!LtwO05$ zE923}DDd%IOPb>vy2mUt65DSr%m_ayYQFd%tc%%%ZRq0vX1WtY=4)xzD+#jpJv{L7 zi#;?Y&t15sxr|bD{j_ z6j}7w{3l3`8ksszepr4vr5HB>Wt&G1F|;&6f%k><6K($-5?QQVaBf8C44ivOgEE*n zlKaDW{tVW}9_okRs1h&o1iNc|UT& zW73`A-nIbQ=8AbYG(>`o{4@jLgG32x?kHmdhdj^pvyco%(kn#trKl)8KYT8RPNEaF z!h;?vhEQnY%0&jIm;xg9_Jea&+#nb0je4L$Bkox(PM6e50kg zO@pUK+z-t?Jr-ATsm@!VbpKI__kwC~U4B^O$JdGO(2C~gg@xqv-^b-BELdq^G zU$QzHcORs1Oa^7Ag;b9FEO<=zc{G^Zsam6dQvZ)I2E9%qn(9)jHc9$Gss8oyQ;cgiznA+DU1O(e@+pwF#lZuy5#8AOFAFFzq8?3g;G)#eSa7{bLR#9i zrFmZ|egFG=bTP3z3sk%h%pLUo@ckby#%%!V+THng?ynHJ7iYy95d|73;m1U_Nn5-9w%{pe2c z0wPuZ5_$d_U+xR#{xFI5ggDbYFDec|u=n>DO9v}f$k0>_lH|=`2;D(eR`_q23LO0v ziB}{8NXsM(6aFY2Y%Iw;q^YS3B|q{S$rp|TBS)6j#4D=;2v5>vqb2CT6LaF4%b~pz zrRN5coeeW>L#e?U9Yd>}xYlFM<@bWm5e|s;*Oeqnk4#3`M=I2A3idV_a~R^Pt_KxD z2XcJy)wURPeUlGj`j_AR4n`-r>UyrWQ&V|+S!6h%aXi=~B7De?jyzlASr?m7Cdf<~ zgAqgx&9!)98DvAn@=={!yScP2sS}+Gew!v}px-p;8%^ak5Olu8~NZt7eo zh)pI5fQ4|J;XA$gQV-KSJOLZh(6Seg#|O`B8EHPQSRI;AJ!N(759fK087zlYpvR@|7jRS3`yKSN#2Q)_A0T{% z0+Sf)AVsya)O$se9ZjLg8F_h{87P6`lbLu!`-N;xAXZC=io;|N8R`{5yH!78VxZzP zncEoHKy%{gea8B0FQaBn@PiV<-k1-eCS(;#CMLQdQGsWcs3@l9zPF}}k*T)oX;Kdu z-W7n2H24a8jjEQ!D>}spI0MwyOfz&IofD2mWv9T@TzGBT+meHTEx{Q|Q)`g}>}fnO zNkt^;K)STR6UO@A+*9qUx)=^og#D-nybqEQh%txpEpP5_uZ+%;{pdU*8iyuX)a*~| z9LrXqXh}sf&6Xf9837@(X2K7Z;2$7m?6hGX)p?|?60w8VqjpJaJ9VEy-csC=eXO{D zuo2955aOfa_6*x3B|kVJvWwgwZ4ofuvu$Rcp;P}k>>#WX{Zv1+TOQ=Layl&2=X4g_ zY&DhlL**;uYQ-{1>sI!w=Z=6o<9kwB&5Dr>zgtjGx__J_yj@!fad~j-e?T0?f zj9`H0CP?gcg^uN1>ZYQwNmLx;Um<@cHFVvX2%Dz6{fm9EtMNkb(+rGF2raYGIqD0G z$gOMGAK#L!Al?d%Mr*tFanz@3oPp#$E^|%Y@N#CcXF;YgX`DVWt60$wZ)rYfKA9J@ z=e%El0|Ll-@fv&|3nnppNT5dl;Mx0l*M@01>itfR>tSASUK1;B+((vHjIs$G^XUSsBXZeA>A8NSM>Ju+>SI~4i zUlCt&7wgLMQ=g4wza-z_CAbuPp`ll4yyEX#?06icIUbkbde5(u!Vkmpv50&0(O^3Z z-BeN{D`}urM%0A*!;pNvJ1$k32ilquI%ks&wsZNLud5z4`$F=JIRZCvwZ;oku}6Xk zLk72iT#^OB0vbo&>5%LbT!OZMO$ZmWNX}NKnqd=2#TLBk#1jiw@rP|lOq$J}h52J1 zyb!pGFKX%P>W;k0pVxI@)|QDVN3;j|q7A;~uDlkvuI>OVi6#=@l`rKQ|VnNz}WYwp%L49?K8rjFN%`t_C z=iWSKqM^XMz=iQm6vwA}RGDG~kznvlh0~O>XpesO$qK1@y;5o}%A&zM*yA+gh&}}t z)HpTs@7sfV`tm8W;DuKwSErf=d*eNo(m5f23kVx(@N4oID1FyeoBC^F^7|wOQp4Fv z<9mL>QqgP{Wmv&Q3&U6O$Lc?EsP>a2{=uP|DYsO*J$xj}NC+!S6CPkE*b{TXG^Nfl z1Z@Kp6)<5V25({uo&_%vhC^s|erDiN0A~_nW7PGy<@w}kAny-xT}1mL1jm0{A!X+} zpow_|*h99&#L=`Nct?x;m3sB#L0kYIis`E?|T}wYl+RDg=tZ) z`$17jkU;4a(x0s3GMCuLOPFTii-5QZ&cj)-y%^1`kTP&;e}kC?Y1g0}EkIn^gP$L? z^L58>leBjPF@1pvG`;)iCfM}~IcGlxhpwXb?v=YblMv@juWHf!g%;{TZ$U&5kAUZd zu5uqThI+_2-=HS1l7>X>M`!a&4@lciAr7yT0=yw5`NC?-{XOOhj3I;zXVCzwlV3~s zcD2*Ys!i~c&6**2WMd05up|lP@c#%PDxud*Pg<6LGxkn&BRY3{kxf=P_h8x6j&vr* z*KF+;KQ$3$bZS9w##^U{Xr$Tm3dCLuwarV4m>8|;SPe>piH+=8Tv6HS5q+XNz zp!>w5P<)s;>|ZPwQ)8nFp}~7O>5d$ zgKgY|o!kP=UmWHBZ6=B%9hq*|UXOgxSuHrrSJXR?2!aI|nUYmkhcOKEuR<{~ESzXQ zHKAg;QukKvqv2ArzFtBN@CL@Rt8z>8<+U4ENq=TtP4hg-4vpIBs&Wl-8(}oj9T~Pk z=5tA9M?+Q!vN<%{Y^j4@v7Kf6(NAASQXP)$dSto7l;Mg3HDiaI9KQp}P$yl~>6Dd`z!;OjBWis2GnlvR zu4F-;DUwoY&@FIZ$!<**oi2W!_#?qy+%7T8qR+2n>qq?AHgRD+zuHe9B~Fbp)rAwRTy}G+ zXmf;u2gZjGn4+E&L8R}DpvAdXAaJ-^x#O2x05OWu!omTT)!t@kK|mO^VC3P<`sLJI zK(;JA*ww1*NxYLQq8=j}Q^;LwT_tvQ(YhjV^}*ytevJ*C*jil&GBA_^C?(RwD|?FR z?$K&tP9szta?qPVR;!cR)eR^C&J&iUw5~1``)mW00h+&53J*pX&G4iKYzN2yn5i?W zAq8)pc0qC`47kcP@Pt?|YB-@08i;lrczn{Uo?2tR>C!*cUO^Ij*_o9iabqwJMt(MJ zdJLw_WMN-bAcxSS5}QwwK4udCfNnH*Y8#?7AM39piIL0tWauCnyWdlZjcP~V^7)~v z-=t5+Eokw)jj=N9w2B-{8a{@7q=6|Z*R;18EQf@zK_eNA@LlANChEcxGzL|!6X(fpZ0^0K8iUD5lkPa#>>51_@I z&LGtnl($8<&ySu*ruL|NhK7pJ< zZUTx+CdXTRyK$7A7G`L#X)6CyKgH0$`YAMb4M-Kr^Hj1FiEqUU(lm`M;LEOeEA*|q z29V^m#pZXmdEWlcaXp4{f9u|?!66*HZ0i{@=eCDcH61O*u2n;0&TIOy3ssD>kFBhC zcfPJVgEl$vd^Dj-{jc(L*R;RkMe$?Znx%$1_S1wPq~d|PgXhfgPVh0$Qwy9DPraQm zBtHf48^%Fj?Z+Rh1OYVZ7+g078)S_mOwF>G?zi$oO|ol+s_19UBAFL{4%~=2Niv6? zT#RriT1f-}AqtF;yHXV1PJLKt+O&Yc#?~I);tewTj{eR!PLBFu4swxw2*{OCQQ$jU zz=8x)csdSR@;-p6B6D5LOOB1o#h)4t2T_hBZ0zW#UvGv0G2?6A1H20Wx&(4H@ z0nKoin{0z|fe=u~Wk^NT!-V<(JKx0tnbQI=TLaDA>JgK(t4EbQX&N@2N3sfcqkLqh zVo`?)?AWohCrwcmw4L6>41uEos+VKbhnz-{*3xn8+Ar+ZV1u}))*Uklku*Ybpv#n> z{or+;R%H1-nhJ{q7K1?nuDGm21$%`#hC@o!pEW)G)~n{fY<)@JRPqDUVGGh{&(#;6 z`1uMD4e+<5)Yx0=o>A~H$Sg-WO>>mVI7TB*FhFT3b@Mlsw_wl|NUek?gzXnrg+NfFZ? zI!GX)D2;gQ;v2G=bLP7>OEDjVhF!~GC?VOespm4%L&)SzUo9C;5b)YCDXfg_k2HLZ zX4cXCCJaB|{=;!Q2m4IdD;GvE{n2wiG!q0WwoYHi8}s^H>aBu7yo`X7>%{vfbMC5L z`gN>e+Lm{)S>Ap_6K|hjBGM+vv9qYNVn19G+?=CDa45}Uf9nojgBz2@f38>BQXD=* z^7UKBPwEz%8kP?^-DITZX08r}8KwlofIR^^er`^cRhwG6_^T#T-#gPUlRH+!*+s}` z7G8-M6?oThWAn*vejhU+)P!!I>YR!<_mQxS55ai7AL}f|j3k|%^Ebpk>de`MEjOD= zMB^v1BiZ9r&Cy^9U#!D?5=`{rQJEcVUkC$VscGv?ioIv@YRns7U_BQGmc1`S(0YbN9_hD8Rd>UXuKn0B z2nhOhAPi=sHd@6z4fJgRa?afc!D6yv^z<4(QMORi+6*oINd@?cu7kf)noms{hY))9 zV@pd>$nZrS&i%(|-Z6p@WK-FM3S$&CHrnAOR~g;9R+c81AY1fGv|=SbL1Mg58eoU1%ly`{448*wj=+N;KwCph zT3VX^N&{awZJQ;k1gYU(z<;6z-JkDD07Nvx5Mm?j$Z4M zKUmdwy6?sdPbv5OhehmSbTHf8Pt2F`uH%*=K-8o6D*YnxCYb^ub@a&j3{u;BrzCkE7_R_VYn$%3jv zo6lHxi+&nHam4IQV6{VNIJklsK(ax6ZRr)^SIP^5^kaVRUW#)j^Y<@F6# zf;7IkgnUQT`iO0?o~FDJ4N4u<)KW;vpSm2FhI!h)IvZd-%_N44H-L#Vs)YUExCqC< zEOV66rO20iT_^v!%>Oa@QNZF!BcBygS#I`L(wx8%`Jrj>4ZIE1bI^var9@>ujnhM9 z*PuCH=pAIx6ePx{-u7VHbD1ITDsLf@w9}+U8W~2)h=LxC1jC>&aKSdSeDBwq*wA2? z<~(lT$&s8%`4)}Y93;QErovc;F&`=rREwV_F8ckgfKo7}tOu>Qo{l@Uu*hW8snE$pI-E)rYE+>L%j)P2E#y`V^Jf z^w21(G2{&8K332IvHCTfZ=4vExzs=$O?Bbh|9+HVA==gGZ+?hD3pX>k)H1XK+dyI( z?Z7Ho)0@9|TJc7_-D%J;9E&u*P{*ITfeHTc24Er}6YGD6mmKI=CNc=d0LDYz~PF^ml{M5Od}=0FTjdc+oN( zI9&@8G7?r$CZiF7dd5yVzkRt-+gx}U8MTeNdEko-@Yofm8Ks=nKaLd-MDNa?4b3pV zIN;U9!A43$C62~{y<0`be`@bQhKlv?s!zA1iJmk#6})j7vIL2N9<7Xrt!kN`iL3b^ zZWsjYy)@H;M!41k5D=d<4r)P@LsVfFn$D=(iSgSK@h>hfj@`4F{F=;&D=I3;HGua{ zcJu+12YZw>vHpnv|M!y=>D7?&i0CciJEPDCNalL=Mn8!DAC_htCgnxa?=mA2(fMpa z6L@Kw5V7#oya`yx5urqq?c5W_AIswx|KKOc!Q{FE5>t3iG@OAL1hNheHGA`EmM z=pqibdgzW9PVG9RS}+5^@#IuKyW);2ptFzwM|>L<8Z^wALIOD}mmOr^IupY|LTX~3(M(ze%SeMhBkCp>@6BUUEQ?r Gm;VL$ZQ!r~ diff --git a/docs/source/_static/images/tpch_queries_speedup.png b/docs/source/_static/images/tpch_queries_speedup.png index fb417ff1db0f08ac92f69b1f084a15fde9147c97..69c29ac9d26c2b77c75c21ff243df843ef72079b 100644 GIT binary patch literal 37231 zcmeFZXIPcl)-AY8i=xT^Rb~;vfFdAZAZG*01{FnuWEB+2IY%R9KwtwR83jZ@K*<>u z6;YB%&XRMGu*ux9sB_Nu-R`IR{3<=#~fqKb^rXi)9cr;tf5dS z>#1@ll_-?eeH6;7OTYhy-;A@B?!*6y+Q?qAxoCdf#{P=cHOkp5HWnu4HYSEwciCOD zvNkj~JIW`@cj(}*8#Xo;)?)nprvEvB&)mv@U*K1{44mW-3psUb3WfO!`R|vP(y@jV zicd53hH8mZ%*GZ<-O{k{5F=NJ=yN=&4sLwKbVwfn~Ns*c3QHPFJ_O14(51=*4j#a zy|V4zZd{*2xy;3$uyN%l$|gJ`{?xBx5L@{{NaiLJ`N3oT`N-Aehs&$~eF)|M9PsT3n`I5YNx>uF6((2;8oaNN?u zgraw`T_B5~(M#nxCHp+H;>SDXLxd7DOq+UczM{0hJBESIiZcBE!m#w|Vb$1QqOp3u3`sRD3l?;mAI`L0T5ufPkXv$Z{f^0>U-@|s9z0V}P+%p(nwh`!kiX#k zc*n;8K5a3{g|VW!nNDxN{E;^IWtEAZvdyfluin0u5j3ekKR-QeUCf(x;KK7e12qY; zH&-*}NN%A>$Q`Ksg<^m5=Hu#te0uA}uTOY4Z`#ylqM3T#>o;aLlY~{SKi=P+TbQWm zwh&Q%dHz|Gj`)@LzuFGfU)ZPcq`H{h{neKMp1}Y$oo_EML}?}EI*dOLcj)*dQ7c=? zykPbP&he@&L^#1|YCz1o=aIPiY1(9!0>5Evo?~OKeR7lDFK=((TD6lx5z)@JlKtb4 zKU8BaPai&fczIS($h`Gc;nJJ}zOH!lQFDSqAm4)r52_;Em*Qf>od%xNX&1Pto99nG zA&=9LYB2b5t8TTLu4{~tY2%H)%4cHsBcT(GmYtva@s)447}idH+F|}uEm=n+MgRWe zo#HQ;rDom^)0Gocd9Rg+s$YyfXX|{B;&2{I&ckzYezb6)^4V>x-f~`dclW#Z?#1K! zIZjjY6Tz|r|^-JF)`ky0xeEysosNrd#zqtJFmQh=QD{Y!g;Zt5L zbXVz;hCIjAf$F#@Oss3APcH}>*H-7)4ux}%x>8A3Cwd>d0E|s&3Sjf=2 zEJUgkHHBTV$hbAqURK2@#J+xgC{ZJwyJWDf%s`uVRZp)b*WUbEMY#6%Bl_oYDcgZ+ zu0Ug}uFuaV`@gqMyKxxK{_G}`TgPqoTHth&cJ2U9!?(OMNYXijTlr<=n$6rIO%FEn zya*OF9>^Unu%4a0&Bz>o?DlVb5kVi?a=U6x>$5DQ;+5l`AGd%lA2gJKPM)>-`~io z93#K^>~MagT@$TSPX=sd5iJRc|;QRK;7=kd$x>O4nlvi4)og;kNYZ&VX+h%mE;iK$l8 zJU`~;%L6+_txiUr_I|>FsWJY9p5;1kC%HKJ-Kop>w1EESKWFFXjVr<>OSV+wjDbZz z&PPbO+WhRIdTl>?D&43?NnbxYb(?yINf4P}7>V^P0&-GPQo$0ADd8?Zskkrm_)Bk| zbM$|>ztL&vO>roL^7K%{6joHi2~Va6j~}bz?E8WwACX!4?1bm!7oOw;_9M-6bA!oP zUMb6y_Rg7f>je7Hcrm-$<;#(+4xQ^~=jM#w-}xgRt4=jZD^5OuJI-bHCx*_aV17hA z-MFs2yB8}f4y%l3&z_F()`ur?v4Dt(2p#8P!H=IleW|G6Z;%M&)5bK^h`U}ET>hg< z+6E3_ggM5*$T&Eh*;YH!k~`I45YgZM?shpp14ru9il>L8oaiHoSP@1I zDc4v;EF!T)rdpjxv-4Vmjv6S(p4&-6Lxt4*D@;3IlgAr5&s8h%=SK^-xm+x|w?4Pq znJ?%>gF*oJK3!d1gQlYu5>c2mn%`p;>-xS$1@LMLQI+tT!s=cjEFU!}ujWb4+k>#? zwL|zE91^P%!fT;C#^4=MlSSz0%)+6hqEL3zNW0unZV@-?x+5I&) z&GYWvlH%giA;RXhsTEQQckcXZ+K?j4pEq`#R`(Jk^yHxW^R-*~ND$1i>8Fvw+bM2$ zM$&2We0iv-plPE3uWo@v18aGhcr2nvyyHX^9I z>=Xi!*UXGV(V=yESflqiqL73?Ar6MsWf;|bu@GTB6((k@rBi6t{pHlnRScoUc;rJ> z(Nw*4>{89$If@Wr@Y?uW7-6h+^ z?OshtI?nv?^!oen-+Slvw%z+{<3_zZ_wKPMN>2HwCSuN*YtttE z4<9~kyLV-q-ls>0IkuA%O#~y+Y-g67^x0iHX-aA%*@TP7*vGYW?*)*ynFZV(^~#Ki;7X% ztF3p}Y_`g1Ip@tL`3M2LvoYN`Gw=G(u9ABkdk-Eg?WX^1v2PQ^t++4er}Ab$MIaVJ zV12}+mL$>?ua;~kcjS{ddqmgVDuy-NkR*=c)qi@t(^!O+Ne@ZaK5uK$p+|X+6D((g z1*la^n8e#@j~3_YoT8j1KKcu@y%FUZh<+^nRWV0dgw2LbaziSdI-HT!cmvx;5#P5P zRz)8!kC2WqY1PiP6EvGr%tA>fss-uSNbZQ; zaN{++OP8^CjQ2S7mKJ8qjrnv7Og8bT9xO4~KR!Ben_0?Y<%L}P;(z$~@#9))4{6Mm*SN6ZNKLm9UBiQDCRQr8aKY58Z2&~v|+;r5<0`ATpMRU z2OW)NbDR00IsLeWicHR7J@YfH=SZe$lPX}N6`QbWqjI)Ych&p5YxhxoeSPP8BixZ- zjl)E(KLvFnQ7WfiKZ^_{543b--3jZS(hG}oQ+V>PiziQ>JiyHjfN4t8FDS%Gl)k?> z8)mtB)v9Vt+WDL|=j@2C55>jC$oeGLf9=pE2MyLHraMg;_4-U~;ng_6$9K-Ux7=1I zQ@>&xU!Z6N5 zdFPI_)paH1jt9#=+`^?8E@4y|dGo6vf5FVv6w9g?XA=-=#Jv|67OJzY4Z$1Fk0yH*ygf@kwyS5#TVEFEgXCcH(B4s5oCz~-7adZ(v`o76}?B5@hB z?Mrd-l=F`eJ#khh$;rwS<+BCk+Z_A0SGh*vbRR!{>|dCj2o|w?zW-e4(;-_<#c+T9 z0(XRj^Rzv2H@-cm+eDOPe_!P2#Y=f`|2|d7jT8X58RXOi{Sunfq#cT%-;C-!OW>!Oh2V@-PsB2x& z(FeH0^ssW|&oFOGT3VbpMs}k1R_`C-vqV<7phEaJz9=TkkR^Y(l3#;HA7oHztgNPtlLc3hCw0q4dlU12=I@#kzQvgfy2~E8q}CjkFuS z*OALPBV3deo*c}+rc**4y6HZePgh0&)lV)#GPYKDX%OK${p91qUT+C2bK&QjTi0XO z(QVkd;xMc%QZ8BNBBT-lUHd)R-OeYdCi)8{&K6F7pbzP^KFsZx42%YXmOXXK2;(w` zkVo)|F=DJyQ^v4l@&{Lxb1KO?+Ba|BWI3Y$_nNJIuWb6OkZH~iAUDvyZO#1p2E!=k zzLZz6G-qr((mb0~Xz@8%aL{%0JFek#5jD=fJZgNvzk^*Rerx{t!!iS@K?M-y!bo%W zKB{_(zHeb+A!?i_U4k2$HYD4|exOTRCP`Bq#MjtR#(ahvQh5Nnn=(vuhiw_wY^nq% z@k`@mn&mwoDf6J~VwS~~?f0Gm`5BDXCN>+7KJoVUHmZFUM*u9Qc=g|Z?VSYRBfuB0 z_Q^!f2ezLl?_Ie8>{Y%h#p1_%S~PM?U_`Du0(Jn`#r?R|wrQ~vmt}zY{Y8hgMtHL6 z5?J&lBX?cL+TZEZpBH>zZ04qWcz9?oh0V{*WIeiviGG)qQli#9q*x)?4%Ml)&Zt@2 z!}%Y*y|Ku1BtXl{$yI*kFMRdK#{Dg3NQJ@|U{*w z{_%ePHQT}38X&YZyJ3yqY=8s9jt}>_FTIYG_dnpfXI5gO^r*bwep0Q6M#q*PT4#`) z#(^cXyTqqcQ`C9dgqHE)d3(=~G+o!J06wj(b_rCX8KnNjvX?ue=78V5ZgpRF)y{J` zTr#-dOg3QuSr$RVs*@NWYQ}*a%dhbOXaj&4!T4e-@BaP!8-BE7k=EkZTKCrRUwYmC zL^HOf-;L&nZ{GFv90X{y-@fG9@!^lNAwp-7+ilfbava7xxV3ZA$l}8|Ak=H|F^uHr zZKV%NF4+OX~^_Mes1jbf#J<9z5tbZ*_`Ro1{H>#CsV`%G}cY=w>#yBmu*!<|%`g zog@riE1}6^g$F6jW1+vm@B$Q8G}NXUUR*?7z$ZG=ny-n;b}{vOS*Yn>fBiK_Z*!l* z%ok|_fO}5$+rKZQ2 zP;&ujsRmhWGrx`&XrS+})Kk-tbx{^1n70)KOS$Hdnnz8?Q5Ng>8$x?JCP!J&QO**H zk$Dv?$@M#g&R}wV2X#{o^al__ou`l10c?@vm_OZUOe;O^*i#xb(2|=;3QrUi@mL@w z1tUJYPox>uaN|xD$#O68;}kS&7U?bvsk`<=yoB|-X1b9I$aS|yMaTx!&b*h9j;-zU*Mk8un0PYyn=NiBK@(Xn&25W>&Eu*@m#GbMcQWngyo=u{Dmc@QTy>KfdGh20? zt~)Qa{Fa#Ij%&1ZL@l^P*N=l?llO19?e6Z*zF^>Ub5NslIO~2$sxja~sq;Gt%=H zW43i~Eton2T{m+nokTIHjG7xjqc0^xR!b`s&l03xVMiG=V$(j#M!AfHTzR7@BhgpV z*{-Oqt?d9p+qZOMwY@?@wT@-2`OaP#Wl~|B0v>z>gv_#b%RvipVZ14Lg6^vDEZgs8 zP2#904>Sx{U~=jyCu-CnJl*!)ed4#MOK+~8s!P)LYRR=1PtFE;_zg&RuZT#!k4+PA zo!jD+S`=RLDWtXa($dmQa zg|~lO&$10O^HhdO!y}aM+a7M=l~1|mf90v>neGz5;xRg1c!^%Pyxo0iwwE_kJ-2C| zT8_8y2xZ@16mqA)agBf2sJP9z_7!!y5l0ngx76Y0{PKIuS|4gIGPQg`2KK^~*~`K4 zVQgaJC~&1Yh?fQtyP>BO!5`WR19;R}P@5h(Vo+`Z9QN(T_ZME4J*C?N>*m3Oc@?uc zm&$>@@dZ5U^b&48w1r>CZ$mHt$;mIzvu(;@za_#H>+5E$g#NBJnr_|3bIA@OwZfv4C zH_zlgy@9lI>f3W!G7wS7@t=?pv@@&}Y-Yx;5C;1c*c9KxdZ#+Uh6>@$L~zs%H)U3U zaSivCa&_A4JU#RY6VwO`tk_5Ym1btSqjpDI4Kk^ra*|d9f$B2AmTm;s;g6l;(3oHPKHSd()@$umx{>c@( zk0LQcICzSMCP_3X8K%GzaZ5?*?34GeLf~=K?Tf!&2Xb&{%lVON`X2+gwUo8xt?ih6xo(+ZCS&LE-rbP#<8KI z)<#e!#4uS=`0{{rLW+AIMPHGs}2;dR_g@px)04XAQ?YjHzwwW(p990Qil2#)Ep(c5UUr;^| z)FmhXvidg>=WmD|2`EOzA6l1$Zmf+_*lOBVpiN-i22LR%)i-b6ly`!f3X^nJ1!Esj z^uwY!kvzBqW!t@Hgq}h``8j^&HothJOs357zlzb6dz#u;eRobXk}g*Vpr66;%SQVfQ_Z+`B{NPAQj5ptRD( z*8jb#Wdq)B4UjKM_kq*r!o)-)P%vwd(i^Wa-F{SOGoRK>!PTk3I>mfvx*Z!TEH3w@ zg~men!Xwwd*tiRmleXWq|M8vyN`%tZ`rjFHJaKbp5o;@fz(;0yF%tqfs90kcz>WtT zH8^Kl|N32bFqlF)8p2!9knzm?{MWWNX{s|`=caAj9HcYJS$Azl01V)GQPBAN#`O{( z4y_}nMHToRp z9TDd`cqab*F_y#c2TaKCcXgkSEUKLzwU-PRRA@|*Z7k3Z7Agx4C7-?BS)~z1u@KkJ zl~OM6oMD_AKbO0k;n-y^ZLx&ZRru7yUzSbL}~GU#|Qu zZZh0OHGE4cITUTxU7GP$rvJNQjG*zWvi0O7N*g5q$p|Vh&mTj4Rg#nY8@yUHz*Mf5 zb@!LeB=I5csQ}W~L{m40iCDhx4hBvp*m2G8zn{nPI_`_M2v22xnH?8Y;^oqfs5S9H z6P4ytE3YAr0ar+|>Xs*G+RUw7iK1tc2#pdFTHl6IOPJv-1Pejl);5iqFOBgQ{`ocXJ*1yfSe- z80&89?1(;v490=W(flFGQma0B?by6^Yph1Pk!Ug()HXr_9<2PY%tL+f_uuCc(9@uG zt!OQws87JVi+SUU`>2+dmJc32wEmhO6W#rFC#ooc+$;=odsU)ucK+?TpKdSUD1 zrtSAa9fged2ZnCkFe%ea^1%^qCfCYpL(3vT40}? z&At-~ID(p9 zq24@2>Mdx#2^hXXRCpIr`HwnVCjBZ!oS=DlamFaZK zupg(QDjXa8sGv!{0X4^?zTp9)0YpTG3;jAaMI>a9LBx?lBzuS-{bBa47aG&-g*AU1 zTliBahhuhL!SK7hL4|yXu!8JEdEwGOaV3g{H`|BCUFlIW^droTo?{8g(dA(=CJoVX z7h?<}1lLZu&POXs2WX_d7CV8X+uuz_*8JA$A=`1;n8d)weX_C$5!AwkJpud(iF*YN zkFS`Oc1=afcqYbB_ijFLS&ErS=%F>$sc(0Q=RPJfAENj8i@X+OXE&U;Gn_KA=9?QW z3>nU~qmGZ)_7H0qh3h^ExRuX(D(!I8r*(1e?)~+LCk|h^_3vo-45zP2aPtXa1%O^l2tWy>?6*X&LN)l(?1e*x!8{+M_W zqYM7*h?H45^V{F*udjdNWH4dq{h?n|DCBep`Sk7Lb5cm)&wm}aQXu@lpXz_>5IJMp zRpBwP!3FN5-$-H2c!Gj4m|mwdVew?;O6EPA8Rk%k_(MKv((ywX36p?4#IGt)h!XzWR}R{rysabr zQfM)Ble=4WDuMn+2b@+uk@(27X7VJgQo%A>B40u<}cb zADgW!%_ql-n2kYA%%Q~I3~$Q1?MJdg-9ZE${IZT5)?UU6;Zc?I2NfYEQ=->GB+a}{ ziurn{g*Xt8Ca3P(bMT4Lp#PWV4vNBga zGJoc*%GWJ)gR7x1?dr!hPzB2ybfBR^^7;OL1z}KHO9y)*Dnm3?@3EEa`zO!J(WVhjvNsK_eBeN<+uYavHLy-mda+C5+`%uX$E^`M z8{*SeKG!K9yqDz-ViRtdZx|p_IC||AR|P1jMD$AoWJvhF0@@w8k_V5$(i6XD?_Qt? zK^WCE2pfvb4vsbfa^HVHFfbajr!nLlc;U!}JpAgWB4eq3r%-d8sd57Y5<~U-bnDT| zO^m%?)pF>Oev_V_UUzSA?+=re-yU)-`;+c1H77^QHkwDRnS>fztO5#zF=&3`mQh*B zP?WsTx=nq=s-iy~>CG&l0=I>L;uD6RQEI-)Wr&j;_!+7$o@r6rlC66IJN^vB~g^-#40)w3v~E+UJP z0WK$ow&xtxjJ}E!7e=V!(3opj3t9k1Uk0 zv(v*(Fj!?tpr{$PDg!J6*SZ6!hFe7B5;9#SbdQ!NM^|prIxk5^8tx<%fm~FLEQ@Ct zo)>#$AM}y6Qk$sRvW1S)iAaW10CBB9gK_miSh32BVMfv}&jb(BlxbdvW703bTSatd z@KRQq<>cg)3W%*88@d5U2pR-8o_6Eg9@psx0~&!8fdMTSHE=Z1%A0gLM@L5&ezspH z#u%uhlMpyw!e&E@_Lg3{L^KWn#^h#@Dh~fXcg`vzsSr()L>Z7Y7(fLYVYa@~kLe=u zLBsclSw(WT%a29K5$oSKzPuzsJWZk)w zmkLmJ5rpTC(VmeR4lOKH@n(`rE^{vIXmG*Hvr&gQA}p?6P9Z2BD+caVV}L9Q@ps1KWex z3T@ym@pDBoL|yw56f1o(2SoElkmg~<2t%X>eqU4sZhj`h>f9eunW5uyp*r6BQ~wG? zz9KnWrhf!5%pqiuRZ4IX0KBaz9ZsoB{s9Tc@2=s}%rI$8HCU1S2wH>4$XBP8Yj+l; zetvhJZAoZdxQKlFDFjyV+-Jx0MpbTk8@MqHTO{KdJi^Az_2N@g7~xv*ZRwVsr(pY`((t%1z}#Q}O1>zsgaWR& zgBj|BH3@-){Q<{K;1@BekyM2!lE1t(Z}!l6yh9ekEBBEjO0+brTl`D{D8#4dq|$+- z=Nz#$L3@aYq&5IrUkSO5BM#zJymoE|Q7=H890R|c3w7%S?jq&ZO2~~~rK1*k8fU9f z(+z+wf)Eu2n2!*tNZ5*2mx;5P9~UPoq*ajKPz9*cs$m->KR~lq4i+#>yY_VpvEa!* z*t8Pekzd-}=F6@*%z>OwfaXYiW8`s3#X&fzt{%cOYpI<)+04~lJe=4qP(m~L>m zC%AR;vTohFMVQyjkBP9Ytt-mvp2ebX8y8H`FL@3&Hb!5Ae4P?9hD-tD+Z{5ZDX z+wEC&#qt!nFeU5fgWV(i`I9s(;sc&V_~aR;UF*+Ac(<6ZufRi(5@d3`C`|zTY6zIs zk;hh_4VO?yIiwoPKu+S}r?#n&rVQHy7RDKH_(~A(kV7xq$~(yL>hjr+@_qaux0+FY zxJfhYFY<-zx5hvRI5I_yLE`fB=h2aogTZtM=4xo{z=_`Y>}rddF%bH!0na$XB*&rJ0eG<WH^_rsx>C_vWI(H8#|N#`J{8tfJ2^M4jR? zZA>F&%nHKPcnxoL^v8!~Ke`kpyJu(q_@k%UURrvhLQ(2)?_2NfdpGm+%({^CWO)Lj z0IHLcY6>`rSUAC{#^Cx>Fc}TN-I5_DS%1DUO)1|g$+Wke8g$fozuQEQ(s`eT+tT() z*>YM@r<5wfm4c2`MIH+;A;aO}?AXFO3Huf?m^Q-bOdlMjRh?SYbUyVfA$7um{_{IB=oKf>exCAkFHbyK9Vkh70WZQLi(`Ws-VE~ zmaem_iSnGBZmQx*ma`p{=hY0(Z(w)j$e#|L@8m4MY2MNp{LibBC_=Z$726GCp?!Tf zh>aVh1j<#_*eH9Uh-tadf@`TQbXa?MOj}K)Dk5|tdCX-WxmL<%XP;wNX3LaU^rO`Q zPc_D5m$1J5u4Errg-jD^8@+CZg~A<$?;)mWMpy@a5FKNd@1bG zZP>f(piZcZvy4YJLrAXVa-O=suwh4|lyiooSi8X9fkbtYazH)MKJS`edy>9!T@1JOf13MZ1$ehxn6=1rF z;}wz~g!TIOLzfLQn+6x&Gti48#8d{F)dyO&zXW_W5w|OC4s`vpiC#>WQu}tDi;|ua@iRELydtj8w6~0$v-M(^1zmd48O26E6Z6)kb=+9*sN8S#w4Pma8 zZ0=L7ScxS*RtG9?ioecbY#R9Ygf~{TEW7|R-*#qM*uP+0)Q44dg`jW$L#N@k8&qr9 zc&O0n|_Zh=0DhIeF;VKcAT4 z=$)@ODwf_1e9Fj-@7M2?{8wMs|GjC;p__FQ*^QLjz{CC=#1sM{PpL6YXV#8foMQ5J zkh=;3Y6AKMh}iUC3okd7xF%B!Dw2rpf@E%y6XZ}2VUoFFdQiqt+ID^Z3jvji;jj8p z?uulGxs@Kj@0?s7~?r|ve}_U&<~=PqLK?J`K3N7bzc_3vQmI&!-9vb-e5eE99) z202Na1{FncQp&Eql{YZ(ByXdTDy&7|c30^T=BiUKeiO${|bg7i{?U36r91BO`s5uQOK|Qf z(1L}{`2Y_}4nE!&2VYCM9)QBk%*>{T&VZMabA3@xKkjaw#A@UaT_zbI%3G=mF<~e1n}!kU;lkso=T7!X4Gcrk}o0Jm4qK% zcLMVMh2^D1Vos%2k!B-SQL7t>M8wW*H_}Y(f&Hjw&Z8>aN5v?;K_6EF5~MCl$Lad~ zl{csqj>LsTkSTVTV|uhL8Pp&=4I1Sck&(NAcd1akRA4?zhu;h4AV`~3VzNVP2C00A zZ>OQ|v6q*MiOH)85z9U|)G!?#9nChU?JGEc1fT9`9?_BXyQ8299`mwP8avVv zUbk)21WKZb^ZV;|O1o*%{P&(_3Y^9Rh*Xe&@7}$euNT2M;h=hoNHVn?{Xl~7*>Vh% zYF$R=t^3(UX`X>+B5S=bh5Xm)70;Y`RKwN2ALzIaoC;O5L(7KBB{tg3gqk%L{t$7( zf(P*HO2HTg(_!WN(lh@lO40=w6FS?t$jo$z@YmV4V>#@foNg`p_~FCvFl-1S<-+>X zObcD9Ux_~n-l1+2Y5V|ru`{oS6erU5AQm9MBx1!n1ae8n2a|0(*KgdMCt^Nv8_D-5}`qdH1d?9yxU^FgG6J#sO#3GCjAIUxxtJ z&|-MC)L^wC_#F1$1VCX@v%(2}rB}!p(?b<&hl2_buQuG6J_Wd81Z9~Nsi3y2Q?4nf zT9EhTq5O7m%_LB87VD0C&)}RQtgj^#r@bdx(TWn|cQ7e*98fe0T>G*qfBe%9%y$~5 za5U)`f{sEAu-42rRPgoY@I?yOyi&i2BH5@a`o7d$-`>pbmB$!k#gzH#&$Sop4Svl= zcITnPuCv{NW)C&VZ5}pq#BVsuaG8zdhkLJTU<;m z3vQi4QJg?Af_4Vg*p+|EY(PoS_ElmSrniq?K`X!Wkw`vJ32t%N$_0H^Xkj`4N`pqI zNrqp4J;1?nXVuQEwQt@4NGZKIqiR6pi)c2TH_UUO{ldHv@DgClZ3K%@9xj;xZ>JFu zBE~KB9@9=E+jx&*bXaWKvR~!pF0XA@Rs<)Rfgj))y?$G_Es_usE9CxWUJYe1L#kNm zKx?mKLuyrE52|{?TQxnrA%rw1Q@DzG?q#+0fhd8INP{+wU{2&UFjLNhjVew*sd4t^ zXpmv8T`Ci%745$E`SRIw>)v1lWsJJOxq;7A#+@PR?KfzGz*Gb40@mhNaCgocS?AX< zJ$YXg^I5Kcksc%IT-y6R)>(>OeBB#wFIYHV?UsKKgg?dQ1+zU?cahRr8B4MpW_n3xJ}5@9sq z{IlZ&t@*hCOi=}{^BSSMFY|LM%CG+IspO~bFF~d^Z&t-d$Q#!>?Du@UGbT3TiPIDf z*!(xMXNK;sjj50pF5Km?>^gDy`Xek1LF+NR1|KI!ZDyxJx5Y_#@(kfJlD97^ z90Ok%gA}Rpbo*t+|45^L|Lq#U-Sg-CwXezH88{;|6P{@8(bYqS3l7%DgmiUF9yILo+M!UB5F%MS+(+!~Q($@`;PZRsWx0t><9nvn-D2>$Avbbn57FFhIziK|Um2jM6Y>4asOPbdt)>zH3RU*#Cv2I#=$w$ZGSYu92{&*tW|8yE8 zAQiN;IQ53v6Sp+4ddf*4f$yLjRq#)cz$WM>hz4z^^78VQ#PIX4BfCb;rKFuz8}rn; zHI%=E)GfYAT)Kt3qO`7*klZjX?>`%?Z_Dy_^)rhrhM)iJj|q(a(7oeaxWCl=nKkQe z-4~JzXkzwHQm*;EI2RD^T6?gRK77w~bjW(STZxXpjyKNcfh#?A3o2Xl;10wkwqWwN9sfS=W z)VH+rSe%)Y*L5>A6P_L|?CE^Zkjfjh{x2o1V1b?v8*hm{42-($X3Lc)Xx@xVYqzi% z@!hG5^=`0k%S^`N81W_=%V?)9gA+P-(eizWQ~_wDE1xFK4xn994;_6QZZ>p`iRnp- z{_S^rLOhK;Uud7Pt_i zk+_(UXh|RGKwDuUiNboe%s+MAwRInvja@eESF+wrpBgOFvmb2uKDGANhJB@1mb|3~ zg7|;uFf7>sWz(qkjF?lkR=!&mhvCqr2}N8enn@>5G30{o4*x#X1n3Cn{`v>DX->;+l7S!CQn79 z^R>mTrKHwYBmPB80kni4faQ!Zdk_dtSu!=>i&Zl%R6lvAI?B^#{JC<%lgZ)C>&Z(M z(h3bJ+0q3yx_`>`EDf&RDi4dA_{>dB zlpb=&Ssm`xt+zzecBIt8*a_RwR&7wmPdJGEBj0JN3NB27M;;#i)nc!-wBh6p!x;<1 zQJwxqL0EIRj8!W>EVr+=ewZ!cSo-Z(MrN>pAv|B4J8-?m)Z)5UDUzBE@6%I?uA*;*cQtDR1M`LO`RqnI)&4Q3XI->kI zNJv5Dq=b1v8rFMl+1+NIsc@}gn<(%3&UXKM<||KCWre$q83C>CZOtR54M*Z+*IL)l z-&|dVW!W$=e4UXA9$e}^1t%%s3mGF4u3%b(H&>Ke1t7nV3fKEZP$g-mO+vl4OrAxw z1xovMyZVQVr`X+1_|ErQ$iZdM`N>g8>fhgovYUABm$zpnzCB#tMQ_*4-#XuzUbSrQ zw417t-#Anf7=4-zv>6`A{0E-ie9O+AiBJJh0coqouA-ErIPp*4-xy=oQo4JX<}aBm zp#Qgk&CBd5R(wP09;88Guv5TI1S8wb{=9}dLkIl6&y{V#hxP7(btjUwMCr)HSYqo# z*Pr^73`ZqJ<~(pf72p*dIeb*I(*^W714}!KPxWWtx(NCpX8<4X za4T<3vU&Mg`^+u8rB59vg7bT?TnsZ(t>J(&z1aKl6PPp~cm?b~bEpa3y3XH2gpg~yK75i_ zOO_8hqP~$GcT1RNca#k1pxj4PCej4$;!=9-S1=94TdA32B@3KzuzDv`s>r!6i!5ok zhg+21Tva-LYc}Ss7uuBF7eWBnvmY$LDbtiu6=LqR;!3jzy3wY+fBzo)NAy&wdRqol zOa^{li@Q=ZDA|Ad@zbKRsOKE)%x9EY4#^BOd|$Y_YuDSwakC8f!ojN!sn@BR8Ba%x zqr8>C{diOt@vOC>IQX!N__*XIC&Pa?-)Y{oKY+V?NhGr>Jz=uT@A_!Ix3}o4#Srt- zZ>zT-xfHk+vs->5N4A5qEB*K1$4Z5ay(O+rE4xqn+DyNi?`$#oahyx!(i-N#J$qy5 zZGqE6ha(gtMrK_$eEcN8mCwN(lf&mHv1??iq(lo?L}oKPEiLqHF)m!_XkZny6L2m& z?RA9RynxGf?53b~`#x(nmrGBTpWPC;dWZfINMdAV(->Y|-8kyTR8HsKbJkhTz2!>p zZuML>9kKgyOTa8N?ea|l#}wt52b=!d&8yk7RDYkCAZ@~WDJHzSQgGprAo`HMbg{lA(NfR4VhZJmYyA!(f47Bi zq3&(ZZqIir^hnIm+l19p0+T9kM+lW2ooG*d{O=2pekNikCh|F4U&e3%JE1uskzAm- zf`4{8STM%C5VOaRvn-(N^ke^}r6!ZL>-?n_tF0MwYX0j~s+JML=HaAeTtWvPH`2Hx zuw3}e0`eUJ*$GvvDcG}u&#>vvA3cZc6<;dJqMXv&G~6HS7e3+tcxTy6zEhC(IO9qw z5ROGpSY(-HY^Osu#~?c)kQyAGY_dNMS*s|$zeiL3l%E}ENcjG1``y+Cm&U!8b5F|b zbfxJ<%+#je*3+!Ik3UA;*+7hc}|k zZk4;N*Jf3-VPPPTZR(R=nu#KvaAaE*XZi?f_Crou+eb%S*8kWts*AetB!h zB1q#_v)ehqwt1fyo9T$`$jB<~BWg6o@}Q$iwXa6J+p^v$=aVWQmjOb$&l&Bu2t=Xl z;+xY63aDZ%7|R7^|1M0EvVwzi4ku<5$4u^%%>k5$Bb*Emu7@D3$#FIKe&H5v)#C@bpnEv4~wJ9O+G5b zIPxHb`9tD}M=43VqpA-3GE-<%3llwu22hk0vj@V~FeFO%@48pd`Lnubhib*`OfffQ z|4o~079HT1U38jo{P=)ucon@VnX8iYFx2TdpC!Hl^y5A!Y#O{n)oAUT>WlI=!p2Di zwZw>$&I+jOuN5@Wr2>Z$AGHcod2%2j*$BM)15jp|E2y<&L3v&B<)iQ`MS7&^-*0%h zG}C$gSs&N^qTKa+`-xG^@Ak^XYm2P;U@^;uUBCR9G2$SUzxX zq^Xj4We=8a@esBYiH5Yi|LhS@P3JE~%nGOjm=X$ReTSE7^K>q6I4+V9zZkz zs}JHG{OVWD`fFbWp?E`Nm~}p6_iC0C;A^|i_uEVGy$nEaQi3ya^Iw0FR^Hxhw44wN z3>l)k+k;6*JXF^GhThmzX{)ZwGl<6hCCj=g_9(82zzlCl@fJ&p#0&irP5#CnCX}cs z2N(v*beu|#ilQFMPSnaU0l`|8S|RT^AwNGIH1Sq5O|E5p@n33ZS)unQud+~r@P7uc zlEki6L<*><6+`?9TzBL4d4 zr(EpTGEuSPdxSvAY5szzk;1JOruKpKR)vH_&15ca`FQoCiCcIds9k#cOwu`e3vbmx zkU{gj{SDiNJHHx^U-evAh-P;eKzQ}uvxn7bdC>t{WNl@pZM#QyNW_|^!8U6)-t^8z{ zdF4da1=-F|lJVek(6sTUBXN1hQzdLplZ1T;q~wfM)< zM#>BfzkcW+s1aZh7dpqEpC${pYu!WsC#pAY_-kY|57*9=sK4H&JXqID ziY2yxIP?2J--Z4eOT&(n^ukQlVt!@0fvS&chYv%rP8tt)E>C?cCZm3RKw zXmWT+oX``5w34JZTVtCtKWq?1IxWAP*18_gDqJPjb6=hnE3I z+y4x9t?Vrgib5wMM5g!Zp$a&iA)t8@Y~xBGp)(Gh=qKB&VS^*cc=6CRcwpWj=B1Qn zclu{!fhllE2>>0W9STZE9I%K+@KK7x1C{yL3QFE=RZZp!Ex93*S-hI^hYwesW)BjL znVx~`Bu|UsPTeVG;&%A(KjlO4Xwg<(hyBP~;q8J9PUvx4 zFw?ryw;#k1c1{?(e2QB3^zaGd=7pd{92wZ$<|PEq@(EP!-_iaUj|U(-nvs2MczerN zdx!@d?9X1T*eH8i!HXp*zy6-3YpZS4sj3pcIBBbos0`_o20Y{TltgF5gd<-oOCCG7Q1R244?HE-=c$m?lT;y9vm)yOvQ6A&8WH` z+Z80)_whkceqWe)Za9zf=f!dQN=MOG*47mm4Y$T>Vuv}nArL0SMRaZw-2rK{H>FGi z(~WGw0zQILU(@liB$M&ir*}1syd!29j$MuvB;-lXYDwqd{2o=4EEF2qAu^+_G;ePh z2`0;gdm4Y=e#AkFogy>Pg+Y7%A>5imSxSi9HR0RX$w_B;mu($Be{0vb!ex2&l&6;V z!!);LZElq>%^@uw*OeNQp^L$Q4USA=x!QQbzi*3SK8C5HzMgaA!Hst*6Muy!qBLB`K%jMgg3>} zyHAwr46{i#N1P~rTm@+s(3+>7nQbWhuV00+gV#iBvbJAihF}{r@7 z`LHN@Cg|#Pux{Rsa@n?wvuoA|oQrs3RBhpY)9uHt+3q_L5h!~^<9F>j&LZ%cb#BCL zo*qIEIbg|@cSX)y%1Ha>IesZh^uc6xjwDpLWIKv;p(4*<7$YY2VDR&4r*Z#jD56({=`l1=Tl z=>JCX$Zxo^+1}kfwNA?NN=DGpvgz)xWp_K9(E?`uEXQ|mOyROn)2h`miqcbOZWdTP z+r(;8e$)Grz(Z1#^@azpt^oVi?xu6}10hpEgPaHFMw4`hX#zU3vk(_JDGHv&b?B0{ zGkP(ww_1#S>tW}Vqu29vUgcg4;8sf1=0io1j2S8%* z?`56DJL-d00v)`F2n-KXf2CY{2NQ>CapA^=y2b*KcLXz1PBg~jYD?%F%ZfP-H za7c}^z3-J zBQx6T;?;y7{;(S^y|DU!(pD;tr4(#srznY|zSu`LFCtfg7C(xW+voN5Ib_8qL?NT9 z56v2;Lzf8m{B4i*)2RTS5${5ipz2paqM^>^to>7UWjezbV@o>+YFc(kuS_!{L&2?A z_sY_I=%iiWM@En8@&xz{VFUXg?R|MT)@$4D)3QpniUt}aLy<8og=9*D451_;QxYL_ z=CLeI8VpH>D3K(|l*|fAh?K}YKS<^w^LO4)t#7~Y`@P5B-*N1{k7NI_`{P~fZFwHQ z;l8i?y3Xr7&x@%WRrfx%sNK|kDj*LayogyUtAvIhO3OjpXMQFzt5z>~N6*S02otU9 zjjY9}gN~-(Md(|E@_tYP!d=f^jEV$R!?OB;{0!>WBsCBpo4Q8N)^$p>?sm7g{Ic8K zsrh^ViG(AY$6RfPC)hf#Ggsz)MXOG7I7r5AWC=&cSh&@f!7iNi;clHZpz;@rU*9Nk zCB3ybqYl|pT%LO~yPVykT^Vb?*P|aUQ>CG?#wD@a56AKyuP9AD8|A-Kf2br`t1{cb zgQEByEUbB8&#%5u!xGh%7rcpq(m{c;s(XPR?n5ma{u>rKALcM^aPmyd3f!KYIn-?V zGON+1Ak0vqtBlBmUT)pPu~o)`K^WgSFIoR^A_QBVo@z{7&#are45NK}GGr z^D=YawqH{M!@n(RdbV{&FG_mwws+!Y+odZNft!#mr##l91ZTpRiz8RY-~7>YeKhK5 zYAn-czRGCfI8}iV-qX)bZ2X4ugD|Z% zWG;n|vCK9eMA#q5P}h@`wbD&%Zho$(q>IDdZ>6blD$n_^+d_n_gqZ4LuVwAgn;z{6 zSnNGFRoYt6I^ps~a98~I#yIm&P3-Ix>DP#WU?`)S8M#Xx`kVtYZSKiXA(*q79&?#V z|JHFKs=g=5_H*?^|uaoeq>4E`dm{6!YuY`93vb-l(#Z>AWd}%(9WwoH8}uaNVwq6 zhbXZ|IG1=gZJC}?&T$G$EK5E`ZLqw*jzG$3J=<_Y-SKZ`)?!oW z*j6o}G*t?$9LM$|Hf%H$2P~LGq#N(=OJyR5iU)7q9BnS$Jhu#(Av}cK!7_$tK^QO% z+a@iWvAcWd4MeGrZf`ohp-&~$&6L3-rp?HKZK3iyQy_q`ycAf$En5|^>Hz=A4?j^d zP2IG&c_~W@kLSkhV<*-zV6CI%(nE{x6KIioAIxAAgsd1K|9WBu?4OFhu3{9pa@-Co zX{SaGGSFJvVU6!)<}cJZ0LpPW1Sp5_SBRr40i?iF_=NTf!PiMm1m~x_MZt!0xaKBb z(D@`Lx93iVcXggw=v#62z2ee8!e#0uq#WLYId@xPN_uKS%za=9hZqomc?Ci@#7Uc^ zh7Y$r*Os9-2wKS$pp|^ro~q_1WGGwJg*|K5LGd}ET>=eUVb9#WjSOSUdUCZxP$tp@ zc(a|YuYx>>TAurNx>|6%Q*0$@0=uSC6?H~x;#Ju=0hNuKlSXqhvS?gY&<2roX8s`dmhj!?gZjq6)u(tX+Tgq5vUTa zk_q0oj}(-s{MtuFZ^TkRM2=h#r?jy_yOf7rLA;_++76zVLW7h@Bvx+{DWoaLYy}Wo z6HltZyVJAl-DyWO7ev8qn1|i#Coo3t0H5gw=Qs#9E1a0U*pa7;iak_QerjS=MIzT0Uasp=;C z&;m&BH8<0^N^b^#NtTF4Ied|U<1E@AvgfKf5|87%3+W!q)1{T>njasluTocLJ>U1u zQs020u`xLUE^1hsiQFo&?-UbscJ=ihGcR6_CSIDEcMVr!8eM>{i#}sEG#so9r8}$O zBkL}*1)q>!{$E-8dj%*7H_`M4miIp>X@HA~lT*wye8|Xb#R~Lcda5T6`Nyiu23pkJ zeS0JDI&<1yDN2GLK1sBZHN;pfL{NuV?`on)C-@E&qQV@)H$B_|zUlXTujj4~Xy)O` z4C>*Z3O8e;WGZDEbb6NWQ%1LAUW-87klK~e^om9Y+$ZljRCA|sgjVEzrB((%R?mbC zHevPB>{vv6(zMcm&CfHEA5eY~@h2ageAess{Dc51?SL85NJ7mLdZ+`YmDtckYY4mc zx3m{;bP`Px|BVp>ujJ-lq#OI8s)XrNZP*e;RV*-e8Mkzmi-UkZNtgrh@IF#9%`Xgf zaN+ZX>BB72+Mp-1Hdt>$OI9P_qo%7&qbx|vY&nM*A*CQu`1z{okD%y|memN}@%%f8 z232yx$as`=DH@x>|H?f7)xNGy7QI&M3PM9+j2WW_Z=eQ!PEUaEAe%b{xfOF{taN-LF1|D9CHS`XFU+u>0!4*q8Qq3~O zg((UCCz*OB8laGJ>2=hK%~#?iwYM%h;eT3>|NJ)>cMqwqw(ld~1AH?} zGY>Gdu(1y-z_a$n-la_y!=gAhcftC`bk!E|3Svo>0{n~kM*|ec8x<4$FgQuv)(-@w zm{zicPU6~|VRvnuDz^)f>A*ThHfeBX;lx1xg=_Ze_3NVtC4F-UBaX0v33-E!UG~Ux>~QG(~<$_JW+?vJ1sl*`*tU2I$ihJdAJ>laBp|1E(Rq%;UpuV~&a8_rJ}9!bbE< z&{Zd5)2Ia9Vv~9Lb2!4sjsbjRR2!RmqZ%i;bMOB7Q0zqp~yy@&8+g?=;wPr75`=jZo7ze*XeJ*j23 z6yRwy{~)nf2lbrjkb%9od~JDUcP6yavX7&XRwf_PA>WeuB{vJYsUtUb|Eqp#u7Cd}+_esd)dd8qnlKX=xpobh=Q%4vCHR)ePgW1Cf%Q3jk zN0wsR1>sBs(ACVe*8jE+sUpRibGK%H{V|K?hJmE~V~JW(Wv9#5jj3ERt&DwixYDm5 zU9*izDhap%5MQv>Krq_2ZJTDy!4+hz2qE-=%l8N}Nb~IYd@pjO488`Tx1+>Ved4&f zPB7gcwM)RiS?WgMDC}u-Gz=S3&-x1)FF?UuhDB+N1N~c3F-p8EKqWse4qjyhxr7!L%Ot;GaL=?pZWRB+0OdY5BH)7fB%~& zeD=wxmJA1~nvw?84up!$+?gncUr}FucW0)obNeMT`xiy~9rsz_!e-E_BP4J_a6y9% zyeBAG0pfol263?VAvhz2q8Gxqomr4~vqFu7*#~f7i~ahIjhix$>bItT)#&l_p}&KS zgz(!5H=pys8Zw2a9inulh3pCTP`K?qs z{@6uUwvBiI4yN+bl76S_p z9BG~&gVxnPm{fuEA|h=>nr0A|aNuidO;6>QV)yZ5Y1-<+Ju@SQ-dPgBR5hjsQP=m_ zMTn#946Djke%8u3gf=R zUPj~GxhD{1BZcRIcuw~YYA=eB4L!p>fwG}i8`;qwv5im1#)5k)AVWhhw0(bH@d_Yf zd_rnZ^-ncql+UMdkNIESa&rq$lA(>)Y2d|CWlIvG0Y@Mo_&B*m>sCDy9mo?O;OI_( z-R%ryRbucm{DZb`*#r?6`#yn8K6tw}H*5M(;YP!IGIcU;itBeM!h+wI9=ufFWitNts;rb&&Jj}g_j*kE<@7^gw!tlve8bfACasm%x4df~l9}lzl!FVT4h>%zdXppDQ zrWmF7M2Li;kC9{gDKX!&w33?GTdf`10{6PAGPAGiJ@n8s_}+QQl}Xbyqe73bk_ws9OXPw_;Ew5z8oI@!mEA_GFghcTS!#LI6d z&9$d6DOOdWHj&!kG#GhDGR>?5QmCa%!(@IqP{aL;#Voic2Kcb~SF?15Ne5)(dS)OK z4}m5B3HXfTk)UP_?t({#^de-jaPMk*vx-EsCS{XYo#mkdJXWpN8@{=YDuPY!o|Y2< zc%XeNd^gMDwm#d)%p8X(!w>5Nq3=+w5H~UX>yw{((RxR#4KtZEkV(|`;p-^Qac+)q zLqjJvEz_-D#Nq1v+_b69WDxh@O|Vq%h$VjYjr#M^Zo3%wPS@Yap3uKI`2EKkQvAw; ze3I^(HZAS#)mQ!S8x*xc3%c162p7q3W7Csx5|AA!PO8c( zuXJbwQ}^Tr=;$D{oF8f53TTQ1H6Qwuw^%hUq)$Un@~gd*1-$Ad+EqPcXUk*PamVI+ z!hb{T%2ztRi)=Kjo|l;}cB9{yEa};Beh?|1hT_#}kDtWdbMp)H;CkN^$fjFlrUgwO zdaa`Y)3%xAp~-T$ul)(N7P=B0oz3j*Ef;Bs1BLAz;61@@`};q_hu|1IDP|^5ISkJJ zBGSs{(*2YPo*!~@zi;>o?E`iXj)}fAgLzTh{@|H^&d<|Kdz>Y=H((vSuF7A8;Av6V z4$8`c@l}>wzc5sQB^On;eTPCrPF<7JPH!&Ip#laiL@|LTet26&yB)S2Lzphl?W*s!10v85 z4v_Vd3?Ic$a5`*;4#Gj%`W5X7FiSM3-{>PedYWq}H&XEOXNhA`>{e_|Cr(@h?@IA{tV zos(d441F`(zI{6(ccHpf(!lLcz}1@%mF4tlOg%|J3qO2dm{!0R`uciSTN$GkW!YtV zV`1i-FciD$@byR{{yp%P|DxjoYO*ex?Szs*=4hY}vlysPA-G$~S8}jW76ja?t=_b@ z8tyW!s78Mz2`>o`0K;|+gi4&@o=TMExG8=Rs6fH~1c#un#y<@pmu$E;Ci>HS%9xWF z$P|_z+fvVXpos+c|EcrX|EYcQ|9{cH^L6{b|8%DUh}IEE4xyCW+S;bpv4Ud_O#1jw zf1PnygAs#ip}J?~+x6*Tk+Qb_MFqq&}t;s{XP^Q^h2e7(m4R z2u?!ZiHa)9YsP%5jGHFf3ISX<+2su2-9wCJkdnm9dd~EMW>N#nXrx>nY8>JV0n7F5 z#2^1;t9E=KpZpeS?I)`TcrXgDkPO6tn^*(k+W?M2WR4+vc?2*5BY|wwJ?a3zMA&Fj zcN$9kZ3j&prE9R$Vd)u~DnXWS+eJI%X-exnUrx$jv|dk)HTgbpOWuYlm{+4Ef~g}q zA1J3~%f2DWBR0=!HEb*_;Tj2K`?~n(zp_OhBoD{3V-FUg-kYq2!M?S7CFTtg3JBxb zJ0L3kf_Ff_A;B0>%lmd)?YnjJWUL81!APE|a=9Ha6@%n|MX3JyFKb5@1&TySZV6L| z#(vG=Mc(Q~U&1xS?H7C(L)x01SVuyi`1$&_8Yhh5pk@`zt#7o>-60|(^0mc536md* zUR0$qNh{Dh;P++AbZ6M*6FVC@>1}S?Tw%?X$=_-}Jo&@v-Rs7G{>vA87rf=Gvht2T zSfu|IE{}I*Kv(z-%kkGVnm{fhJo{yLO!| zU}~%5YDw0J57R!z>?^BKCSnCUvV{^mCNcmBFXjLSiIHKTpCPyGr=E41M+6|d{=gUpbZj219dpNEX1rD6SEXYMZCs+m$XI#7!g7A{wBz)Wx zp?JOt&Lp=mmi_AiL+HglA1(1F-0MU46XzO2I(tybBooqq6QA>;JJL*rZw(N8Jv`1m-ToB!=D(G zJ0PdTpJQ+yb3RI>eZK(YSa7hfxQS&@SHAze`t``@CAVDD`JBiIBQaz#yfT-mi?xjbD%-}-{;Takt zWULY~^+=g&vtHc#3K5E!(~+61Ao~z2eEocnuV+TM%S2X@sVcy_pfXylN(r1f_Mr13 zDAC;bp%Zuxsj9i43=oSe!h6S~z<@X6>&DJL6!?R1ao{dMCeoaPu9Hy6$#^-dmfUH% z(t`Ww0+_?fiM-cSolSi*@6eJ!FSuuP6WH-&a4bO5x%hfa3k6ISOqvzIx6Fa1Ho)DU z%#O`9g#e4pRRR%|xEkMNlhFnKSQmjFQM7SuuiDUSGRtVZ_ZV?q!`g`;5ua$B$d8_R$-6*eeUo@6*VtpqJwUD!BxG* zbE=uCiu)L9_zYquNX90BR!5Ba0mYJmTehbvFl-7jR)Iu;{KW-#+_(-AFX1JVN0?cv zeqqdeHXq`6EI`mzi?@&XGx5bB6+TF;e~~Pocwr=ueMXv=d~DyW<>-ka2wWBMc@3Z< z`;fS65MNBLD>>7wOr^tI;8NheYFPKc<3;@^b0P#7fNKGAk}AX?A3r3RkDS`fV-E zO^aw_U|1$ll|g~foXP~mZUubP;_(~q*PPQD^QiEZ5l5%GY2BieH+^RLlvfj9JW_oY z+#P%q(NWk;hC4TQVj(mJn4E8XS?P*FTe%oSGda6pGnCg85D6w73#o6;jy#HqS*7$A zg+xc7i-b1=%7)X5qJU8tsbVDmXt&*SgIJ}NbhP$6*uqSrsIIQ=9@KYf@CCb%X^}Wb zoTqj)ol(QILy;&bFlXJq&t41d)f#$@y|)@Z7Tfvx-+y%?A8ohYgk;Il-2;;@k$`PR zucn~!l-kG-n8RWosyJFjXIKG|2AEWUa*{uK(fJ|$)|Vt9vqT<7#!+Qpv|C=tJBU~qV5)W9Eru|A?)bA&ApHy77AL~$}G455ugL}=K=UT{E3B)G|J@Tm0j-~EF zK_k-b*7juK0{kPA1VX*V5n^YVyJfy^YOdgIL5NMkycQ9Df=l>0*-#FS9Itl9V?0JL zc49T?j7UXjv-RMai$W(S(Fifb{S2|ghQA+(rLFG^Va=Dudj76I6ppgCrC3Q1yDXQ#u- z=$nfyi?EI%wl3t}F-L{CX5dl_zEz!7p34;Z*Am_BQSq8{VEk-uZ*RwVRfHHaBPRv@ zT&cGOt&s{hHl3D+)0YC0!qc1HdiTpam$te>*m(m>F5ZPKFLd*_V9o3TRot_@ZVufu zGPLy!Ch?G%nNN=pB}!%o<-&f{IizU5=gcPOsi;^@Y4wQ*t*UBdKoW_MXJ9Av7IVVL z7jJ=$zX+(CdVtSg9`_`%2?WjH?B!2y5?on6y^v~M_2aFZO zJB?Or+zmhe#x(A2FV`bAU%rDPkAv&R8M{-%dF6TEJ-C$GH#=LzF14=9V2{6wbadGL zi+Pb$nIZeTW6e{MQ}^U%nO6GM=X>VpyT0s^kdXKzNlO5=3U_~he=#`yC1_(TrL1LU zuI01(5l_1C<50Nl787%{SU{7|FCro$pv+|dqmNgK)8<3hoehl)XwdOsR9;h0mPLjx zN`*tIbKZ?4UT`w4JQQ*EHiQ$;bTj$`rzg`B&R)<;(7awzal~$B+&nNUO2|Ysj8{oaP*CZAGwkM9}w!5Z4%R zO%OFBAf`$xXpxsOn!1IRbrrVUk$WOba9_wHgK|}oOiLmWxr`b~KK|3gCr_4RfnQO0 zWG;5eoKa=kK8sauZkTS}`t=Vr5~|b8)TNRCU%h$LZ%xdBuU}raN2jKx1qK9ge9bET z(u3-Z4Wa5aAgK|6K%BC&Ez+0zu1XhXWzzBKSR5~xY=(49QAueD1!;W;ntCO8_iHzA z<};5H`)nZw_QFk|TT9W<+Uz#odl3hfF=kS1s9F-btsqYQu8{t9R4%_!h%FS#)a_DI zTl)I?7%E#cZP<|nvdS{9nU@$@QRb8E~yHih3udSow5*`H^3qGfzjN_zx@2#aRNO_PCk6%1oPBr>95!WZ(~Q z23UiJIy>p`5il*D)y|`UW{4A^8Z~K>zNnXEI zVIx^E7Tu~mHSqP*^RF5^GvFg)Sd9B6Gu6gaAARHy6JrKvn*Z4rystvfo`tW2f^kUHYvF;?|YuBzRs;c^BW=a4+O##ZOC1vF3==kKx6A{Cy^lrL(%p~uziI01i<1(_%-rjx< zGxO36-J8I*jbT%5>*!d5P84gD9CQ!86K0E5>rowf^M~;AI1N4=={&MX+_ge8 zvGji17ja5gx8Y1#;?}fdAckXaK zO-%g$KJ|{7RWxQSv<^|01>tUpEHUoOnoHSZ3CtAI1nz}9A12h`%oUwU`w8W(#s61H65?JJf1&) z9+RJMP;*{oT2)om743gKDkuh5aqRSAAwDRl-T0wHZD68?yxoM zDvgoIAVwZIcaB#Uo_&g4dT?%1dQe#$0&QpuUYFCc&jX+iNh zvf}_gm00KI>e`N@Pkhk}!?}t}ON+|O|KJZEAN3^%3=?B2};RM4)NA5X(kK(HEDknNM5y%!Tt*C4<5O-!^*EQJRFY~p+(BUzVo z3JR5*h;r`T%fiOS#w#TB7B;PIgM+v3-Mgo?y&K#vUOv9p;Pu-=A10`qzMNVWU5d#U zl~t0SGrIv&`y>5P&P>JgY`#<8X-&a1*3r_^TE_W1c721VG%xRR3G1dplwg8F0z5qQ z7^z)~8i6CYj6zxFyM&xCf`SJ%HJK=dm6fYum*9)|Tp*6?!G8t9C0Zbar-k%ntNx*F z+hVM5Zzr!@|7zUY(JTIa-ea)odX$^Xii^i$zLJw8rOD3>j0(~9oR1bq5i&sgvv z=Gp(afL|h1*x0xQMQt&vrhuTJntti={(jQkqeC)BKm6d`^QTWmL+kn#Ni-n9cD-DB7ByMfpAS%i==dtd}@ot?gLMLXVx*&DMfn#l=b6Zu$p}j5r_$xCKxZtMfqoCr)wk4d{1> zR~0i*aOhyXI^8gAVvInOa{(}iAW~U=j6Z~zpZ}6pVhJu{EfdpU#l;7qYbkxYb^^8P zjjd7}01B2siH@Y8bULru`fuB4jZa0f&|1 z4!(NziBy{;ZCTUclb= zQ8)vWKy(ilQJvNdYuA4CUU`^ons*j+2GRJ3-5w>@|3=)vB_js?w?fSS%Rjn6&)4)| VVX@4La4{)Lhtv)}-*0s3KLAKHCx`$5 literal 38821 zcmeFZXIPZk_a$0tV-{LPF##$dNkEaDO+*37l0^_GGLkd43W@?!$1_V0i1GxyHi`Euve^wZkaQ1#aPp0M{?Yp;FYJd~3-arDAb3gW3P+@Im;=0!;%0{DQ*e93mpFXp}T3Xy< z=V!lh{j#yGt(A=c2Z#B8y@B2GnGpxqUy=oIkt0@;>NXe*g&z8|FIhARi@}7RlDcUm=oC(D{}Jw1%)a zYpTy423 z;?jM*>c(Gi8yL*>k*9WhFEC=y$%~-KR=*8y~4`UCY7Xt^Gn)aa=RtG3yG~l12 zeMfEgUi4td1omE7cF1JnFv^b2Xl0xi(&y`l=O;=W?I^+|igB4e&JI|au zmt0~uWzv<0EqONL)3#uNo$D)ip2_d9T^?6Ri~D@yx@N(H6Bl`ut*kz6lSnG5%FK4N zJ?d$y894%7!drvhw0VYv3n$KV-otG?^(7dV4i69ei-yp9^Kfu*oMBeHM~LL@@H%mk zxEvf6m0oN!mJ0X!J(4$TdwW|s$3SXjwpUY>wA@0STd~?@n$u|};Ie8~GA08L$79~r zv*1=>*`GOv_s|&}99&#j2!L;xS)p}b|G=W096Q~aJ5{kY+u|};9y)91L0SnFaF|W+ zDYjv~apU1kSANT2m3K=y4?DZu>1z)j`d;9sq@<)56%`F;Qn;gCV%zLWC>7lyr>MpC(&FSj7>3-pvBA_i59S%Zj9tMO@zCsyK$VF|Jl8kB>9fJ6vFXmSXIYi zVnXP(i=W;17uigPMUimu#0BH^`9U=Fel)_VjiS3+OQVFAZ2em0^hY<7JVt|A)l7ay z@uw;#_(%t{B=oy)&4hHo50n~3w$f=_rf;GVX-Sm5aO1J(sZ*z>bxkCR2{NIrMb`S& zH8m!1ZJ3{CxW86-y!#RothyDhtELn*Ldj>@G{*f!cOH6^Q@6%RP>eK$E7khbc6<+J zZA+BB=(bqT=nQ|TbB>?8Ax0rI?@2!aQ>anqm>uIdbVR$%QQ_ptldDtCYqJ?2?s}Vi z`G?GTG9`&~vi2Zmwi<)k*7v5M<#{&RfDPi%?c=i^x)-rjxw}((({57v7_K)uIhpA$ zF58{toEFEY^5zu7yBJ{?4eBaIW@ct_*d8TgI2`WX=g(qs>bZs!G5&AfSR3Zq!s@8M zalFd-*QXB(DT)a$&i-Lx=jb#G%xa?eZ7iG!@lt`urD_7`B#6Qs^{-#QwleO{Hx0c~ zZ`@Z_T+Vj&>eu%M55hSa2rgQc?!93!qt@H&3*DpC!p?bD>iKL&AHWh1V$#dA=-n)s z(>v#7J=w~<)o*8#xlB(_e{N%OL}n?bx72}m|Ni~&mc|;pyZc<1WZ^N~w(MQHE8Mn@ zTUuJ`|N5l2S`Z&k9|wz2TujU}+i7Xkc1ye5IY0Z^i1gON=E~FwDRE`xpenmJZ{CPE zf4uK2U^f}> zj72NlNhLMD7n5MNyOS!(X+^Z)YUEim(ceq-st&d#E8OLO_WR+ZM`3OnGq49zK7G3J zv@u2-Hf&e1jZs9!+FXB@AGIKhWnbC%7~xXb5FfHCce>nnAW*2nR-%{ky~tZ2&n9a* zX_2awB$wha*T;sUBu47a33zG0--KQ*v0u z_LJ)0b8}hnbf%3l!q`pOXz6VST955)GoEf?kC(%bTlAN=F-wWK4;ld;pujU@%K$;+tSp!J-iKnMul6{>D z9(2n{j#|K0GD9O@F;lC!6#|Rq_Pqocj=KJSUD#MfnYxwkKc-aEAAOkW$V$VuBo>y9 z>b!jQirHglTj1I6*Yp#x-CG;NZ?&bX-#T%L?*?I|udnZdkmDyA@kWM}$6x;W9za)1 zouygnUj8#$FbU#q-v{BVSMP*!=-P+*>{L_fjz2#{^U2>zMiJI#TfEe1x1Ei#)dJXC zq|K?U3$Os#Jn#{`mKJy?E5VA2itmAp{y9%&1etkV?EKJ{7hUTIxb*mWHt_l&6U1v;K$z>S))CLlmi+g%-}g10=X$Q)?Gl-S!t+ zd_CHT&$|8M;KUoon5M9s`nHhIAR%>rNbb!w@`vS-;I_4j9dG*JyfssJ=I_5_S7*9S zCx{8ScoYCXd+x)y)J;Tz&xvTFloCu*T6&7BEh+E0Q5KVt`nvn%!z+`JmV0)&vY2~#mS8hWfZ#BZZp)fq&>(H;#TVls; z_Ui@&OMx<&7Si(K5037+skz?LLUij83z?Xi>)m^k72>#_{g&+Y*d;-<$DxorN{F0V z9Qom}vS!2Tu;fJ4Kj+m|V4lXTl$iYVM~L%Sw1aY{7JvEn;tz6sHe`j(ovk@L2prxW zu(i|~<)W z>)u~M0UaJEEm1a7fU;jfK|xwlvdN}VIBBiVMII7U6Qoy^>*Ad%(ek?g_Uut|^7h~L zfh)iC{S&(9;FjB9A;XKq;oiaL zes!y6>6lA$KKos#x4lqHySf^F%P28Cu@6>0-Sz7YMb0a!;u( zV}1$N8RVK@dDy@!^snsYfbs) z!Rs*F!_^W!x4INRmX=k)Nq-&!nuGD&uk1Lz+|GBfdoFnCojXK+=?4Azx$?D`Dm#b) zWdRUNwrodAN$NdOE^VnQg@iDO#|mEbbeAuGc7)jPd*t-BbAtBMk~Z29u0Gy>d%@b; zdUX{(+XE>rl84Y2(igM|8{q;^Bw^5@$fz*}Mi2HIHnY;nXA8$5a~#0aFhEcqTwvhb zdiZ2pn$b&d@BY<@((s|aBzf9lh(k3KElIxS-38}mBe-tTpNAPNfFyX+xal(FIGeGb zf`%p0t^Gf`g=YRi9&IWIlsG*v=fw4g)86uWkb0c7l&o!7p;`FI&p^yXI8j~_q6Zs^@9*^rCk)05-h))W|+1bfWfJD-Y5zT|4I?hr!V!D}D_LKWg5DgYSW^yyf==Q9E8dqcE#lW>8^ap4Ba*spGFJ2 zs6r?R&^tHL7qZY})7Z<||Mk_ewged}#5-;o{R|52h~Axe217JbA2{aM(Od6L>z=%y z>?ji--y_IstrVi$v2}HlTXXHAB#$rA*VS|GnRaB#E-^j&^w_J+BNd*luA_su*9L+| zVWXEMsim=zqj$=p;u_>zegIcCy|5!v;flX#VPDJG5Fpr2r zlqI>8xB0AL-gjpE2x7MHK7alkXMpc=V)WQv@GdJWn@*ETLaF-M*v|~no%N~@q|vKd zxPSinhhFN+l`9hkNoNU%C}@K8&Ori<7x%jqZV>_)Jk7ZIqeHM@FdmObsI7dND8usT zQ!L6&Frmp3{xs;CRrvt0q-^{hI&+>fNWl+PKHPb=49gfERh#8u>}R%H(xc0Z5VX9v zVp~etSH>J5&hSTZz^^LViL;}y?-8gDxO$&RX^~QRvS-_&4+i-J;4{R85H5k-G`>91 zGD%E$ONxL!AAj`3iSi%v#(Sf%KJ3p50dWdhSv2U*+p>JJ<->v_RD?x!~Ta$p8qBOkBl*ncYuwQjGTR3_@u`+ zYq#8_BQtt)ZH@!3pjG?~@U!&M!-qfNcNup%itJ}JBDxPBrDVDc5O&KNV9q7gAbGo` zMp3%obpdzb;bqo(01M3Mev04OSj*>B%qrc8fXg6KfX#FP+%cp1t=DAJ1rTSI^FG+- zs%B>CUgXrN7QH3>+m%;W=2ql1>-LEm-Umfv_u=Emu|AZ{AK~U#Xv=3Uc5gc^iT;{P z6kefCPgzSLrx98YiA(a>#SK^hGJ1$0M#bhNYlcp_2Azao>$f+j^DTM>V9(6Zma75Y zLpfDV$BChLDTT}m{{NpBFIuL(qb60!c4m4e=PvW}Z_rk%i38+#3$!;85E8m9-ead2 z*3G2?-mN6jx25V#*M(uRnY5>e!}YE~!~~iw4^yF3<#l3iQy4-gf@;yC9y)iujw=A> zLQn=0oJ_g%inNVrJ|5rCCQ%Lf#*7yBQWBn6P9-1=Zz~-(^Dk9Mf0W(L&ZSo2>Lf*+ z?Jc!*!h4{U9IyWOd|d!th?&hJz|{<}`(bgA>uze5+RKjpj8=lBL|B;ighMxTrn@i=(CzAKYh}lBMIFX79f?4* zw*kv?nYLdy?=5N56L5Bx4r2Nc6hujaAVz*HyZYO=aR@%=3*M{1x}cX55fO3FD&N!* z$4iH@XP{7p@V5Id$w4JULj@%Uk*(=lTihve>%5g7lR6D4uOwk??ck4~;dSkjJj9;c zGBno;$c!c%=hpf?3U`mwPe6{r`sB5qTzN~y!2+3bb+xkb$vPu`Of?*Rpt`nJ@ZHC` z{>uIO4^o)C7%;H;pr5^Q}M<&{GOlWV&uFYUCKyi_XsBogvGP z%*pvu zq?w|eqPW^W2eAYK&HV^bj~%v3*DXdxwTVinIcKTWGs~8R$qX0+!=4x9n6-gmH5~;Y zhnuT2X<=buTw@f}0tS^HoGY}SfgITtjYd241<-ppfAFA#@C)0V2xHemy&zyS`VJ@y zS1*iU--6p|VEirWO2O>f8pS{^Z>_Ar&RA_5VpytZl^5B#2OA`xiVikodwYgvK8Pk< zZa~eNcPN&CP_Xjzs}I-s%GmTm3Yh7H1;PSLz8vr+c}v9Z&XOE~t==79YS2d~3`o{i zbgS1jW;xHKO&TS?MvtB0CXuJ#DLc&yxHM&9=I=u+juY7&r-NM6uIF58yC(Y9g9WfS z&;{8wtx31Vk?+Aa#=E}*YxoQ!=RfIBDg2yzQ{{SHAAk=5Z&EYa$5YY073{g z&3R!vl7;Yc(_N&hh^!AB@Lr`5Sr-FchkRFl5|G%=TIDW3924r*?Qcc1{*^c z-l_yaGsb-r8_c32ee>qct`fVno$V!g5|9*J!$~d($%sDt`1+#kMdLjF!FJAiXQr^f z6@c%S=x2!fh$yjKyLOGhV+|oe1ux>ZT)9haXHREmW|?5Kvx5BjKGrH5`h<+_yv?VEnwAZ=tkfvesp(h(z-1NK!SO1_@k(l zBsAd!3=*kS<(rdf0?y0Hvz5E<8S1&>yQDb}ZmR*YEh@4Uq#J&DamWN4a?}8nWne!ek%3*OEIF%UQyLNWt<{;9p0DqDEp;GY$pbA{eemENQtTjY zjzdrmG4Rs~g$)A%qaD&u0*Fj(>N&r4W%(QNqLqkp!9Ga=f(8V6$-G0+xVI!9h;brB zJ=I2`r3T?O7v4ViEpxFVIMz!%=6gYu`N`x^@1I77w^8A zLf?68a$&euhSSJA-fwXX5-RLFrF-{YBH;(Pr!lPC&FS1mF1@dRjSwO`c481KFl>y; zfNhJO9u2?XTv^;dWXHOwN`e}~1%YU)ygkdNoTVcyyg441s#^&W)C)~*tsk|_+Oc|= zrq0U0ggkc2Dt0za&3lRzfVJiKi@!Gk2^J`PrdygIrtaBwbwL_T&TG&W367f+Nxnc8 zE`U-X4p93YY>o?iBt_LoN=m8*c>TMkriWFvwY7(j9JvX3bkxdMFHF-Qp)eXpPX}D! z!we*{2@4NA>t7SAyFLW<{d}H`q#k{0Eibs{^2YuFSjfm>4N|Def|1y z97JQJmz|`fG-yp%Z&}}7YHT4?IIoz*L0tEOm9GG)la7Ujh2TpzGQ>I5tkQI(%-NZ|ZhRpon$|JgYRjhqmx-vL0V1|{+(DdP8U1AS0| z#RtE>+It5CSWb7+idx*gH>b$E)6aY9P5;WSN>vh22#1iSfGX^YF3oUwW3x^S&&?J#3FoLQOChStsT;b+|E{+~! zaBj{LaUmo?284wfSMu1&qyt>PfSw2f-`*W%XcpT45+K|-t`Ew;IIIs}AWVDDi2}ti zEIJ@rz{&D;J93TWfhS(>HRH9+4VfJH@?!6?Ktn-Pta!%Aw7Vdq8KhCon94}=_MM&n zo{T;H^OljHLZ!PK9TQXC{KCRbKm>Zrv%OoH1&?KV4QEvG6BfI2migx4^l^GsUf4`L zw`qt1C>>QD9UV-N;%@>{w)GJ8rxmrZo&ME)`1o-yB=_cFTWZ!xYyB+*tA(Na`Y?{- z`6g}e(A}eT1)H@wTVFyS@U=t36bvqshY|6M*rn2f)HJ`c!adp$Zc~dV16iM^s?PJN z+&&CVmO(uZYAHExCMJfVZ6b}8`5v6F*(%D1^*roP4 zPqD!+t$S*E6XTg?O#zjHx@U;TAxS5rurEj8D5n^dWBQ!NsD01f{AcKrSV)a96w@xt zNNbOihvWVICcAnK!kprF=!|C(G>@*x^WF<4fLpdfY?T4sUP$*0wltU%Xk z#^YU^+bA++B|CXE8Bz`sWgzI9_2Uj>(jmS!U$?8YSKSiYRfW4XI47Q1w0jBT+2F%G z%bsIBEPnsqJ*4S1K}O~VU4a|IsLR@H{+z^hDS1&+UX35Msp#t3TA`@CoSf=ZdxlE# z!z<>UIrkamV-Q{aJvQfdbqok3~;1sBMy*5{y6}Q&W_Z z61O5OD>lR-dN+e~p#16aZ7Jfr^S8JR>i&-Aw-JZw%JiW?k6ripU92WO505G?2*RrO zDr~nYct|D0jq2$9je#PpDlEg2WdRjNCrU;UBGWREOf1N^@Ld`a?x#+l4u+*JDhCN0 zJ{`=VD*{7hRt#0{k+{hD{yL`GB~ecuUUt`?;C zT!Dl4o?En87pN-%9zJv7g0%L4I8&2$IxIU4zzY?-J6nS8GAesQ1Y|Ci5?kyU z7G=rqLQ97tNYCOp<>ZGSZ2PMk8|B^I-LYLz>`D9l`LoN$VKfjq7zmb9L+xt3Uwsd< zKRM@xpf6;06X45;Bo3Q?jEiHCf>;qk4~mCosl!}5Xf`t)>Dr~Lz+6%vWq(w9ws(VC zFHYK0!Lv3&%2@8--BE+pn$v%ahLSQKLLw+bcQrmeey)*kk^o^B3;gZ^m+^b}6U(BK zDv1NlI1|cnll6@FR!z(D1i+-&Z?8}KynZd|5OfLM>U*b8gow92CS7$v03p$Gpg{)t z?1+WqG}i{{ zW~zrUJ8#zd0z=q?&3IC@__0vEK8+?vtR_~>TeG`FB~^Q?W?_^-EU_D1Vo^!2-!n4& z1m8g+k7)>(N5%4Jq@tLoa;ztpeLDX?F}zuA6X@EPhpi|Cdb)aw6-(_E(^QE$Pl-Xy zirG&TXM0AK9nd@KSj<3GRQ2|p`c_@d%)}&_ot=I3`0+e}pW3C*?t=ibImLmURA%P; zHF*!-E;bsXZu%nq9rLV}X{~?cv8s(t>8!G=)PmRSW?1az*jFa>?e>Cs9p3|MZ_O#< z{iDZjCFQS8Gr#hdG3}5Z`ze*7sTsXD*qr2a<5s5D*2F-0={IkHf*D2@7V<~*Wyj@r zqFcL6rx06}VK;F#n1drBUNRuSZ%jHk=h|uXJH7*Nzd4096#aeAwT$0ncFxl76#B$e z-MjqTsps!zkgznCA-A`Uf8K=wR87GZ?{1{3 zC@u_*o;B@vH??Cy-%#gBInK@r7+rY&^8tvFp@=jA&VV|@=fQ|HwJa8>6@mJo1m$8> z6kxt~t!6L+gxM6(2Nu`X)F3J7xNkhwhGJ+iZFPdHYkVzDplW8&>P#SuTFB-~pma!3 zjbFaNo`=NX(otKg&LZo$t5>fsLyZu@NNH(lMTlT_ATVxm10sUi`$2i;OafF6CxM7M zFE>lfj0J*fQetNc=u;M|G|d32mqEKX1$|r9l|J;?kpS^`@5!lgBHIr)yVowF2Ya8A z?NfRm?mBK@;4wn3CoXV31T_iRLo6_w<0PybU z+3QSF5jXX{P>uj)6I97HKaycV@CDO0ocnh3Bqe-@SbJcn$pQJ#BI9S@0I#A4*sX@k z)2}6GU|>Mtf7SGRC`T4(BHhRSmlB)b+qZ%A@L5P(^~`uSI`c02r~ibZFG>tNKe`B)lmOr5HEMe8=ymwl%{Sc|Ksha^=h9;q{A0ZI9gp z905R}4UkTBU&klCD$4Wcv7&I3N&vthy2mglX1i{t$euje_Z_dhIbUTUycSjHbC%oo z_iNdfM20jQquD07C}#HlOpln)51&Iw3M+@*gYVl_MCr==^ zUD@JBxz4e&Ri#fUYeNuh9Z)$ zFZN`b4oFo1>%aL?*&7b za84XmWjT7noWI4JwCQ2196Nix$zyj*7v|=Nd!ysShwq^{@F6z#^2$Vu1V8~KC+t6P zUN10j-ase+aRswPw3Q()%Gc=1?F3(zh~P#r;7nDpiCdstxRu%<0z*e$I$B0W*hNVwzHd=84F$ zYkpeW5%@082IkzhgE4@ssti_l4+jP&j2ko4-#bcPAaDYP;Jwq+)vn8|L+m;REHF@x zMe0$ZY^HV>(D2mpfnZfwVA_$L!)4wQMP76)p+`UwP%;DV8g$8tP{`0>vJ)*DtAK2}R4oUb+yR1Hn-VbjP zQ^gS`F=B>GeJ(^Ob`!1zvogx!E$ptM&#Uhv6RcE1@BpHQ$sm--Llp^O|h<)ibQuMt+*h*`c}jd^d2CZXBc+CBXLi5HD5W z61}E?j6Wg)27qE4C>`7&3{fS$6f|vrkg6sRIa;IWVT6DRT5o62Mbf{~9bE=#h6T#e zh2_@(TmPr{F$HjvUD~T*VBOkAk55;3B3Lb6zcxNoJ7KmrLBuUx4&B7%wE5S}vnkb1 zK;B}UCz-)!^*b~35Pgaa74!9msP?|xBuX0g3~knELB@Ow^FR4s{gnjec#5m5Yo0;< zsWkn|fp6Xh#<(g@_G$_^5zUrXc_do;T^hU0;QE*xa(|X4h7=eyP^Yre{hmLFm!V2= zb+Vx>*A=a`Nsv|#JOXbMYafQ;?Vm<&w_n-%E8s?85)#tdw{IW$*rDU+9xHGbr3KI> zJU>8Cu<`InYeAQ;OZ@|9tP+}As3+1PEHTASlt6QUsJ!SBw$P%h3<&NVzph9ad_L$ApJ$AQC}(Pv*D*s&neV;2qT z8|9rzcGz{R6je6t^dn|YMdyY4X{XVGtUD@{DJE|>)5VRPX}~O03(RP?^gnB_uj|{i zZW9@nbe45Jg74f-$Tdn3_xsDW)*nTp&?448Xic^W31}IZnlUIG$S71^4jC3n`>*N; zGPk$E8W-Ov5@^Q5ywYJna<&=dgAZcr4%6@!Zb8YA2w51`D;p3-7Z=ctkh$bo;1})k zt<2?yTe#3r;5X8iWmi_FnMWF!-FAxqAWMTYG-|PU;IUu6e8F~c=vGWW{B$4%czajz zQq98GSHxnQ(L4tZ|Hv0;WIn44WtKrf(hK}LdZKb{-^UulM?upvv(S82Q@!%3bV8n|4SK1&-bp!ls!A3_fl6%LzA$SfeBVOh4|1&S0hX{u!DXd@5<5^&qXfLB*xs7XhNtD9wBb&a%AV%Xan zFo^yc4YgPqMJTXvgB24k-}U*s*q@gsq8;K*+8PR)9@?Dw1DBa-9H=s&$Sj8!iB2TJ zc3L!SydEFtV2S;*AMK`W{e<5JHox~+94D^!%($0ce0NM(7*l`b*o_0c%nElja>b7x zem~g}dg^qB99m_U(-y**XXh%mGU;vFR-u;51e(IC&A&x(6_Dl9hOV6c)EOVDgVQJo z;xW(JODmq~HTp-F!6*ZoLz7WM*x-%m-i{762APmnkQvf_ydT6XGAvz@diq1Ul4NL3 z1K)&Ee^E<04)Gv>9;rLJ?i&eU&(ycRYH+sHzIRj0wQTAcdRtwWP@iDl+X@9X8Fq@T1gz}>X%gQ8;zmT|B1$ZeXv_+cx=pQcX412 z%mqoX1t$O=;lY`5Mamg;;nq|YR@oRK_4_`Q_45tr(wL5gqW$G?CzCzJxkstkuS2OB zl9x2Dz`WaVLuC9SkJ&>Y9WbMDs2GR5eig18^ipX3&-N70R2S28gSkRSM+Y2Mkw}Y~ zbph3c<>t+1_e)e%j8cejK+HcaO6XmXE}q4KNiP@e-r1KPbfvT~VY)6e+$xz`>fmHm z`{M^@lv-R}Z3n%B?MEAU-;h`iQc4?mR=Gf66W*S$Lgf~aq7nKgJ9J2}>>AJ%g!SZL z0)DI^>s!sc^MQ$OkR*XMHO@CK$Oqdfh_de5DoDV0i(LzN|NgPvU)#R0mvF6M-Q_X| zeIxI3{RyPFi&7jrrUZo@WsrivWpW$5HN|6V)I4TkSLx_xYQe{I^vDqfU{PTDUeDq* zddXa>l`G#F%d(+G^iR7a%%WX!T_x3Ek3AZwrsSFqiU{S1Rl08EAVCk}@C3{_`53t4 z9xJd&AOrc%_J-a3z-?|U@gA%tA_zp(nZ@81MP|PrTkTLATy0g~&%3k-$>qJqc9Im! zw3b-M$?@EAWot2cwkZa=QX}7`T9P!Q5i$!`P~GHxri$V?R1G2Zvog@rkHcC-gY+{- znAO9>0|u^C_W2H26k+OZz^eL*J_Rr`s(aZ_|M~!sT?&+esUR?qL1|s%Ja>zW{+N2w z+RZ$T9T)eR_6)_DE?z%g%aHbrTX?mUMUEE;AzNLfDI|2o-hP2p#&KBYV2Jc~-(HX) z0fRhBLD2!rKe~Mm@)#vGb;kI(u@n)^zvwj@7uaaX*S!K{b#lBpZgaMDt~FCzFqm0M z9Q+g#&K%9BxizMuFjn^>u3(HePln4ahc@@aoKBP$Vx`LL{d+0G;cLVIAPujJEJkt!=GYj`iC6V zG11)oXJ@5D7p2+|e}8`nJ9?HB@Nf{Wu0J@~qplUT{5w$b$ww}m#Lm$fbH7A~X=8I9 zbo1wH>BO4^ZO3DwNG^?ot=%%sNM4l7t{uNPLQq(CCtYd`W_c=mZSTE}fJDKQG!6he zi2l6Sl}Li2_0@q{-%HF;h*Y%ATb|BUO+t%t;5yCCYEq{WUZvE22Esl3i^N<_oye%D z>8}Q#d+UQ`z~S8OKKM@%qyN+rx_X@>jY4cCv`|cT<)?wEB^d&t2}HW@(Sn6-*`B3~ zE%IP6CcZg6sxuBap6qDYSOiyq`%Y(|2QrUe zpeK!qDrRVm?u>z~4@~+D8_RjGf58u>KXa3kFO6{5^O*0$O#DC^zYO=)bMMd;JPwz!Y3!tt8 za;hZeS@u_GLo%^*yF5S+!p{ZX5b0ptRl0;UHOr_p?d}q-y=#ZS0))z-8idX~pe0hs zb)f^%9Tjz9Z{+Y%zi(e^l%W=sR!|^z=8g_oEi8-{N50+bn+lJt%^0e9Qhl@>(k1Fk z*ptAaBH0Eu@(dZ&F!_NqKk3fhyDP}xfNXK#s>+9l z$as|fb@H|6i~Dbhmj3U!*6tz>^q$+X=kO*}VbYKw10P;309)la6b75Qd4jzzE-u}o zoxt||`hza`{NY=`7RK4T`lQ+i3vvm&+<~6XOzlDuqiW&tsSX*Avdawgl5yfv<<7+- zd%O3Hlh*SX9PTPdR^I7Dkam|Lx~DoW44sYNZ;@f5l42lLe}fn!={*hCVJpYfv5E(-VWd{!sN&eD*Adv<=qXONr$NWPj$d1K0I+zDg93J}WsscrF6YY~+d9gz@2qQaS2TL8i7U zw=r?&yFs4ST?N79v~s&zQ|&+JJO~$<1vJ3@EMWb$$J=ey4BN%EBoCLuoUhVPEb>@s zQ$sq~1#T01Dez;IgH1F;g2w3{SOW6t>kV_&!~T}+&%+jq@alUlZLKW@*7%3o&xJUz z1_{`{7uoo>K|;VIrbb`9sS%V)C{6wHhn|ouWD@{GG;#|8p1S_vP*GM8C3Zxb`C3hn zzc90uNvC30W&flBd|0e~=qBAb$Y&kUJ926ERwHP*U>1o3(Jt+-H~HDb13hYEKS!@x zA2~t{)s1-jE#H#k#3f_)$GpyY)%#$XdS3quUk}cROlaXs#5RWCNH9S5CuBIXDBW^g`$A+EDJ1+zwV zfn{aJ)GtA~|DMtK@!5Dgi#8Tp<7VpPeElkfTO-Iq#Ji_}cz zHZ|N+jV@0uhV7@eQu{BR#zQ{qM+n=Dy#*D!WZ_1|F?jmV!t-HJ)#-^GH@3N1G})FQ z?9%Y0+*l#S(Hh&q{wE8lA0iX*><0Qek>3%&??Ak21}Fi9t>%6=wF^SO_JF!r`|aoC z0|i4v4d!j2pdG*KyeO4h?&dPFj6JpoBGiwgYjD21*m#k*Vuc^R8MEsraduo!Pfq~U zkccr1k>yd-rTd}rPD>Vfcp>JejTU>k;WF!Dok|Z>Ra4{R!CFl?yD9{$Q%@->%3 zrg91{VFBLYdHuzH>d)pYvur2MUxkRFzeS}RHdbI+c#R5v`@YTb_f>1mY+v{}bD z$zKXSzIINCvR(f1QS6YF=Ejl?-kl{OzQZY1)V`$@E*c4NRdci=!C?QZ}=h62_ zwbX}STz?Z>;*f$bRx@Ac(geK%Sx^9scOt)>!TB_Pt|AcZ+EhM)4P-+{DL?W2`{f4D zE1-MH7Jr~9e?J-V?8qO6CW{yvaKSpb^WN_KsrIpK}~yQRG;k~}i)OvlzlO+#8MEBO<&Ip96Xfr>}iUjY+B&n$ku|?0~QWV;v8yF0%F0}YvZM7 z3>KveH=n$O>oH1gf)5f|F5DmyA$0ivMX{|rjrM7Q$>@JB@SU%MZ-%I>L4Cy+jMC$+ zDN10fPbp`Ex+ya8gP#oX8*l-uK*}hf8pE~bVU?VmoUmYXhI;+olDX)xFjRWjvYw>e zfi^~{Y4J~-?iJ1onqr?Av&XlcIiEheYmfet8~sU0>@CfjA~G;vzaD8nTb9}MaAH~= z>St!oJ5C+H0jJcN^n9T4qyuVR2~ey%#8H$=4F8`aL2;xt|dR!O75b(>8(e-yEB~dS2X=K9(bgS3{v~{ z5dc?e%yb*Yls-%Npqkkeb{iEwgk4~fPr5I`Iwod1RZ8uVO1fpBEyM{AUMj? zr{jRgc6XzCS-y$N7B_4>dq-r!wcFWt+L~$s&4p1^w32HEF}j6?de7xuIiFBdcQVU0 zb?tyJlRK87@YmdLC=;&0%q;P<$GJrgpKxqwV zb|^hqd*q<9nOd*dUiX#8^1=WMpdtkJA+Qq8XydpyKjfLTdVvBS#Q>rJn#*{*Q^^24aC_lB5a zyh(uI^NSYs_rp-+TlR~frU}$p2vd1h~b3rTeXvp zn4X63!Kl8xG&(xD5XdNVh$5nqian1H;kFz~VA3Q*sNXe#RbvHA>sZj5aG*&1NQ2(b zvy2CXIxY$}3Y7bQXj|-GGTmAY?#c@|cmn4`6>}(IU^FtBal0!mgO4+&_k0HlFz4y;m-SY8y~fMtb^WdAtakGE_r> z<{>HKF^;lW-af~wAzMt3tT*{yMFgoa_A~KMf5bD$#Q$4xLsvMu3@*?KL@yz|1d$T? zQ=#yII@Z`89Q;fjEy4zhiEI=TX*II*>^Y|LU6DNs$Qe>gnAw3E8shM<6l9=3u}>#O zkpcNK_r^S0K(QqeWE>~j&Hl!{5;YK%C`gR;t zBs_090d#_iK(_tW;nt6?=!^tt6Hv*<6lF^X$G`k*A}uR^*GXY+pgEhKxp1uV!)!0r5V`13Ap5o##@#`jZ-KRXcKao`#e@!-?LX>Mx z0)viWkFl;{lAxTFm->OIt2=S1JsJ7hik%JKKj!(3L|&aUcHH33QlMszCKCUnxV-X zE#jtuLuMCaK%cN^1}wm}L)s#YiS96w{nf)aHe)V(tzJt-M5M1|2S)S#0!06QaP6QQ zHSgvDr~5rHzvJi;kG=rs9#l_YiXyFt8%{1SjwDFX1k}8{*)1Q9o9rmO)&i+yaFBR_ zLJ|}_S#;Kpfe_?zxSLb}sHUJHEUv8N7m7ls#U|*nd+Q+IjG0=9iGb&Tb{6(i(0OTa zj+m~VS#S0G^{};aP_bigj4<&Zvdnt88fFaQlp9nw5blUr1+Ne49Qo%SrbpeM2DGHn z*DLa(;bfalABd#2$nMPIJO7XU*)KGLI0?u-XT|}~w4)xWyt$rYC1{7v-wOjjI25fu zZMZg^1AP&@We+gtVoPtdIq-1@ixejtW`VBj?G!Xzcn+=ch~sc6D-nOq>T)oTT;Mc# z4PJgw`H*N0XPexHh5=PzgXw;UJ`*514LjZWM>#)rLx@XRTFzeEufh!)3FB|SfaX&I z$Dte}2W@CyMc2D{8=BlYkgogcF}NxL;tv>tAg$x70NK){tF?@( z_eMPQg)WXc$a<|ig4Yha@eFo$iX8Q=juN}f3B-vqAS&>Q&OCvVgUM)!p0d$0bt;se* z!Dq1v(pmzWrdB8tkbaIpGR7@~ewzw$9Mydxie*CdGlk6x8rne3HYwrLjT8;;jQ4P#@CGJ+1$z}>2S zibZ~GuD})zOY8E#z_2novWZrMdO)qLwLBD^PzP3?x3~8coZfMke!pk;*6-6c)$34o z9a;SzoS;|Lz}A13!`^*2JKL|eB`I?2^IRXBY{c(vv`cpR}qSEjpA26l4!PRY`iWw$Ey7 zzzmvtDcW05J4OK&hqg8J<2*nR%D^C=RN~O*RGa!5^LT8AEtpA9q%m4kI{Gi=#COGH zQMnoFbiNmj{MzLs4c(bJFJ)Zye5{`T8{hd`pDzFcV;9AosQwUooob++_`k^<$U`Z*wE zC&8?^LGD$Rz<7EX04{mU4els#D}6gW@$JY&D>JtJ`q&yCcNOrooVjkkH1wFohp;82 zC?zb8ME8rLOU%Q>&J7I>LG!%;lEgUkM7x_J%ix;-aBNuKj>#amnUe4uQ(uriUq(x}XP7 zd^tqjD7;g&v;~P~o{|RI>%nmkI(0BTU`S~sE5lJw$V!RMnmC(uz%%doaLv+8H(`bV z=&Mc=+b1-(Mtg=KL&w}A1$tPQwqP60D+B8Be)-ZE(i5uJp+kotI7;E5p$?tYq!~qy zA(l9|Y2EIo!NcboA z&koZSKQaH&SDtX?F8Eb#%yx&4E%mI<{A{$!Z4?cy@e9t>4Rv0A??vHYjsoz{A?vr_ zq!b+;i(pO1LK{j~HQ!^37DuTiPvO>=8@DQdxpxb>g*f09+6X=t92 zf8OKgdoyBn#{W8a&>c8D+yQDO1O;6*XJ#JV58hY)%H7w8v5UVilKwB)$XB_mDfJ*| zXgFgEg0+D#9MR_49l=`x4Lzw{6_X1+0qnGP76?dUt(?V?WAvLgPRSr)e~Y4a}0 z7YD=L{Lj-tZUXXu$&yhr+R*YGBu^V!ApjHSkh{);mF4?(uGqd@J{o>S7Htiao?lvo z-p--ngo&1fkskz=;i=XDNJi}$Htzi?J=PJzO4Iu=bjrkm_14?m3@v6txV~I2UZ~UcL9~fs&XWjX*3JV^dN1c!4V)dmSMsIwJLCks5QVRx4fQR_;u$;W$y?Cpak5nCfcx2{Y1yvI|V z?Q+>}Japc5!~Qr6!R7mR?COK;LzvIEpqx-)JE3Sd-t@)23_>X~lJ@oW;ZQpsI+;tg z`T(ZTZY4F(rBMBNmzkk;%y&Z@_g%Y5N!iE?)|=Ez=VI zAy8f0AW7T{Bi$31*3scj{MBvNo@oGa8-Q@niya_>$W(@0%JeDV1#({PGy*#x3zVTB zWR)%Y&^Qk4d#kkHa|4>|hIOYa*&Y<`TlL$VM8 zwOLt@5hmcP!8VU$fWqs~xqfQ390(xi8=*1~3a%9>FQma1bsl_0g=)z_yyU>Bmr9~p zhSOOHlVO+vdp{c4%HG=A>kqX*C?+g-H%8kKbf#x-5c(#AVeUPboEi?7(4!7tu=aL< z!4`>R!VBL{cL9Hbh7@E+NfUd%Usd8S&j(fg{R*JW`Mq2A%7L!X+T8R`6xOefhO@@B1w4Mq0^e)GnWlIDX4e(q%O)_u%&s zJ6Sxhjd$?0Wev+BIP1M@-^J2G#``y1mpII2f`TXyqQ-l$ry-3Gb)l+Cw3=p>UyCG+ z&cmrUWu$GW9G-!K9`w5)P0@KeyB5S4eZ40bs|s@J1~x+C1F0>mk;4`V(u#Nf0W;62 z0{iDlZl0c_jpw1!a?@#;m z=@VB6bmQJSOK*eD=SYLTFa$A?as?ta!mem}!YOh{;N03fGXYAf`5-xKo*jpy(xe|e z$fK>Tu9kwg;DUmJe)mw=P?afWmd4||-Bi^K{+0eb)R6v%L_iM*UdK-xV}u2x##LgA zeA0NZ+NJL3k;M_SRQ07294;Pw3N?N!A%u{0u>OHT9y$SvJ-ne$pQ#~Z({#GCMJvo= z{;}n&m**}B3shJ5l(BMVJh~zqHjwb@m^qvyW#D_Bu6#A35Xc~fh@^rdZgBf=%=>DC zc8`k&Pid55JaP%3nWu$-Ug&Km4)2s zu5NC@f({nm9h4x-fqkH4#05^+LH%)3Efo;>Hnz5mmL=9}zZWOtgFQ4N|06JJ2mj|) zZ|~|1*655Z-W`s*n4*#*zM$ya+M^PLam>N$=J`w(8;EuU2^GR zFpH6PNSrEL%EJ#@FpW7T?LsMX(SFjLbIrQN#HTl*_9ZDP`AJ~^PL1KYgSz~sDIig% zLQ^d|gDZ$mf*ek*?AGJb+B=sDxd)+aZ&)%H(0AAwnjXNYlqP1>&*emwI4v-Sb3dq3 zydqwoS@+m(x9|u2mHB<|)xZR){|#Q3r!DnqUB{X@sNy_r>DW5M!1i>hpRq3s3!L+1 z%blHAbwwt&ov_Ux4Y-XTj{|mXIA`fL|J>%5a%zyQ+g76StQH3tID1QzlGU;(LuDN- zJBgYrQx_rk2{^YVk{!dHp4}j^iWdZLP1@)t<2El3930$QqatNn=H7I+D~Y)Be12bl!DX z9P}ne@$<-rKhy7DYI=yRI@rCdJGedg8))u}LkV}i8}3i-KadOE4K@KK{5DbeD@Jl) zFfCgFFHi^@@xQhA-SJqrf8S?`x*D{VLW_#*QJF17Q7L3MWM+lzQ7R2GOPOUPyX;L# zA!LMXA+qT-vY+>%>-s&v`+4ry{d(?yoAHmxiOQ0ii%uJS>G!x-6=RCJva zxY?|)0e6A>F{1R2a7i)=m>%*ZX4*%nZu zl{lk~R4n#@lOE}>bpwfHi~HuamxgZT*(OtGLivsd2$vu5`s88Z(^mi@XW zu{V?d9`fVxdX>S0CJ_--4bFG09R^&D(51sYB^1H27R2Q4(Rw5$Q&B9=v*tK~6$m8( zKs9m*#2^dsi9oBk`19veq+kKu&!?ux z&Rb+HOpjxjQ2px%h(Q&D3za)9#-=?b%XrwdoHZx7>Hx$}GqEyR@bTjcv^7E@uloGc zX>-$>R~{j@-CGBnGX=^saR>ScEr+Oe9p$IEWOM#ImcTartktU}G$!Ij5>}x$dKijX zc&EMIM`mQQ8_@U`vmTG3&g5?xI=Y+!yt=^cc~Xyp?Ry^e3W3R@DJDx0oKVSB+MaqcSzaS`}3TYH6ogbJS_P3ij?YwCUtb;JA;LyvBDJjO$>gDCn zzLRc^8Lg*k(HtHr5oNh@K2Dr3xE+UTBq4Q1VcaWplzsrgi*L%r2Td7Oiu0&|7 z(^OFo0|pzQds%$V`GaDaG^q#7r=S5)^#ydn1c5f8rK5Xwm=W^M=A9qWeU{(6hlWy+ z8=0`HE!IEtWK(%;aj3*#!0Mq9ws&o8>`Ax9C_Ap&mTjB_sJ0)2b4VcE$m$`G6nrCF zcq65Ni??WgB2G(6E6FJ~Gk<2*0*sH|X{%@%28(O<6v{l8=hmacP3ehtNJz0U^^tvI zL~;(2C%$!i4e`)8V&Dul$5RzI|K<{o(Q(d)YP3oboXzO|+C-je7@VWFu z4paxUYcc$$@>#`LP9>$7y(DG14IPL&=kMOTr|5@bkv8MksoVebVasYeKe?N?Y+NkF z6Ep>6#+Fc?hoTB|+Dmkf&z}*$W`!PK`@Mf^-^fghND(U(fDu;xai7PgdO(B&jH0?f zQ62qEFY|qs3vHMS3JrG$HliM%*It~MRKIgo{1O88GNCRn)?f)G1;?x?UgPAsA?cq^ z>tSG(7zo2ac1QAIM>VFy)f#+x|6Z8Z>s)#%LMficIsrB^7wiG|rhy`#tR(c63d*6{ z7a%M&JY`bDK6qH=U%$x%jP04LX3_3bYg#d$Y;-);Qs-_YOa4{v^Yp zUYE6fDB~W??fBpo_M@^)3N2aBXvKhxRF z8}>g+tKVEE$9P9kLPxZ)Y5Mz z0rV#UY3P;*M1s?y<_Ds92IQ2=Xd4xQPDEO=b0>xvM*I%{kyeBfuD}*EQ~wAcQu+{4 zYl*%h)aQO;woG*6Ny>%F3e%^5P=25KwqViveoOrW_455*$PjthA_CH6F-ihW$Bwhzd_e0GSGao> zkgP7wVT+hhi$}@%*|vSz0)V!Ok9mDfPrQ(yH`W*1U3?fc_eJ}G4A=E+5#@oA&`1nf z^_RKvVBIx@W!UaXvg z_{^HcwTs5KIJl6#3ReNuAV4nQuqc$ciH{Y@z@q4<&Q8v%S!>{8yxtN_J(f5K-H>om zXlNpp0LHSan7A*#CSKLo7Vk@dRYVjIl`w}^PR5>ItcqV$Y7Kz5f>7g}Nd>YwS&0vq zO`))S#2rCSs?NOlzb=B98nR+KSc_$YuakwHBtyYh0W)4Y3)**(;^z7S>Qy!`Sbi@1 z1>*KBu5ivuq~CpJLFm#UGa|q$P}tq2qRHP zhK?>cqaRic!hs$M&Uk4zEO64F&qa%eLoMZX?K$#o3QH+w+C3${pDueJD&*bB%E|*4 ziX+--n@OI5*WyPrIqLNDj`v>! zMyvXI>D4FR4us|4`5OEG8 zDuqB|lbwBYs5zS40q{FGAeKZ%3RVOFzmdG{JSSb7xUkG5F zftP5#eTV!q9}(u9Jb3t?l$AnzbT?gS~kRY3P5^Ke2ApjjNMNfBf|hf>lI8KSbx z;#<7n>}!x6eW83QY7%=2@a7(t^z-g#3%7V_23%o6NGk zUda_=;-vA!n(~iuaiRQyHD>G`&Y?agVpha#!sqbrOK5t5oaHHK`Sa|fMS?tK z=m{XUL_iJN?a_s93}nU*p5L8Ur=N!bGYMV5o64WEBQdQtz`Otgd(U#qQF#zx0)TEc zj`CNogrNjkWLfz{;Nm$+Pv+L}`_zn80JAk84vhQ<%+{v#=U-yE&ep|f0%RYjJ2Tv- z$xj(M05rFkzLESkf1ogN1V289g2 zR^k`APIqqyY4$uB>>YwGEGR}qIv(@XYvc*HE3__*=bV~;@H(iTYsOlME_Nng6aV@1 zuk&;&kX1Tky~^++0VEsxX`I3n775FMl+#DB1I&SoW!QmMY7 zq1pix(CXme<*nR*cnM@xI=AU(M1-PyntAP?X30No6YJ7;IiS_ZU9!=*#lplC11=Wq zJl?-_#E8-)pMeajp~f~23>E2r`$V9Cb3}>S2Th68@5mfp38$zz#qOrKe8RRA2SN># z1(2c|?1fdxlj-bdeI@Lp1VB`IY04I!mSKhnSCGK74N=sQX3nMM)J9n1hHDtjyR+gET zLErX^-%VT(zs%#42NQR1B3LB0t@l7D#ViQS2xMTukC0gkO8#gtAblbLc-#kz6!2?P z4gs<4V*lX27if|X-Tjcb7z~CwvNv3ZhK|tlx!{ttTNqhL%jhD{O2^6uo_}4Dc z7(`u1JRcAUqM&m{RJ?p3ykn|cVqAwkI3NFnGqO1olJ?MlKrY?eev~@({%%6a+HL(d zVvboU#(|i32pTBX4YRrc@4Ivk9s?K9cD005{EuSq!q(Q8$Cr}_w-{y3v>j&v!Zpn^ zkhsU--|D2!p;Rg~mu}4II9lX!72N%v&~i7DYf zq)Q#1c_C0w^ZbLIR#4fm@RoZ*tITIz{kCgrIrYSd*!)`lT0R^D~&nB$Rmh zTxng!x?y*`8gR5Efti`KmtIizY^O0>Lx3T$Lm&+jh$x)HS)_!JN}Ac67`&_fj8Rg+ zOY?i9CV3?v`Qb#gd|*nC#!Cym{rieFgyuf;l<#jHY&u1=9E(*hTt;tjKAz>>@^&WN zT1HgC`Ln+egw-MtB+2#m)kJw>nGU_tCvddWe}*gPqoE(ne!MSc)JJvgDJY}>u#})G z+uApOUa`#EUR#^`WnuQy_IIHoN*vm)r~q-*oRw_*5#Q`O9_%F@0E(Bvg%};sWrVm~_?*Op(q`!_`c$>A8$X@jh64ffz3S<*ZJ@=YlukrjVjeaN!-fWpG> z!UHR{oi$5CTEtYc(^9NrsRDo6KHghDh;s7t_(XXs?%>w&%>l5F*(=FPI6i}cdq-rS+1R|fSyRc zB&CjQEHGF5s={RgCWuZq+V&|xD`CfH-zXZj`E5oG`57ov9XwdX7+J67OSjrfE!8LM zrqJ*wr%6qNk@mPYXFFW);zT$X!#&8R`=a5|S(#@`)x~hBEsUfZL$SwbEq;tVEC(`vArz=KC}uP3yTXMT8Q ziyGHLzVimk!b2cf&~C2v+IgH{`WJyGq~ixQPC1Mcs<3qj&A9LhqSRCN88_wFD(Xah zjeKOpzvZJxkH@yQX$|E;=Z+8Ch!lk4#b*Eo0R`6mbG|yV;tQo<3aJ71Y!4ea$7kvx ze_bQKR~9%~;5oU=F#u|RjIMPWhM$i+HgjPzNV_T^{M)Tt-TpMu3L!af!yJ@{7P@(s zq?!6V&8z;J5YFF|dDxRPUYjR(I^Io+)KOcSxtVj-GWM@T())Ks_KTtnXl4h#HWuUEN4Qw!C>zW5JacPB@SsU3LeU= zy|=gWUS63|8XAG_c_ZuLh0;yYnxfQo+c@W@q8eDX870uRuLPz~+rhl^TYdfZl7%Tf z3ujzO2wGT(__Zhvm>VUgx;CJJ!2z9T$jiDTU*VENuI-f>!R@2VrsCP5-mp*=4zU1m zo4dm=@>wivt5;D=`XIEmE{-ofu%9s8juF9+iR;uy4;zv-eU}vXiFLD#0U||gz2FYz zv4;o&J+N}E1hznnn6&R6P-j=6TO-AT}*h9 z5zkD!W%+-iT`GxoQXLS-priMTQGO!rdDEChR_?T4f~N0{A2%G^Tm)>&R@e~@t*+vA zvPb~x$3UiOnmp}ATeM-9qhcQTjTHzXyL?~)C5 z=DzE8ib3pBD+7ZLEyiu>AY0>TCh9wapbB747V9)OJwZjYQV{`_>onoUXC(gb@STN) zx5C-4_gOS$irYuft_o_CY+uPDP~_O0VNMVX#Al3{ZK09ii=%IDmIySX#s25knQ0?f zIE_e1C?@MB3`_1hhIX}uVZOir;rO}aFJ+GHG)px?mz&WAUcQ)n^sW(%OgJ#i#>n8N zJ|xF;0Ltk-o1!O0WCc)2Vqgo!s`$sGBx+7Y=`QssqQtj+t|v<23AeWJqe;a`r1SZ2 zsRQ}WzS-^Ev9;&7Ww4@UcxKbajeaKfsNXb$P>52B#lphM#4Fj*xweQ+MePpFk-@r$ zjMh%~{?%`rU&JX}JLS8?+LRF)RfM3HKZqyxyLV-m)1Sm%;K}Wae$R_gM9_W5NW0 zUK+DFNARJYz|0Eiw>Qr(QTb!hXeRTw;0blc^S&2!7Y)CfU0-Nhh=>Z|XMFQNKLa4B z5IpI;X%t-%41kJC;w!ESw%%LI%kFmmk^Uz|JIJ5Ltp_&k_nFW*d7nDNqp8Lus%$r* z3em*tqK<0Q?KJf*Gz>)63-{4Q^q+=%E@9d&ly6r4^9jO4 z0^J@i_BsOYW9-p8Vjn_W4pRhFHBhq9w<1}F&J}96thjwvUK_QiT$0*t<=CeBSL&^q z(IoOdX#9bbYcvhP>*)@MIXl9jy0E{u(=)#Ob{yD|_>BT_^B)?`$K4`6dX(Qj#9yr; z$gMQkSTQUqTsQg$&j0|7tS`2h7|q{LVg1(Lth^9sEy9_6=u;8$1bQ29bDPSGvXaqK z`uEJ#(~~;mruX;VJ`>_lZRHp&VLWt~g~fz>OMOGg&8A7|B15} z0Sf~SSb8D1l$w#mP%?5kRknM&F&vb6Ysw;P%`Pbk2{sEkd#qJA`ff(rBhtjg1|PU7 z_J3i3>z$JIhT`ItD`nedWfea^t)8E&7~U4*YFsC_T75tMWQTy<9EFf+m8=3$ix05*P_d{uefhdF~^pp z)&fK>8knE~PqaX0GBHT#N0mkz9x#qzgCv&{5t9ejnIMwz-21Nv3T50b=e<~6py_u& zRgshjkiL1y$ddp^6@$tbN5N4I`dJC<#-gGkupRP}@^t~xgFAqmC75I&Y40unclDeo zFR!)42Iy9Ac@n_O=z2P95>lJ1-aJ>t8(=$W6I%Eva_h!dqzb z5=Q`Nu70I6xxIhLLkC_%z|qO7NwPqJdWYPo@&lYc%Qun z#XQ15J6YEQ*wx?>vZ7t$dcH_*EB&Mw{;S7-{rZ6Yrmd-JnTo$gz3Ex5M?B#?c#(OO z_0%7r$Gvryd$+-m3)I^3Caq-fh=_8u{0uoVaTt@j1}#l;ltPO+2IYHWVgtAxq_v1% zeGEZhwL%CP@7mrLdv152({yjvp5ExFobf7|gVJS%g~@0f^(#wd>$$(kW|giC9>CBX zYc1ErULF=9)y9B8y&^LNXm60&8?gw&OmKUGtT;Dml?;FXk7T{fM`!R}low}r_Lgp9 zrKOW2ByH{%7dN>@9yCV0faqhrVMNX^){dMbxQ?L8tv_ z3wj)&`^BKY+6WWA%{*Fhgig!PU)?c=*K_H4E?nNcE!W08)^|tjrDZMhzo(`EXpth^ zUO#X^!48D(YVph8^eb0Zqbo?nFaW|Ls{yF|D-7hW7A%eHzU%&RmpKbDd4hMxfn{WA z%@T6D_*6vg{vl?B`^&I#!dJ6vnMP2d_@8+4|NV<8|F4=d5A7nnKQz&Zi6`dNQTFAC zo?ii6M-I)2G6%~w37cxjtQ-S%!UlvyY&48nr@_?-W?htlYqx~9@eSw{0v;gyMSM~t z>%Y(uo-iWKcNFZgn5u(nT32w3rZ(MF8*WjJN~~n_P}=th7xJ!-J#OQJ4%tOCB)$^W z+Q$5;?ro`{jU!T>Mbu)G70@lKM2&0NUA7f_#}Gb@YHzRHT_iyCqsJaYramZst!teD z$@-UCW5Mbf4e~DhjD|FmzL5OnB{UTv;wL&Zd@x~p*%M-~fMezyW*aG1{mO)J_}z+_ zIO0l8r}F*q|?KagEmlSSoSC-=S~ITH5J+estD}kxY87F0>Ayy_xf9 zedOP^n2nz26%`lHjnDpPK+;Rnm?S=_q~Gi`o55CN7B|$!%nw-Dkk`AK8Ed!IA+7nM zIwNEAEjwMKCWq;_E;i255MF>8>)&GjagT2=$lRLe$${4z$hGyxiTjmezG~*G;22DS zXRccs5K%LkUnS2s!6lBDm$#+LLKg8H&F9eLP4UX!UrwWX;hGD5QTum4{=d8eFw)yD zbv5UneX+2LfRbQP!{0?*&*I^pNF3MT#qe|w#JB#D%AX6s%@fHnLD&U%@a8=oa>FSX z-Hhurx-cvwD+vqN5AqObXTt~w#P3r+q*RLq+>Dqbj}uKx`1g>mGGws_cDEB(W_a<& zkuK8AX(yTR1hJT~P1>Fj;sVS3%=OK>~9@H~btrd>Zq%aUJlzMmH(ydD`&)|Fc`g zs>(3)kJ#?B#qx>b@?T5K7yISS{yIe*DKPf*E!RB{3$h<`q*qR;O=v1a;ChL!A{;i& zRTq$ooX7e61h`Bhih+AvgHlrw~YmE(+@hUj#odq=x$85y>tE#^i% zAoIHccp*ItOuopg5R)q$!*!_b`S%nC&(i}k#}QcmbO@F(rEFX06Lal7ACFcnQ07C5 zS7$yu;qzbxJNM2oD^-6f1r-fgLVt)nwqHu_QA(*vmb#(Rm$z!$zH)2U`lR(M{t&z~ zB)x8D>N$?8N{{<*xV=|ASeAK`Beadx>G`I2VzMDy!)@~`N)DHl34EEMI`qt(4jC3Z zNV%A9!VKL#O0E`zhqUYa_s5iyo^&y8+}Mszw#<D>+dkKV3z{2p9+RSbK%mcb`6YbL*GJ;er2-cw<6n(NiK_9hFbZYF zO}_o;(JCCm_jVq?*NzTxx>~ENNn4%=3>o3bwNgPr;V0l|#QrvU7Rv8e&<%u>%H8B- z0n8M=o*5N7bBD{tLc7@+reQpwvJBDU-Izbtsw1@&oE{uj?su`b+w|B;R|hU`pVvD=!Uo> zK+>~1jNEGJf}-SBeBFQas1O+}pWT=~US+qCP%Zluu-_+6pI(itf2;R}<^GD(>>GxV zc@NKif`oe!utV+elSy^=@mY`fc?oLA@MnF8&ZLLC`)czDz!nfl zHYg`(Jiv0gg1jl_>!n_CSUm~dxN#$9?toU#mV|_a6#yd1t|`b(Lw#`0IMc}28G~q~ zfwW-%9Op_?o41b-Dhk!RYFRK%Vs*K|T&PSPG? z4_fi^=qKx@1^~BuAE(5TJ!}hP-@cAS}omgNo@)o_Zh^VMeFn&+y z=*OtRK_CSi!4Fl#~LPDG1B(jBHX9WuI_z>+M@4ni1FgB+K<<5yr+Xp#Xo=k+ z>miMS#?(~+JrrVOX^zj1Ii}hZhe)9dQzL~8FRSz~d{*|+*3ek(Te1v0028q75)NCC z&Tf%#IK;up86U#Xye$aqvjQFd@7T?UZVqh0ocMct!FNX09Vc%7#B7frnkML0t%LK) z1E@qG4@MfVklKr5kcYBINND^)Pi96&F{8G_xj{Y#28JojSv8^A4aI@k2KJ0H;SBtb!C&<)}l}a5P)8gOOXqrb`TL7qTWAwzYSO>_lgw^y$goPbTo#pWCr@>UTqq|#|FF8+$@b1=c-mE+v zl+`VHvc&GCbZh5yL1!P@ph}=?EOQxXO8>$>MY^~-aM7#PszGt43Y(Q$ssKO#G8b}s zi_Jj}w&)V?zUel27&2_wyqO(4BT02W!@RQ##Yd8At1FAf@H{niVFl$d{KRYnoag$r z!>aW6h>EJG&dS8Q-MxD^tk&=v9!vKp>;+mEIL5{{(mkDLjd=A+4uU@Lkd&dEMDI`j z*W^IMZhCrp#S0g7oG>9dE6C(A(2`x>H??aQQF8`WRHo|J`W8>^?_Qet2=DQn;quz*N@$jd=+&`H?%THl>#Mk;;@*8Q7ikVzbgw?m?psR*O`L{O zJF!vp(4iN|8_5>DP?CP3&^19T_aVW_KH073*QKJY+yRf?qreyNp$lwdZy%0=YFxxG zE3aSk#tr`FmX<2G-%tesU%ch&>KazV)0fS{BI&e!)22<>mG`IKV&}cn7u*H6#VWC{ zzYoA@{2c_E-hg`n^RTadspIAO7CrQWOlz19np}5^gHEEEnc3MCldlIQd3YG$bS@4F zOSq!lLFPf+AZy2coZf|u9b9&T0W8L+KP^S3P&Af#{^;p>1hWmLi5Ia?tUEuh;d_31 z=@Hycsl@uvunV~UlxJ5G-jr-M%KEiy&-Uc@$Qe#!p&dY*`$=i3+&XW&!J~=t+wR}@ zd-_xkC{rDXc6tWs>FT~hFjobeKW0j9rWsC@Y(NEENlP_+J;-8dl=2QS^n<=&DN!#i zXMEed{OdQ@goszzkHv^mY+PLOXbR#_&e}~5aRGH?k(>0SyX_1%kTTgoP~IdM3!kS? z*&i?v! zgOACBAAE55!+1e-rx-^h59Nane?!E*<&IG4fCz!WelD(5S*O|%RXsgDRb=(pEv3ll zq{6FC0w>aE%V77-E*;wN1SsWUu!hdn0p0qTACiXR&6uzKw9I0n0!o;k1DglWa+O%R zb>4u_oFr$Y>Aq&y1ZgKNtq|NNgs;aN85uuAQkD%s21+r}x(*2q3OYdfP*`{z6aO8M z()op{{6mE(UkF`XyiQ~I?Z9yLj-6F7}i?}SP4sD+J(vQD?u|;hkiwd`R+3 z8feh4{-dR1U~$JsgF;g)hY&1K)}r1jfZ`G0qMThtG01R1zV*ogf71(k|_4W6ih|p?~GqQmm(>7kz zK(!SMX@1&P66O~#Zt1QFUPbxYnkzaFG1sfeL7|STEHvy?_xpfd^}?#JF*~`r>7i(2 zl|}E#G)3pSgq(F)-=p>U^P3PiZQxRhb;t0i$0gQVyOu{Ay5FaR;bAZmt39_83H~R% z=^OA!i>P7u+}tuYMDO6_T!jSeBe@uuAXxU-twD}{3@s(H;R9>GcXz`SS>y%8g}&uE zibAB_1FW7}3y{#f{QZ}QSFM1W+aVE=4m^V2EyGT3!Z4=}QHeCXO7xGvWn1%%**SU` zpf7YAgqB?hZV1iEYFwVlD5yYVy99{anLP`#uuI#F)UOicw+|mb9*eDw5jJIWY?nn{ z!HmFZHZhJG$N+i|;U{jv}9pl9S z+;XsS6NY_T4w`HS?~|PF*++hYZ3oE7{h*-jBo&gBWW{YxmW8$rK9RTgEmT59y3Zk9 zgjYQQ#(k_f3&sOaVlz~1o9%i!J6&-#hBZK9e!&6rdi-Up)pvXo0&q;$qZvQ|$^Tli zW9QB#s;a8UNDrFsr4)eRwlR1*iq5y>4i;6{k00N0vA;h_&fgjsxE>g{4Rtkz5;%Ek z`dT|Bct8PQwwJYrvB6PRh>75Te>9=SR3ti#s5hVpQ_WF(o%&Se#ExCv78{KU73E=y zaZF8(i4-97qvPXJ7cOj^7-$H$V;RZ!cv)6<8b>@sNJxlKw{LK;#}XR4XNmfWV{*}k z`OAT8{sdg+CEhI?7q^ZY**bShbZ#p|9Y?dEBUj&We2GwPq&#vJcbsL^kIz%7wbwTo zjlYX&V!-$N)NpiO3znEM48#!d7LH62$B9^PXUT&H4QLBW0HKiEPXlrZNPO}Le5c?23@HjS>6IJlqO`B-XoH>J&@+w*?wZbkK zL=r1Vs-sM-1jW(8!EkQy4U;XYY4Ydd21>y|WtN;U0WUCT(=6th_i?inrdu^4xtd`mOlwwjK{U3#}dh}Z)_%PzsJF-9{& zfz&lTJy3kRN92%w)B|Ucu(@wLNIy3-`3ZRv5^_&m18RnI<6><6nwKu^Ks@1foVJA7 z;ASK@_ubt;KwX@x)#rl8cx8F{8bs!2s9JDty{PXWMW_zPdin{@cy7}X*)^VxECNg( z`Vh>lr7qdQ&Q1rvLf$Q_Pbw&U#HJ*l`cczY-m<`5!gwzxZl^Y7t^h}gXXqMjF)Y{eL;Xd z(N$#}F?LlUUSbjg1g8*de;@}~Tr$8`ja3>T`Hc(Fwz{0GmWEBgR79p9GK$k>Y@8V1G&Hx{NY$M7%@ zTup0DtRi&fBO@dEQBETgZ3p83?Uy5pV3`9Jcn8#irEx1`V)0KZVY$0pkJVP@#1U4wCY(bsYhVV3qlwT zx0M~6@QqwZ^*Fbrq_!3)jOJEWn{u^yP**(xVkJ5$M{b@G*)Qwkj?KE+RtpKGjE`Inv6NpU-o|36 zuiyX9w<6t*iOEhvXuH_*VAyTptC-%*MFGxd_|jQ!b6*q-5?|iT?JSmvOXN$mV2Q}T zzDxPf{_N0^y?=f+iDvtY4eQp4OF>op4hARh>gyRrg6UCfKNHZSN0&0x(g1IC;mn_3 zV23Hgo^#LuF zmo>d3s9<&-IxkNw{%n1q~zqckOic;NQb`z&MhXl4zV{Bl5|tUNU3&;iESSm z8X6fH!C|mv9=TjVXej@+mC}#5KY+PP0j?DE8XH)2kou_Zbu8>@Q1rhP+Wh?f{ntmo zlMnuL|Jk!=UC>dv1X)PL)|iPPPfrF|+uldrbnL`|idaNc;NXs9#OdwlcL(F8!rIzR z=m*zM_zmMzp+`fqk(+1aANLE!+PP`J6d ze?#vZx2Y@Wg6oj1st<~dH8*Qq zZ%p|M7f)wq>J{wkd(}w-iwgMB?tI8f+`{vD-_*3tE;moviX@6Z{I#Xg6|)p!{^-f7 z-5F$q1s1c7r2{)*%QPboQvp?z+8X(7>OM%(t-P)o;}|O zaql@g{;aDiK!mr8-dv4{d)&t65D4dMnV2486$d1Kpr^oXUIq_*G_Y-!4w1KD*m=SJ zkf>;2Pv^jZJL*tW%2(i$3^!@Kluw4APn|gNwNn2? z$=pS<>$wpYSD-gu0D5#4#y){M4_83z#|7c(G5}1%B{26hTwz0xonLkSIz%8~x4s4w zqubgG)Ag{U8G2`q5~dIwng_V|tusMzxxLoxN+QWa13@MmTieShqTo^e-j~}AJBRVE zeI=ZAPx3h>p24~n&pEmMzZX-;Z#OKVCoe+21}KYb0dIHWrV9xSi(mi!Q1yT5WeXOT Y7*H}ec+^tKsFxxurEntQ*rnV511F5s9{>OV diff --git a/docs/source/contributor-guide/benchmark-results/2024-05-30/comet-8-exec-5-runs.json b/docs/source/contributor-guide/benchmark-results/2024-06-29/comet-8-exec-5-runs.json similarity index 52% rename from docs/source/contributor-guide/benchmark-results/2024-05-30/comet-8-exec-5-runs.json rename to docs/source/contributor-guide/benchmark-results/2024-06-29/comet-8-exec-5-runs.json index 38142151d..f139578eb 100644 --- a/docs/source/contributor-guide/benchmark-results/2024-05-30/comet-8-exec-5-runs.json +++ b/docs/source/contributor-guide/benchmark-results/2024-06-29/comet-8-exec-5-runs.json @@ -5,197 +5,198 @@ "query_path": "../../tpch/queries", "spark_conf": { "spark.comet.explainFallback.enabled": "true", + "spark.eventLog.enabled": "true", "spark.jars": "file:///home/andy/git/apache/datafusion-comet/spark/target/comet-spark-spark3.4_2.12-0.1.0-SNAPSHOT.jar", "spark.comet.cast.allowIncompatible": "true", + "spark.app.startTime": "1719691158901", "spark.executor.extraClassPath": "/home/andy/git/apache/datafusion-comet/spark/target/comet-spark-spark3.4_2.12-0.1.0-SNAPSHOT.jar", - "spark.executor.memory": "8G", "spark.comet.exec.shuffle.enabled": "true", "spark.app.name": "DataFusion Comet Benchmark derived from TPC-H / TPC-DS", - "spark.driver.port": "36573", - "spark.sql.adaptive.coalescePartitions.enabled": "false", - "spark.app.startTime": "1716923498046", + "spark.app.id": "app-20240629135919-0008", "spark.comet.batchSize": "8192", - "spark.app.id": "app-20240528131138-0043", "spark.serializer.objectStreamReset": "100", - "spark.app.initial.jar.urls": "spark://woody.lan:36573/jars/comet-spark-spark3.4_2.12-0.1.0-SNAPSHOT.jar", + "spark.driver.host": "10.0.0.118", "spark.submit.deployMode": "client", "spark.sql.autoBroadcastJoinThreshold": "-1", "spark.comet.exec.all.enabled": "true", - "spark.eventLog.enabled": "false", - "spark.driver.host": "woody.lan", + "spark.executor.cores": "8", "spark.driver.extraJavaOptions": "-Djava.net.preferIPv6Addresses=false -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false", + "spark.comet.shuffle.enforceMode.enabled": "true", "spark.sql.warehouse.dir": "file:/home/andy/git/apache/datafusion-benchmarks/runners/datafusion-comet/spark-warehouse", "spark.shuffle.manager": "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager", - "spark.comet.exec.enabled": "true", + "spark.app.submitTime": "1719691158623", "spark.repl.local.jars": "file:///home/andy/git/apache/datafusion-comet/spark/target/comet-spark-spark3.4_2.12-0.1.0-SNAPSHOT.jar", + "spark.comet.exec.enabled": "true", "spark.executor.id": "driver", "spark.master": "spark://woody:7077", - "spark.executor.instances": "8", "spark.comet.exec.shuffle.mode": "auto", + "spark.driver.port": "34629", "spark.sql.extensions": "org.apache.comet.CometSparkSessionExtensions", "spark.driver.memory": "8G", "spark.driver.extraClassPath": "/home/andy/git/apache/datafusion-comet/spark/target/comet-spark-spark3.4_2.12-0.1.0-SNAPSHOT.jar", + "spark.sql.adaptive.coalescePartitions.enabled": "true", + "spark.executor.memory": "32G", "spark.rdd.compress": "True", "spark.executor.extraJavaOptions": "-Djava.net.preferIPv6Addresses=false -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false", + "spark.executor.instances": "1", "spark.cores.max": "8", "spark.comet.enabled": "true", - "spark.app.submitTime": "1716923497738", "spark.submit.pyFiles": "", - "spark.executor.cores": "1", - "spark.comet.parquet.io.enabled": "false" + "spark.app.initial.jar.urls": "spark://10.0.0.118:34629/jars/comet-spark-spark3.4_2.12-0.1.0-SNAPSHOT.jar", + "spark.comet.cbo.enabled": "false" }, "1": [ - 32.121661901474, - 27.997092485427856, - 27.756758451461792, - 28.55236315727234, - 28.332542181015015 + 28.735982179641724, + 27.904003858566284, + 27.98918342590332, + 27.998026847839355, + 27.7985897064209 ], "2": [ - 18.269107580184937, - 16.200955629348755, - 16.194639682769775, - 16.745808839797974, - 16.59864115715027 + 15.840301513671875, + 15.137918710708618, + 15.086657047271729, + 15.252221584320068, + 15.093742370605469 ], "3": [ - 17.265466690063477, - 17.069786310195923, - 17.12887978553772, - 19.33678102493286, - 18.182055234909058 + 18.124080181121826, + 18.498253345489502, + 18.420130252838135, + 18.309802055358887, + 18.46897006034851 ], "4": [ - 8.367004156112671, - 8.172023296356201, - 8.023266077041626, - 8.350765228271484, - 8.258736610412598 + 9.55617070198059, + 9.518851518630981, + 9.514896392822266, + 9.583910465240479, + 9.444581985473633 ], "5": [ - 34.10048794746399, - 32.69314408302307, - 33.21383595466614, - 36.391114473342896, - 39.00048065185547 + 33.23771286010742, + 33.053247690200806, + 32.84638738632202, + 32.790276765823364, + 32.90981197357178 ], "6": [ - 3.1693499088287354, - 3.044705390930176, - 3.047694206237793, - 3.2817511558532715, - 3.274174928665161 + 3.34500789642334, + 2.9966821670532227, + 3.0137181282043457, + 2.9657068252563477, + 2.919524908065796 ], "7": [ - 25.369214296340942, - 24.020941257476807, - 24.0787034034729, - 28.47402787208557, - 28.23443365097046 + 20.84096646308899, + 20.373249053955078, + 20.337918519973755, + 20.32623314857483, + 20.321190357208252 ], "8": [ - 40.06126809120178, - 39.828824281692505, - 45.250510454177856, - 44.406742572784424, - 48.98451232910156 + 36.99943470954895, + 36.097434520721436, + 36.08603119850159, + 36.26709461212158, + 36.22776746749878 ], "9": [ - 62.822797775268555, - 61.26328158378601, - 64.95581865310669, - 69.51708793640137, - 73.52380013465881 + 58.00954031944275, + 56.75375247001648, + 57.23253607749939, + 57.04572892189026, + 57.06179666519165 ], "10": [ - 20.55334782600403, - 20.546096324920654, - 20.57452392578125, - 22.84211039543152, - 23.724371671676636 + 19.51328682899475, + 19.17092227935791, + 19.110991716384888, + 19.05888819694519, + 19.292072534561157 ], "11": [ - 11.068235158920288, - 10.715423822402954, - 11.353424310684204, - 11.37632942199707, - 11.530814170837402 + 12.222111463546753, + 12.186187267303467, + 12.177972316741943, + 12.100908517837524, + 12.061741828918457 ], "12": [ - 10.264788389205933, - 8.67864990234375, - 8.845952033996582, - 8.593009233474731, - 8.540803909301758 + 7.657347679138184, + 7.598176002502441, + 7.568347930908203, + 7.4833292961120605, + 7.551736116409302 ], "13": [ - 9.603406190872192, - 9.648627042770386, - 13.040799140930176, - 10.154011249542236, - 9.716034412384033 + 9.64631199836731, + 9.536576509475708, + 9.564186096191406, + 9.570204496383667, + 9.662892580032349 ], "14": [ - 6.20926308631897, - 6.0385496616363525, - 7.674488544464111, - 10.53052043914795, - 7.661675691604614 + 6.022975921630859, + 5.84771203994751, + 6.049532175064087, + 5.998222827911377, + 5.899066925048828 ], "15": [ - 11.466301918029785, - 11.473632097244263, - 11.279382228851318, - 13.291078329086304, - 12.81026816368103 + 10.946545600891113, + 10.68128228187561, + 10.473867416381836, + 10.72830843925476, + 10.45834231376648 ], "16": [ - 8.096073865890503, - 7.73410701751709, - 7.742897272109985, - 8.477537631988525, - 7.821273326873779 + 7.951048851013184, + 6.773421049118042, + 6.630566120147705, + 6.826274633407593, + 6.515024185180664 ], "17": [ - 43.69264578819275, - 43.33040428161621, - 46.291987657547, - 54.654345989227295, - 54.37124800682068 + 46.03706979751587, + 42.801599740982056, + 42.59856081008911, + 42.84500861167908, + 42.899412870407104 ], "18": [ - 27.205485105514526, - 26.785916090011597, - 27.331408262252808, - 29.946768760681152, - 28.037617444992065 + 34.244925022125244, + 31.239882469177246, + 31.353251695632935, + 31.224499940872192, + 31.53875970840454 ], "19": [ - 8.100102186203003, - 7.845783472061157, - 8.52329158782959, - 8.907397985458374, - 9.13755488395691 + 7.07506251335144, + 6.813824892044067, + 6.79759407043457, + 6.941055059432983, + 6.83566427230835 ], "20": [ - 13.09695029258728, - 12.683861255645752, - 15.612725019454956, - 13.361177206039429, - 16.614356517791748 + 10.964829683303833, + 10.757019996643066, + 10.806366205215454, + 10.990953922271729, + 10.887315273284912 ], "21": [ - 43.69623780250549, - 43.26758122444153, - 46.91650056838989, - 47.875754833221436, - 57.9763662815094 + 44.07762622833252, + 44.03535461425781, + 43.978052377700806, + 43.928617000579834, + 43.93204379081726 ], "22": [ - 4.5090577602386475, - 4.420571804046631, - 4.639787673950195, - 5.118046998977661, - 5.017346143722534 + 4.871158599853516, + 4.810696601867676, + 4.873842239379883, + 4.817774534225464, + 4.927582740783691 ] } \ No newline at end of file diff --git a/docs/source/contributor-guide/benchmark-results/2024-05-30/datafusion-python-8-cores.json b/docs/source/contributor-guide/benchmark-results/2024-06-29/datafusion-python-8-cores.json similarity index 100% rename from docs/source/contributor-guide/benchmark-results/2024-05-30/datafusion-python-8-cores.json rename to docs/source/contributor-guide/benchmark-results/2024-06-29/datafusion-python-8-cores.json diff --git a/docs/source/contributor-guide/benchmark-results/2024-05-30/spark-8-exec-5-runs.json b/docs/source/contributor-guide/benchmark-results/2024-06-29/spark-8-exec-5-runs.json similarity index 100% rename from docs/source/contributor-guide/benchmark-results/2024-05-30/spark-8-exec-5-runs.json rename to docs/source/contributor-guide/benchmark-results/2024-06-29/spark-8-exec-5-runs.json diff --git a/docs/source/contributor-guide/benchmarking.md b/docs/source/contributor-guide/benchmarking.md index 3e9a61efb..d315c559e 100644 --- a/docs/source/contributor-guide/benchmarking.md +++ b/docs/source/contributor-guide/benchmarking.md @@ -51,7 +51,8 @@ $SPARK_HOME/bin/spark-submit \ $SPARK_HOME/bin/spark-submit \ --master $SPARK_MASTER \ --conf spark.driver.memory=8G \ - --conf spark.executor.memory=64G \ + --conf spark.executor.instances=1 \ + --conf spark.executor.memory=32G \ --conf spark.executor.cores=8 \ --conf spark.cores.max=8 \ --conf spark.sql.autoBroadcastJoinThreshold=-1 \ @@ -68,8 +69,8 @@ $SPARK_HOME/bin/spark-submit \ --conf spark.comet.batchSize=8192 \ --conf spark.comet.exec.shuffle.enabled=true \ --conf spark.comet.exec.shuffle.mode=auto \ + --conf spark.comet.shuffle.enforceMode.enabled=true \ --conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager \ - --conf spark.sql.adaptive.coalescePartitions.enabled=false \ tpcbench.py \ --benchmark tpch \ --data /mnt/bigdata/tpch/sf100/ \ @@ -99,7 +100,7 @@ is an ongoing task, and we welcome contributions from the community to help achi The raw results of these benchmarks in JSON format is available here: -- [Spark](./benchmark-results/2024-05-30/spark-8-exec-5-runs.json) -- [Comet](./benchmark-results/2024-05-30/comet-8-exec-5-runs.json) -- [DataFusion](./benchmark-results/2024-05-30/datafusion-python-8-cores.json) +- [Spark](./benchmark-results/2024-06-29/spark-8-exec-5-runs.json) +- [Comet](./benchmark-results/2024-06-29/comet-8-exec-5-runs.json) +- [DataFusion](./benchmark-results/2024-06-29/datafusion-python-8-cores.json) 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 1e61ef75e..6838e0237 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -2301,7 +2301,7 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim .addAllSortOrders(sortOrders.map(_.get).asJava) Some(result.setSort(sortBuilder).build()) } else { - withInfo(op, sortOrder: _*) + withInfo(op, "sort order not supported", sortOrder: _*) None } From 917fd4334ce0028946abbba84cd469ef6c9c8fd3 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 1 Jul 2024 20:18:04 -0600 Subject: [PATCH 04/16] feat: Enable Spark SQL tests for Spark 3.5.1 (#603) * Enable Spark SQL tests for Spark 3.5.1 * fix conflicts * finish creating diff * improve docs, update slf4j version * improve docs * fix diff * fix diff * address feedback * add IgnoreComet to patch * update test * remove unused imports * ignore 2 failing tests * scalastyle * ignore failing parquet metadata tests * add links to tracking issue --- .github/workflows/spark_sql_test.yml | 3 +- dev/diffs/3.5.1.diff | 2815 +++++++++++++++++ .../contributor-guide/spark-sql-tests.md | 128 + docs/source/index.rst | 1 + pom.xml | 2 +- 5 files changed, 2947 insertions(+), 2 deletions(-) create mode 100644 dev/diffs/3.5.1.diff create mode 100644 docs/source/contributor-guide/spark-sql-tests.md diff --git a/.github/workflows/spark_sql_test.yml b/.github/workflows/spark_sql_test.yml index 352e0ecbe..1cc6a1ff4 100644 --- a/.github/workflows/spark_sql_test.yml +++ b/.github/workflows/spark_sql_test.yml @@ -45,7 +45,7 @@ jobs: matrix: os: [ubuntu-latest] java-version: [11] - spark-version: [{short: '3.4', full: '3.4.3'}] + spark-version: [{short: '3.4', full: '3.4.3'}, {short: '3.5', full: '3.5.1'}] module: - {name: "catalyst", args1: "catalyst/test", args2: ""} - {name: "sql/core-1", args1: "", args2: sql/testOnly * -- -l org.apache.spark.tags.ExtendedSQLTest -l org.apache.spark.tags.SlowSQLTest} @@ -75,6 +75,7 @@ jobs: - name: Run Spark tests run: | cd apache-spark + rm -rf /root/.m2/repository/org/apache/parquet # somehow parquet cache requires cleanups ENABLE_COMET=true build/sbt ${{ matrix.module.args1 }} "${{ matrix.module.args2 }}" env: LC_ALL: "C.UTF-8" diff --git a/dev/diffs/3.5.1.diff b/dev/diffs/3.5.1.diff new file mode 100644 index 000000000..ebc4cdb8c --- /dev/null +++ b/dev/diffs/3.5.1.diff @@ -0,0 +1,2815 @@ +diff --git a/pom.xml b/pom.xml +index 0f504dbee85..71fd49a3744 100644 +--- a/pom.xml ++++ b/pom.xml +@@ -152,6 +152,8 @@ + --> + 2.5.1 + 2.0.8 ++ 3.5 ++ 0.1.0-SNAPSHOT + + +# Running Spark SQL Tests + +Running Apache Spark's SQL tests with Comet enabled is a good way to ensure that Comet produces the same +results as that version of Spark. To enable this, we apply some changes to the Apache Spark source code so that +Comet is enabled when we run the tests. + +Here is an overview of the changes that we need to make to Spark: + +- Update the pom.xml to add a dependency on Comet +- Modify SparkSession to load the Comet extension +- Modify TestHive to load Comet +- Modify SQLTestUtilsBase to load Comet when `ENABLE_COMET` environment variable exists + +Here are the steps involved in running the Spark SQL tests with Comet, using Spark 3.4.3 for this example. + +## 1. Install Comet + +Run `make release` in Comet to install the Comet JAR into the local Maven repository, specifying the Spark version. + +```shell +PROFILES="-Pspark-3.4" make release +``` + +## 2. Clone Spark and Apply Diff + +Clone Apache Spark locally and apply the diff file from Comet. + +```shell +git clone git@github.com:apache/spark.git apache-spark +cd apache-spark +git checkout v3.4.3 +git apply ../datafusion-comet/dev/diffs/3.4.3.diff +``` + +## 3. Run Spark SQL Tests + +Use the following commands to run the SQL tests locally. + +```shell +ENABLE_COMET=true build/sbt catalyst/test +ENABLE_COMET=true build/sbt "sql/testOnly * -- -l org.apache.spark.tags.ExtendedSQLTest -l org.apache.spark.tags.SlowSQLTest" +ENABLE_COMET=true build/sbt "sql/testOnly * -- -n org.apache.spark.tags.ExtendedSQLTest" +ENABLE_COMET=true build/sbt "sql/testOnly * -- -n org.apache.spark.tags.SlowSQLTest" +ENABLE_COMET=true build/sbt "hive/testOnly * -- -l org.apache.spark.tags.ExtendedHiveTest -l org.apache.spark.tags.SlowHiveTest" +ENABLE_COMET=true build/sbt "hive/testOnly * -- -n org.apache.spark.tags.ExtendedHiveTest" +ENABLE_COMET=true build/sbt "hive/testOnly * -- -n org.apache.spark.tags.SlowHiveTest" +``` + +## Creating a diff file for a new Spark version + +Once Comet has support for a new Spark version, we need to create a diff file that can be applied to that version +of Apache Spark to enable Comet when running tests. This is a highly manual process and the process can +vary depending on the changes in the new version of Spark, but here is a general guide to the process. + +We typically start by applying a patch from a previous version of Spark. For example, when enabling the tests +for Spark version 3.5.1 we may start by applying the existing diff for 3.4.3 first. + +```shell +cd git/apache/spark +git checkout v3.5.1 +git apply --reject --whitespace=fix ../datafusion-comet/dev/diffs/3.4.3.diff +``` + +Any changes that cannot be cleanly applied will instead be written out to reject files. For example, the above +command generated the following files. + +```shell +find . -name "*.rej" +./pom.xml.rej +./sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala.rej +./sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala.rej +./sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala.rej +./sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala.rej +./sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala.rej +./sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala.rej +./sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala.rej +./sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRebaseDatetimeSuite.scala.rej +./sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala.rej +./sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala.rej +./sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala.rej +./sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala.rej +./sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala.rej +./sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala.rej +./sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala.rej +./sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala.rej +./sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala.rej +``` + +The changes in these reject files need to be applied manually. + +One method is to use the [wiggle](https://github.com/neilbrown/wiggle) command (`brew install wiggle` on Mac). + +For example: + +```shell +wiggle --replace ./sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala ./sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala.rej +``` + +## Generating The Diff File + +```shell +git diff v3.5.1 > ../datafusion-comet/dev/diffs/3.5.1.diff +``` + +## Running Tests in CI + +The easiest way to run the tests is to create a PR against Comet and let CI run the tests. When working with a +new Spark version, the `spark_sql_test.yaml` and `spark_sql_test_ansi.yaml` files will need updating with the +new version. \ No newline at end of file diff --git a/docs/source/index.rst b/docs/source/index.rst index 1e9eba21f..100145a97 100644 --- a/docs/source/index.rst +++ b/docs/source/index.rst @@ -61,6 +61,7 @@ as a native runtime to achieve improvement in terms of query efficiency and quer Benchmarking Guide Adding a New Expression Profiling Native Code + Spark SQL Tests Github and Issue Tracker .. _toc.asf-links: diff --git a/pom.xml b/pom.xml index 4dcc75834..0195e9292 100644 --- a/pom.xml +++ b/pom.xml @@ -551,13 +551,13 @@ under the License. - spark-3.5 2.12.18 3.5.1 3.5 1.13.1 + 2.0.7 spark-3.5 not-needed not-needed From 0d2fcbcee42235e371f0b2ad5fe7edfa9c96f4dd Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Tue, 2 Jul 2024 08:12:53 -0700 Subject: [PATCH 05/16] feat: Initial support for Window function (#599) * feat: initial support for Window function Co-authored-by: comphead * fix style * fix style * address comments * abs()->unsigned_abs() * address comments --------- Co-authored-by: comphead --- core/src/execution/datafusion/planner.rs | 196 +++++++++++++++++- core/src/execution/proto/operator.proto | 59 ++++++ .../comet/CometSparkSessionExtensions.scala | 12 ++ .../apache/comet/serde/QueryPlanSerde.scala | 162 ++++++++++++++- .../spark/sql/comet/CometWindowExec.scala | 135 ++++++++++++ .../apache/comet/exec/CometExecSuite.scala | 27 +++ 6 files changed, 589 insertions(+), 2 deletions(-) create mode 100644 spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala diff --git a/core/src/execution/datafusion/planner.rs b/core/src/execution/datafusion/planner.rs index afdebce32..a969c3baa 100644 --- a/core/src/execution/datafusion/planner.rs +++ b/core/src/execution/datafusion/planner.rs @@ -20,6 +20,8 @@ use std::{collections::HashMap, sync::Arc}; use arrow_schema::{DataType, Field, Schema, TimeUnit}; +use datafusion::physical_plan::windows::BoundedWindowAggExec; +use datafusion::physical_plan::InputOrderMode; use datafusion::{ arrow::{compute::SortOptions, datatypes::SchemaRef}, common::DataFusionError, @@ -50,12 +52,17 @@ use datafusion_common::{ tree_node::{Transformed, TransformedResult, TreeNode, TreeNodeRecursion, TreeNodeRewriter}, JoinType as DFJoinType, ScalarValue, }; -use datafusion_expr::ScalarUDF; +use datafusion_expr::expr::find_df_window_func; +use datafusion_expr::{ScalarUDF, WindowFrame, WindowFrameBound, WindowFrameUnits}; +use datafusion_physical_expr::window::WindowExpr; use datafusion_physical_expr_common::aggregate::create_aggregate_expr; use itertools::Itertools; use jni::objects::GlobalRef; use num::{BigInt, ToPrimitive}; +use crate::execution::spark_operator::lower_window_frame_bound::LowerFrameBoundStruct; +use crate::execution::spark_operator::upper_window_frame_bound::UpperFrameBoundStruct; +use crate::execution::spark_operator::WindowFrameType; use crate::{ errors::ExpressionError, execution::{ @@ -980,6 +987,47 @@ impl PhysicalPlanner { Ok((scans, hash_join)) } + OpStruct::Window(wnd) => { + let (scans, child) = self.create_plan(&children[0], inputs)?; + let input_schema = child.schema(); + let sort_exprs: Result, ExecutionError> = wnd + .order_by_list + .iter() + .map(|expr| self.create_sort_expr(expr, input_schema.clone())) + .collect(); + + let partition_exprs: Result>, ExecutionError> = wnd + .partition_by_list + .iter() + .map(|expr| self.create_expr(expr, input_schema.clone())) + .collect(); + + let sort_exprs = &sort_exprs?; + let partition_exprs = &partition_exprs?; + + let window_expr: Result>, ExecutionError> = wnd + .window_expr + .iter() + .map(|expr| { + self.create_window_expr( + expr, + input_schema.clone(), + partition_exprs, + sort_exprs, + ) + }) + .collect(); + + Ok(( + scans, + Arc::new(BoundedWindowAggExec::try_new( + window_expr?, + child, + partition_exprs.to_vec(), + InputOrderMode::Sorted, + )?), + )) + } } } @@ -1322,6 +1370,152 @@ impl PhysicalPlanner { } } + /// Create a DataFusion windows physical expression from Spark physical expression + fn create_window_expr<'a>( + &'a self, + spark_expr: &'a crate::execution::spark_operator::WindowExpr, + input_schema: SchemaRef, + partition_by: &[Arc], + sort_exprs: &[PhysicalSortExpr], + ) -> Result, ExecutionError> { + let (mut window_func_name, mut window_func_args) = (String::new(), Vec::new()); + if let Some(func) = &spark_expr.built_in_window_function { + match &func.expr_struct { + Some(ExprStruct::ScalarFunc(f)) => { + window_func_name.clone_from(&f.func); + window_func_args.clone_from(&f.args); + } + other => { + return Err(ExecutionError::GeneralError(format!( + "{other:?} not supported for window function" + ))) + } + }; + } else if let Some(agg_func) = &spark_expr.agg_func { + let result = Self::process_agg_func(agg_func)?; + window_func_name = result.0; + window_func_args = result.1; + } else { + return Err(ExecutionError::GeneralError( + "Both func and agg_func are not set".to_string(), + )); + } + + let window_func = match find_df_window_func(&window_func_name) { + Some(f) => f, + _ => { + return Err(ExecutionError::GeneralError(format!( + "{window_func_name} not supported for window function" + ))) + } + }; + + let window_args = window_func_args + .iter() + .map(|expr| self.create_expr(expr, input_schema.clone())) + .collect::, ExecutionError>>()?; + + let spark_window_frame = match spark_expr + .spec + .as_ref() + .and_then(|inner| inner.frame_specification.as_ref()) + { + Some(frame) => frame, + _ => { + return Err(ExecutionError::DeserializeError( + "Cannot deserialize window frame".to_string(), + )) + } + }; + + let units = match spark_window_frame.frame_type() { + WindowFrameType::Rows => WindowFrameUnits::Rows, + WindowFrameType::Range => WindowFrameUnits::Range, + }; + + let lower_bound: WindowFrameBound = match spark_window_frame + .lower_bound + .as_ref() + .and_then(|inner| inner.lower_frame_bound_struct.as_ref()) + { + Some(l) => match l { + LowerFrameBoundStruct::UnboundedPreceding(_) => { + WindowFrameBound::Preceding(ScalarValue::UInt64(None)) + } + LowerFrameBoundStruct::Preceding(offset) => { + let offset_value = offset.offset.unsigned_abs() as u64; + WindowFrameBound::Preceding(ScalarValue::UInt64(Some(offset_value))) + } + LowerFrameBoundStruct::CurrentRow(_) => WindowFrameBound::CurrentRow, + }, + None => WindowFrameBound::Preceding(ScalarValue::UInt64(None)), + }; + + let upper_bound: WindowFrameBound = match spark_window_frame + .upper_bound + .as_ref() + .and_then(|inner| inner.upper_frame_bound_struct.as_ref()) + { + Some(u) => match u { + UpperFrameBoundStruct::UnboundedFollowing(_) => { + WindowFrameBound::Following(ScalarValue::UInt64(None)) + } + UpperFrameBoundStruct::Following(offset) => { + WindowFrameBound::Following(ScalarValue::UInt64(Some(offset.offset as u64))) + } + UpperFrameBoundStruct::CurrentRow(_) => WindowFrameBound::CurrentRow, + }, + None => WindowFrameBound::Following(ScalarValue::UInt64(None)), + }; + + let window_frame = WindowFrame::new_bounds(units, lower_bound, upper_bound); + + datafusion::physical_plan::windows::create_window_expr( + &window_func, + window_func_name, + &window_args, + partition_by, + sort_exprs, + window_frame.into(), + &input_schema, + false, // TODO: Ignore nulls + ) + .map_err(|e| ExecutionError::DataFusionError(e.to_string())) + } + + fn process_agg_func(agg_func: &AggExpr) -> Result<(String, Vec), ExecutionError> { + fn optional_expr_to_vec(expr_option: &Option) -> Vec { + expr_option + .as_ref() + .cloned() + .map_or_else(Vec::new, |e| vec![e]) + } + + fn int_to_stats_type(value: i32) -> Option { + match value { + 0 => Some(StatsType::Sample), + 1 => Some(StatsType::Population), + _ => None, + } + } + + match &agg_func.expr_struct { + Some(AggExprStruct::Count(expr)) => { + let args = &expr.children; + Ok(("count".to_string(), args.to_vec())) + } + Some(AggExprStruct::Min(expr)) => { + Ok(("min".to_string(), optional_expr_to_vec(&expr.child))) + } + Some(AggExprStruct::Max(expr)) => { + Ok(("max".to_string(), optional_expr_to_vec(&expr.child))) + } + other => Err(ExecutionError::GeneralError(format!( + "{other:?} not supported for window function" + ))), + } + } + /// Create a DataFusion physical partitioning from Spark physical partitioning fn create_partitioning( &self, diff --git a/core/src/execution/proto/operator.proto b/core/src/execution/proto/operator.proto index de25f94da..335d42596 100644 --- a/core/src/execution/proto/operator.proto +++ b/core/src/execution/proto/operator.proto @@ -42,6 +42,7 @@ message Operator { Expand expand = 107; SortMergeJoin sort_merge_join = 108; HashJoin hash_join = 109; + Window window = 110; } } @@ -120,3 +121,61 @@ enum BuildSide { BuildLeft = 0; BuildRight = 1; } + +message WindowExpr { + spark.spark_expression.Expr built_in_window_function = 1; + spark.spark_expression.AggExpr agg_func = 2; + WindowSpecDefinition spec = 3; +} + +enum WindowFrameType { + Rows = 0; + Range = 1; +} + +message WindowFrame { + WindowFrameType frame_type = 1; + LowerWindowFrameBound lower_bound = 2; + UpperWindowFrameBound upper_bound = 3; +} + +message LowerWindowFrameBound { + oneof lower_frame_bound_struct { + UnboundedPreceding unboundedPreceding = 1; + Preceding preceding = 2; + CurrentRow currentRow = 3; + } +} + +message UpperWindowFrameBound { + oneof upper_frame_bound_struct { + UnboundedFollowing unboundedFollowing = 1; + Following following = 2; + CurrentRow currentRow = 3; + } +} + +message Preceding { + int32 offset = 1; +} + +message Following { + int32 offset = 1; +} + +message UnboundedPreceding {} +message UnboundedFollowing {} +message CurrentRow {} + +message WindowSpecDefinition { + repeated spark.spark_expression.Expr partitionSpec = 1; + repeated spark.spark_expression.Expr orderSpec = 2; + WindowFrame frameSpecification = 3; +} + +message Window { + repeated WindowExpr window_expr = 1; + repeated spark.spark_expression.Expr order_by_list = 2; + repeated spark.spark_expression.Expr partition_by_list = 3; + Operator child = 4; +} diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index 37ca55e27..ffb7de83b 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -40,6 +40,7 @@ import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.execution.window.WindowExec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -541,6 +542,17 @@ class CometSparkSessionExtensions withInfo(s, Seq(info1, info2).flatten.mkString(",")) s + case w: WindowExec => + val newOp = transform1(w) + newOp match { + case Some(nativeOp) => + val cometOp = + CometWindowExec(w, w.windowExpression, w.partitionSpec, w.orderSpec, w.child) + CometSinkPlaceHolder(nativeOp, w, cometOp) + case None => + w + } + case u: UnionExec if isCometOperatorEnabled(conf, "union") && u.children.forall(isCometNative) => 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 6838e0237..3465315d1 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -23,7 +23,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Average, BitAndAgg, BitOrAgg, BitXorAgg, Corr, Count, CovPopulation, CovSample, Final, First, Last, Max, Min, Partial, StddevPop, StddevSamp, Sum, VariancePop, VarianceSamp} +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Average, BitAndAgg, BitOrAgg, BitXorAgg, Complete, Corr, Count, CovPopulation, CovSample, Final, First, Last, Max, Min, Partial, StddevPop, StddevSamp, Sum, VariancePop, VarianceSamp} import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, NormalizeNaNAndZero} import org.apache.spark.sql.catalyst.plans._ @@ -37,6 +37,7 @@ import org.apache.spark.sql.execution.adaptive.{BroadcastQueryStageExec, Shuffle import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, HashJoin, ShuffledHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.execution.window.WindowExec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -199,6 +200,129 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim } } + def windowExprToProto( + windowExpr: WindowExpression, + output: Seq[Attribute]): Option[OperatorOuterClass.WindowExpr] = { + + val aggregateExpressions: Array[AggregateExpression] = windowExpr.flatMap { expr => + expr match { + case agg: AggregateExpression => + agg.aggregateFunction match { + case _: Min | _: Max | _: Count => + Some(agg) + case _ => + withInfo(windowExpr, "Unsupported aggregate", expr) + None + } + case _ => + None + } + }.toArray + + val (aggExpr, builtinFunc) = if (aggregateExpressions.nonEmpty) { + val modes = aggregateExpressions.map(_.mode).distinct + assert(modes.size == 1 && modes.head == Complete) + (aggExprToProto(aggregateExpressions.head, output, true), None) + } else { + (None, exprToProto(windowExpr.windowFunction, output)) + } + + val f = windowExpr.windowSpec.frameSpecification + + val (frameType, lowerBound, upperBound) = f match { + case SpecifiedWindowFrame(frameType, lBound, uBound) => + val frameProto = frameType match { + case RowFrame => OperatorOuterClass.WindowFrameType.Rows + case RangeFrame => OperatorOuterClass.WindowFrameType.Range + } + + val lBoundProto = lBound match { + case UnboundedPreceding => + OperatorOuterClass.LowerWindowFrameBound + .newBuilder() + .setUnboundedPreceding(OperatorOuterClass.UnboundedPreceding.newBuilder().build()) + .build() + case CurrentRow => + OperatorOuterClass.LowerWindowFrameBound + .newBuilder() + .setCurrentRow(OperatorOuterClass.CurrentRow.newBuilder().build()) + .build() + case e => + OperatorOuterClass.LowerWindowFrameBound + .newBuilder() + .setPreceding( + OperatorOuterClass.Preceding + .newBuilder() + .setOffset(e.eval().asInstanceOf[Int]) + .build()) + .build() + } + + val uBoundProto = uBound match { + case UnboundedFollowing => + OperatorOuterClass.UpperWindowFrameBound + .newBuilder() + .setUnboundedFollowing(OperatorOuterClass.UnboundedFollowing.newBuilder().build()) + .build() + case CurrentRow => + OperatorOuterClass.UpperWindowFrameBound + .newBuilder() + .setCurrentRow(OperatorOuterClass.CurrentRow.newBuilder().build()) + .build() + case e => + OperatorOuterClass.UpperWindowFrameBound + .newBuilder() + .setFollowing( + OperatorOuterClass.Following + .newBuilder() + .setOffset(e.eval().asInstanceOf[Int]) + .build()) + .build() + } + + (frameProto, lBoundProto, uBoundProto) + case _ => + ( + OperatorOuterClass.WindowFrameType.Rows, + OperatorOuterClass.LowerWindowFrameBound + .newBuilder() + .setUnboundedPreceding(OperatorOuterClass.UnboundedPreceding.newBuilder().build()) + .build(), + OperatorOuterClass.UpperWindowFrameBound + .newBuilder() + .setUnboundedFollowing(OperatorOuterClass.UnboundedFollowing.newBuilder().build()) + .build()) + } + + val frame = OperatorOuterClass.WindowFrame + .newBuilder() + .setFrameType(frameType) + .setLowerBound(lowerBound) + .setUpperBound(upperBound) + .build() + + val spec = + OperatorOuterClass.WindowSpecDefinition.newBuilder().setFrameSpecification(frame).build() + + if (builtinFunc.isDefined) { + Some( + OperatorOuterClass.WindowExpr + .newBuilder() + .setBuiltInWindowFunction(builtinFunc.get) + .setSpec(spec) + .build()) + } else if (aggExpr.isDefined) { + Some( + OperatorOuterClass.WindowExpr + .newBuilder() + .setAggFunc(aggExpr.get) + .setSpec(spec) + .build()) + } else { + None + } + } + def aggExprToProto( aggExpr: AggregateExpression, inputs: Seq[Attribute], @@ -2352,6 +2476,41 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim None } + case WindowExec(windowExpression, partitionSpec, orderSpec, child) + if isCometOperatorEnabled(op.conf, "window") => + val output = child.output + + val winExprs: Array[WindowExpression] = windowExpression.flatMap { expr => + expr match { + case alias: Alias => + alias.child match { + case winExpr: WindowExpression => + Some(winExpr) + case _ => + None + } + case _ => + None + } + }.toArray + + val windowExprProto = winExprs.map(windowExprToProto(_, output)) + + val partitionExprs = partitionSpec.map(exprToProto(_, child.output)) + + val sortOrders = orderSpec.map(exprToProto(_, child.output)) + + if (windowExprProto.forall(_.isDefined) && partitionExprs.forall(_.isDefined) + && sortOrders.forall(_.isDefined)) { + val windowBuilder = OperatorOuterClass.Window.newBuilder() + windowBuilder.addAllWindowExpr(windowExprProto.map(_.get).toIterable.asJava) + windowBuilder.addAllPartitionByList(partitionExprs.map(_.get).asJava) + windowBuilder.addAllOrderByList(sortOrders.map(_.get).asJava) + Some(result.setWindow(windowBuilder).build()) + } else { + None + } + case HashAggregateExec( _, _, @@ -2652,6 +2811,7 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim case _: TakeOrderedAndProjectExec => true case BroadcastQueryStageExec(_, _: CometBroadcastExchangeExec, _) => true case _: BroadcastExchangeExec => true + case _: WindowExec => true case _ => false } } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala new file mode 100644 index 000000000..9a1232f0c --- /dev/null +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.comet + +import scala.collection.JavaConverters.asJavaIterableConverter + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, SortOrder, WindowExpression} +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.comet.CometWindowExec.getNativePlan +import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics, SQLShuffleReadMetricsReporter, SQLShuffleWriteMetricsReporter} +import org.apache.spark.sql.vectorized.ColumnarBatch + +import org.apache.comet.serde.OperatorOuterClass +import org.apache.comet.serde.OperatorOuterClass.Operator +import org.apache.comet.serde.QueryPlanSerde.{exprToProto, serializeDataType, windowExprToProto} + +/** + * Comet physical plan node for Spark `WindowsExec`. + * + * It is used to execute a `WindowsExec` physical operator by using Comet native engine. It is not + * like other physical plan nodes which are wrapped by `CometExec`, because it contains two native + * executions separated by a Comet shuffle exchange. + */ +case class CometWindowExec( + override val originalPlan: SparkPlan, + windowExpression: Seq[NamedExpression], + partitionSpec: Seq[Expression], + orderSpec: Seq[SortOrder], + child: SparkPlan) + extends CometExec + with UnaryExecNode { + + override def nodeName: String = "CometWindowExec" + + override def output: Seq[Attribute] = child.output ++ windowExpression.map(_.toAttribute) + + private lazy val writeMetrics = + SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) + private lazy val readMetrics = + SQLShuffleReadMetricsReporter.createShuffleReadMetrics(sparkContext) + override lazy val metrics: Map[String, SQLMetric] = Map( + "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"), + "shuffleReadElapsedCompute" -> + SQLMetrics.createNanoTimingMetric(sparkContext, "shuffle read elapsed compute at native"), + "numPartitions" -> SQLMetrics.createMetric( + sparkContext, + "number of partitions")) ++ readMetrics ++ writeMetrics + + override def supportsColumnar: Boolean = true + + protected override def doExecuteColumnar(): RDD[ColumnarBatch] = { + val childRDD = child.executeColumnar() + + childRDD.mapPartitionsInternal { iter => + CometExec.getCometIterator( + Seq(iter), + getNativePlan(output, windowExpression, partitionSpec, orderSpec, child).get) + } + } + + override def outputOrdering: Seq[SortOrder] = child.outputOrdering + + override def outputPartitioning: Partitioning = child.outputPartitioning + + protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = + this.copy(child = newChild) + +} + +object CometWindowExec { + def getNativePlan( + outputAttributes: Seq[Attribute], + windowExpression: Seq[NamedExpression], + partitionSpec: Seq[Expression], + orderSpec: Seq[SortOrder], + child: SparkPlan): Option[Operator] = { + + val orderSpecs = orderSpec.map(exprToProto(_, child.output)) + val partitionSpecs = partitionSpec.map(exprToProto(_, child.output)) + val scanBuilder = OperatorOuterClass.Scan.newBuilder() + val scanOpBuilder = OperatorOuterClass.Operator.newBuilder() + + val scanTypes = outputAttributes.flatten { attr => + serializeDataType(attr.dataType) + } + + val windowExprs = windowExpression.map(w => + windowExprToProto( + w.asInstanceOf[Alias].child.asInstanceOf[WindowExpression], + outputAttributes)) + + val windowBuilder = OperatorOuterClass.Window + .newBuilder() + + if (windowExprs.forall(_.isDefined)) { + windowBuilder + .addAllWindowExpr(windowExprs.map(_.get).asJava) + + if (orderSpecs.forall(_.isDefined)) { + windowBuilder.addAllOrderByList(orderSpecs.map(_.get).asJava) + } + + if (partitionSpecs.forall(_.isDefined)) { + windowBuilder.addAllPartitionByList(partitionSpecs.map(_.get).asJava) + } + + scanBuilder.addAllFields(scanTypes.asJava) + + val opBuilder = OperatorOuterClass.Operator + .newBuilder() + .addChildren(scanOpBuilder.setScan(scanBuilder)) + + Some(opBuilder.setWindow(windowBuilder).build()) + } else None + } +} 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 a0afe6b0c..9cc4e7f78 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -1430,6 +1430,33 @@ class CometExecSuite extends CometTestBase { } }) } + + test("Windows support") { + Seq("true", "false").foreach(aqeEnabled => + withSQLConf( + CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled) { + withParquetTable((0 until 10).map(i => (i, 10 - i)), "t1") { // TODO: test nulls + val aggregateFunctions = + List("COUNT(_1)", "MAX(_1)", "MIN(_1)") // TODO: Test all the aggregates + + aggregateFunctions.foreach { function => + val queries = Seq( + s"SELECT $function OVER() FROM t1", + s"SELECT $function OVER(order by _2) FROM t1", + s"SELECT $function OVER(order by _2 desc) FROM t1", + s"SELECT $function OVER(partition by _2 order by _2) FROM t1", + s"SELECT $function OVER(rows between 1 preceding and 1 following) FROM t1", + s"SELECT $function OVER(order by _2 rows between 1 preceding and current row) FROM t1", + s"SELECT $function OVER(order by _2 rows between current row and 1 following) FROM t1") + + queries.foreach { query => + checkSparkAnswerAndOperator(query) + } + } + } + }) + } } case class BucketedTableTestSpec( From 68efa57740534990734062e2b18df694f46572e6 Mon Sep 17 00:00:00 2001 From: Parth Chandra Date: Tue, 2 Jul 2024 16:05:45 -0700 Subject: [PATCH 06/16] fix: Improve error "BroadcastExchange is not supported" (#577) * fix: Improve error "BroadcastExchange is not supported" * Update queries list * Add more explanations. Fix plan stability regression. * Make extended explain more user friendly (provide a tree view of plan with info) * fix ci * address review comment * spotless --- .../scala/org/apache/comet/CometConf.scala | 8 + docs/source/user-guide/configs.md | 1 + .../CometTPCDSQueriesList-results.txt | 14165 +++++++++++++++- .../CometTPCHQueriesList-results.txt | 712 +- .../comet/CometSparkSessionExtensions.scala | 92 +- .../apache/comet/ExtendedExplainInfo.scala | 99 +- .../apache/comet/CometExpressionSuite.scala | 7 +- .../spark/sql/CometTPCQueryListBase.scala | 1 + 8 files changed, 14183 insertions(+), 902 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 42fb5fb4c..787e8b4e9 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -274,6 +274,14 @@ object CometConf extends ShimCometConf { .booleanConf .createWithDefault(false) + val COMET_EXPLAIN_VERBOSE_ENABLED: ConfigEntry[Boolean] = + conf("spark.comet.explain.verbose.enabled") + .doc( + "When this setting is enabled, Comet will provide a verbose tree representation of " + + "the extended information.") + .booleanConf + .createWithDefault(false) + val COMET_EXPLAIN_FALLBACK_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.explainFallback.enabled") .doc( diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 104f29ce8..283749076 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -38,6 +38,7 @@ Comet provides the following configuration settings. | spark.comet.exec.shuffle.codec | The codec of Comet native shuffle used to compress shuffle data. Only zstd is supported. | zstd | | spark.comet.exec.shuffle.enabled | Whether to enable Comet native shuffle. By default, this config is false. Note that this requires setting 'spark.shuffle.manager' to 'org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager'. 'spark.shuffle.manager' must be set before starting the Spark application and cannot be changed during the application. | false | | spark.comet.exec.shuffle.mode | The mode of Comet shuffle. This config is only effective if Comet shuffle is enabled. Available modes are 'native', 'jvm', and 'auto'. 'native' is for native shuffle which has best performance in general. 'jvm' is for jvm-based columnar shuffle which has higher coverage than native shuffle. 'auto' is for Comet to choose the best shuffle mode based on the query plan. By default, this config is 'jvm'. | jvm | +| spark.comet.explain.verbose.enabled | When this setting is enabled, Comet will provide a verbose tree representation of the extended information. | false | | spark.comet.explainFallback.enabled | When this setting is enabled, Comet will provide logging explaining the reason(s) why a query stage cannot be executed natively. | false | | spark.comet.memory.overhead.factor | Fraction of executor memory to be allocated as additional non-heap memory per executor process for Comet. Default value is 0.2. | 0.2 | | spark.comet.memory.overhead.min | Minimum amount of additional memory to be allocated per executor process for Comet, in MiB. | 402653184b | diff --git a/spark/inspections/CometTPCDSQueriesList-results.txt b/spark/inspections/CometTPCDSQueriesList-results.txt index cf1af9bed..5d2016a3c 100644 --- a/spark/inspections/CometTPCDSQueriesList-results.txt +++ b/spark/inspections/CometTPCDSQueriesList-results.txt @@ -1,941 +1,13524 @@ -Query: q1. Comet Exec: Enabled (CometFilter, CometProject) +Query: q1. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q1: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -ObjectHashAggregate is not supported -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : +- HashAggregate + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : : +- Subquery + : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store + : : : : +- CometScan parquet spark_catalog.default.store_returns + : : : +- BroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Filter)] + : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_returns + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.customer + Query: q2. Comet Exec: Enabled (CometFilter, CometProject, CometUnion) Query: q2: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -ObjectHashAggregate is not supported -SortMergeJoin disabled because not all child plans are native - -Query: q3. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + :- Sort [COMET: Sort is not native because the following children are not native (Project)] + : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : +- BroadcastExchange + : : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : +- Subquery + : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : :- CometUnion + : : :- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.catalog_sales + : +- BroadcastExchange + : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : +- Subquery + : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometScan parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q3. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q3: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled - -Query: q4. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store_sales + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.item + + +Query: q4. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q4: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q5. Comet Exec: Enabled (CometFilter, CometProject, CometUnion) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Project, Sort)] + :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Project, Sort)] + : :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Project, Sort)] + : : :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (SortMergeJoin, Sort)] + : : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.customer + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.customer + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.customer + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer + : +- CometFilter + : +- CometScan parquet spark_catalog.default.web_sales + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q5. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject, CometUnion) Query: q5: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Expand)] + +- Expand [COMET: Expand is not native because the following children are not native (Union)] + +- Union [COMET: Union is not enabled because the following children are not native (HashAggregate, HashAggregate, HashAggregate)] + :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_returns + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store + :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_returns + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.catalog_page + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometUnion + : : :- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_returns + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.web_site + Query: q6. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q6: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer_address + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- Subquery + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometScan parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + :- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- BroadcastExchange + +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.item + + +Query: q7. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled - -Query: q8. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.promotion + + +Query: q8. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q8: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store + +- BroadcastExchange + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer_address + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Project [COMET: Project is not native because the following children are not native (Filter)] + +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + :- BroadcastExchange + : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : +- Filter [COMET: getstructfield is not supported, xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : :- Subquery + : : : +- Project [COMET: Project is not native because the following children are not native (ObjectHashAggregate)] + : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer_address + : : +- Subquery + : : +- Project [COMET: Project is not native because the following children are not native (ObjectHashAggregate)] + : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer_address + : +- CometScan parquet spark_catalog.default.customer_address + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.customer + Query: q9. Comet Exec: Enabled (CometFilter) Query: q9: ExplainInfo: -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -getstructfield is not supported - -Query: q10. Comet Exec: Enabled (CometFilter, CometProject) + Project [COMET: getstructfield is not supported] +: :- Subquery +: : +- Project [COMET: Project is not native because the following children are not native (HashAggregate)] +: : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : +- CometHashAggregate +: : +- CometProject +: : +- CometFilter +: : +- CometScan parquet spark_catalog.default.store_sales +: :- Subquery +: : +- Project [COMET: Project is not native because the following children are not native (HashAggregate)] +: : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : +- CometHashAggregate +: : +- CometProject +: : +- CometFilter +: : +- CometScan parquet spark_catalog.default.store_sales +: :- Subquery +: : +- Project [COMET: Project is not native because the following children are not native (HashAggregate)] +: : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : +- CometHashAggregate +: : +- CometProject +: : +- CometFilter +: : +- CometScan parquet spark_catalog.default.store_sales +: :- Subquery +: : +- Project [COMET: Project is not native because the following children are not native (HashAggregate)] +: : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : +- CometHashAggregate +: : +- CometProject +: : +- CometFilter +: : +- CometScan parquet spark_catalog.default.store_sales +: :- Subquery +: : +- Project [COMET: Project is not native because the following children are not native (HashAggregate)] +: : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : +- CometHashAggregate +: : +- CometProject +: : +- CometFilter +: : +- CometScan parquet spark_catalog.default.store_sales +: :- Subquery +: : +- Project [COMET: Project is not native because the following children are not native (HashAggregate)] +: : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : +- CometHashAggregate +: : +- CometProject +: : +- CometFilter +: : +- CometScan parquet spark_catalog.default.store_sales +: :- Subquery +: : +- Project [COMET: Project is not native because the following children are not native (HashAggregate)] +: : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : +- CometHashAggregate +: : +- CometProject +: : +- CometFilter +: : +- CometScan parquet spark_catalog.default.store_sales +: :- Subquery +: : +- Project [COMET: Project is not native because the following children are not native (HashAggregate)] +: : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : +- CometHashAggregate +: : +- CometProject +: : +- CometFilter +: : +- CometScan parquet spark_catalog.default.store_sales +: :- Subquery +: : +- Project [COMET: Project is not native because the following children are not native (HashAggregate)] +: : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : +- CometHashAggregate +: : +- CometProject +: : +- CometFilter +: : +- CometScan parquet spark_catalog.default.store_sales +: :- Subquery +: : +- Project [COMET: Project is not native because the following children are not native (HashAggregate)] +: : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : +- CometHashAggregate +: : +- CometProject +: : +- CometFilter +: : +- CometScan parquet spark_catalog.default.store_sales +: :- Subquery +: : +- Project [COMET: Project is not native because the following children are not native (HashAggregate)] +: : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : +- CometHashAggregate +: : +- CometProject +: : +- CometFilter +: : +- CometScan parquet spark_catalog.default.store_sales +: :- Subquery +: : +- Project [COMET: Project is not native because the following children are not native (HashAggregate)] +: : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : +- CometHashAggregate +: : +- CometProject +: : +- CometFilter +: : +- CometScan parquet spark_catalog.default.store_sales +: :- Subquery +: : +- Project [COMET: Project is not native because the following children are not native (HashAggregate)] +: : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : +- CometHashAggregate +: : +- CometProject +: : +- CometFilter +: : +- CometScan parquet spark_catalog.default.store_sales +: :- Subquery +: : +- Project [COMET: Project is not native because the following children are not native (HashAggregate)] +: : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : +- CometHashAggregate +: : +- CometProject +: : +- CometFilter +: : +- CometScan parquet spark_catalog.default.store_sales +: +- Subquery +: +- Project [COMET: Project is not native because the following children are not native (HashAggregate)] +: +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: +- CometHashAggregate +: +- CometProject +: +- CometFilter +: +- CometScan parquet spark_catalog.default.store_sales ++- CometFilter + +- CometScan parquet spark_catalog.default.reason + + +Query: q10. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q10: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q11. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (Filter)] + : : +- Filter [COMET: Filter is not native because the following children are not native (SortMergeJoin)] + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (SortMergeJoin, Sort)] + : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (SortMergeJoin, Sort)] + : : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : : : +- Subquery + : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.customer_address + : : : : : +- CometScan parquet spark_catalog.default.customer + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.customer_demographics + + +Query: q11. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q11: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q12. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Project, Sort)] + :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Project, Sort)] + : :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.customer + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer + : +- CometFilter + : +- CometScan parquet spark_catalog.default.web_sales + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q12. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q12: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled -Window is not supported - -Query: q13. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q13. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q13: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled - -Query: q14a. Comet Exec: Enabled (CometFilter, CometProject) + HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer_demographics + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.household_demographics + + +Query: q14a. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q14a: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native - -Query: q14b. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Expand)] + +- Expand [COMET: Expand is not native because the following children are not native (Union)] + +- Union [COMET: Union is not enabled because the following children are not native (Project, Project, Project)] + :- Project [COMET: Project is not native because the following children are not native (Filter)] + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : +- Subquery + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (SortMergeJoin, BroadcastExchange)] + : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : : +- Subquery + : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- BroadcastExchange + : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- BroadcastExchange + : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + :- Project [COMET: Project is not native because the following children are not native (Filter)] + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : +- Subquery + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (SortMergeJoin, BroadcastExchange)] + : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : : +- Subquery + : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- BroadcastExchange + : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- BroadcastExchange + : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- Project [COMET: Project is not native because the following children are not native (Filter)] + +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- Subquery + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : +- CometUnion + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (SortMergeJoin, BroadcastExchange)] + : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : +- Subquery + : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- BroadcastExchange + : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- BroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- BroadcastExchange + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q14b. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q14b: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native - -Query: q15. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + :- Sort [COMET: Sort is not native because the following children are not native (Filter)] + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : +- Subquery + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (SortMergeJoin, BroadcastExchange)] + : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : : +- Subquery + : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : : +- Subquery + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- BroadcastExchange + : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- BroadcastExchange + : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : : +- Subquery + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometScan parquet spark_catalog.default.date_dim + +- Sort [COMET: Sort is not native because the following children are not native (Filter)] + +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- Subquery + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : +- CometUnion + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (SortMergeJoin, BroadcastExchange)] + : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : +- Subquery + : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- BroadcastExchange + : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- BroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- BroadcastExchange + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometProject + +- CometFilter + : +- Subquery + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q15. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q15: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q16. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q16. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q16: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -ObjectHashAggregate is not supported -SortMergeJoin disabled because not all child plans are native - -Query: q17. Comet Exec: Enabled (CometFilter, CometProject) +HashAggregate ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : +- CometBroadcastExchange + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- BroadcastExchange + : : : +- CometScan parquet spark_catalog.default.catalog_returns + : : +- BroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.call_center + + +Query: q17. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q17: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q18. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_returns + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.item + + +Query: q18. Comet Exec: Enabled (CometExpand, CometBroadcastHashJoin, CometFilter, CometProject) Query: q18: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q19. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true, Comet does not guarantee correct results for cast from DecimalType(7,2) to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.item + + +Query: q19. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q19: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q20. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.store + + +Query: q20. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q20: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled -Window is not supported - -Query: q21. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.catalog_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q21. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q21: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled - -Query: q22. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.inventory + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.warehouse + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q22. Comet Exec: Enabled (CometHashAggregate, CometExpand, CometBroadcastHashJoin, CometFilter, CometProject) Query: q22: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled - -Query: q23a. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.inventory + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.warehouse + + +Query: q23a. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q23a: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -Union disabled because not all child plans are native - -Query: q23b. Comet Exec: Enabled (CometFilter, CometProject) +HashAggregate ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + +- Union [COMET: Union is not enabled because the following children are not native (Project, Project)] + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : : +- Subquery + : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- Sort [COMET: Sort is not native because the following children are not native (Project)] + : : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : +- Subquery + : : : +- HashAggregate + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : : : +- HashAggregate + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : +- Subquery + : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- Sort [COMET: Sort is not native because the following children are not native (Project)] + : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : +- Subquery + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q23b. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q23b: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native - -Query: q24a. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Union [COMET: Union is not enabled because the following children are not native (HashAggregate, HashAggregate)] + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (SortMergeJoin, BroadcastExchange)] + : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : : : +- Subquery + : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : : : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- Sort [COMET: Sort is not native because the following children are not native (Project)] + : : : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : : +- Subquery + : : : : +- HashAggregate + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : : : : +- HashAggregate + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- HashAggregate + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer + : : +- Sort [COMET: Sort is not native because the following children are not native (Project)] + : : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : +- Subquery + : : : +- HashAggregate + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : : : +- HashAggregate + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (SortMergeJoin, BroadcastExchange)] + : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : : +- Subquery + : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- Sort [COMET: Sort is not native because the following children are not native (Project)] + : : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : +- Subquery + : : : +- HashAggregate + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : : : +- HashAggregate + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer + : +- Sort [COMET: Sort is not native because the following children are not native (Project)] + : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : +- Subquery + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q24a. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q24a: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -ObjectHashAggregate is not supported -SortMergeJoin disabled because not all child plans are native - -Query: q24b. Comet Exec: Enabled (CometFilter, CometProject) + Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] +: +- Subquery +: +- HashAggregate +: +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] +: +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: +- CometHashAggregate +: +- CometProject +: +- CometBroadcastHashJoin +: :- CometProject +: : +- CometBroadcastHashJoin +: : :- CometProject +: : : +- CometBroadcastHashJoin +: : : :- CometProject +: : : : +- CometBroadcastHashJoin +: : : : :- CometProject +: : : : : +- CometBroadcastHashJoin +: : : : : :- CometFilter +: : : : : : +- CometScan parquet spark_catalog.default.store_sales +: : : : : +- CometBroadcastExchange +: : : : : +- CometFilter +: : : : : +- CometScan parquet spark_catalog.default.store_returns +: : : : +- CometBroadcastExchange +: : : : +- CometProject +: : : : +- CometFilter +: : : : +- CometScan parquet spark_catalog.default.store +: : : +- CometBroadcastExchange +: : : +- CometProject +: : : +- CometFilter +: : : +- CometScan parquet spark_catalog.default.item +: : +- CometBroadcastExchange +: : +- CometProject +: : +- CometFilter +: : +- CometScan parquet spark_catalog.default.customer +: +- CometBroadcastExchange +: +- CometProject +: +- CometFilter +: +- CometScan parquet spark_catalog.default.customer_address ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.customer_address + + +Query: q24b. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q24b: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -ObjectHashAggregate is not supported -SortMergeJoin disabled because not all child plans are native - -Query: q25. Comet Exec: Enabled (CometFilter, CometProject) + Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] +: +- Subquery +: +- HashAggregate +: +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] +: +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: +- CometHashAggregate +: +- CometProject +: +- CometBroadcastHashJoin +: :- CometProject +: : +- CometBroadcastHashJoin +: : :- CometProject +: : : +- CometBroadcastHashJoin +: : : :- CometProject +: : : : +- CometBroadcastHashJoin +: : : : :- CometProject +: : : : : +- CometBroadcastHashJoin +: : : : : :- CometFilter +: : : : : : +- CometScan parquet spark_catalog.default.store_sales +: : : : : +- CometBroadcastExchange +: : : : : +- CometFilter +: : : : : +- CometScan parquet spark_catalog.default.store_returns +: : : : +- CometBroadcastExchange +: : : : +- CometProject +: : : : +- CometFilter +: : : : +- CometScan parquet spark_catalog.default.store +: : : +- CometBroadcastExchange +: : : +- CometProject +: : : +- CometFilter +: : : +- CometScan parquet spark_catalog.default.item +: : +- CometBroadcastExchange +: : +- CometProject +: : +- CometFilter +: : +- CometScan parquet spark_catalog.default.customer +: +- CometBroadcastExchange +: +- CometProject +: +- CometFilter +: +- CometScan parquet spark_catalog.default.customer_address ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.customer_address + + +Query: q25. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q25: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q26. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_returns + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.item + + +Query: q26. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q26: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled - -Query: q27. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.promotion + + +Query: q27. Comet Exec: Enabled (CometHashAggregate, CometExpand, CometBroadcastHashJoin, CometFilter, CometProject) Query: q27: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.item + Query: q28. Comet Exec: Enabled (CometHashAggregate, CometFilter, CometProject) Query: q28: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled - -Query: q29. Comet Exec: Enabled (CometFilter, CometProject) + BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not native because the following children are not native (BroadcastNestedLoopJoin, BroadcastExchange)] +:- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not native because the following children are not native (BroadcastNestedLoopJoin, BroadcastExchange)] +: :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not native because the following children are not native (BroadcastNestedLoopJoin, BroadcastExchange)] +: : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not native because the following children are not native (BroadcastNestedLoopJoin, BroadcastExchange)] +: : : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not native because the following children are not native (HashAggregate, BroadcastExchange)] +: : : : :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : : : : +- HashAggregate +: : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : : : : +- CometHashAggregate +: : : : : +- CometProject +: : : : : +- CometFilter +: : : : : +- CometScan parquet spark_catalog.default.store_sales +: : : : +- BroadcastExchange +: : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : : : +- HashAggregate +: : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : : : +- CometHashAggregate +: : : : +- CometProject +: : : : +- CometFilter +: : : : +- CometScan parquet spark_catalog.default.store_sales +: : : +- BroadcastExchange +: : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : : +- HashAggregate +: : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : : +- CometHashAggregate +: : : +- CometProject +: : : +- CometFilter +: : : +- CometScan parquet spark_catalog.default.store_sales +: : +- BroadcastExchange +: : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : +- HashAggregate +: : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : +- CometHashAggregate +: : +- CometProject +: : +- CometFilter +: : +- CometScan parquet spark_catalog.default.store_sales +: +- BroadcastExchange +: +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: +- HashAggregate +: +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: +- CometHashAggregate +: +- CometProject +: +- CometFilter +: +- CometScan parquet spark_catalog.default.store_sales ++- BroadcastExchange + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.store_sales + + +Query: q29. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q29: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q30. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_returns + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.item + + +Query: q30. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q30: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q31. Comet Exec: Enabled (CometFilter) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (Project, BroadcastExchange)] + : :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_returns + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer_address + : : +- Sort [COMET: Sort is not native because the following children are not native (Filter)] + : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : +- Subquery + : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer_address + : +- CometScan parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.customer_address + + +Query: q31. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q31: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native - -Query: q32. Comet Exec: Enabled (CometFilter, CometProject) + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Project, Sort)] + :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (SortMergeJoin, Sort)] + : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Project, Sort)] + : : :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (SortMergeJoin, Sort)] + : : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.customer_address + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer_address + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer_address + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer_address + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer_address + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.customer_address + + +Query: q32. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q32: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native - -Query: q33. Comet Exec: Enabled (CometFilter, CometProject) +HashAggregate ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : +- Subquery + : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : +- BroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- Sort [COMET: Sort is not native because the following children are not native (Filter)] + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.catalog_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q33. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q33: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native - -Query: q34. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + +- Union [COMET: Union is not enabled because the following children are not native (HashAggregate, HashAggregate, HashAggregate)] + :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.item + + +Query: q34. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q34: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native - -Query: q35. Comet Exec: Enabled (CometFilter, CometProject) + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.household_demographics + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.customer + + +Query: q35. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q35: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q36. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (Filter)] + : : +- Filter [COMET: Filter is not native because the following children are not native (SortMergeJoin)] + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (SortMergeJoin, Sort)] + : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (SortMergeJoin, Sort)] + : : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.customer + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.customer_demographics + + +Query: q36. Comet Exec: Enabled (CometHashAggregate, CometExpand, CometBroadcastHashJoin, CometFilter, CometProject) Query: q36: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled -Window is not supported - -Query: q37. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.store + + +Query: q37. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q37: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q38. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.inventory + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.catalog_sales + + +Query: q38. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q38: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native - -Query: q39a. Comet Exec: Enabled (CometFilter, CometProject) +HashAggregate ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (SortMergeJoin, Sort)] + :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.customer + + +Query: q39a. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q39a: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native - -Query: q39b. Comet Exec: Enabled (CometFilter, CometProject) + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.inventory + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.warehouse + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Project [COMET: Project is not native because the following children are not native (Filter)] + +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.inventory + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.warehouse + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q39b. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q39b: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native - -Query: q40. Comet Exec: Enabled (CometFilter, CometProject) + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.inventory + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.warehouse + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Project [COMET: Project is not native because the following children are not native (Filter)] + +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.inventory + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.warehouse + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q40. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q40: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q41. Comet Exec: Enabled (CometHashAggregate, CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: ] + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_returns + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.warehouse + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q41. Comet Exec: Enabled (CometFilter, CometProject) Query: q41: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled - -Query: q42. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + :- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- BroadcastExchange + +- Project [COMET: Project is not native because the following children are not native (Filter)] + +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (Filter)] + +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : +- Subquery + : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- CometScan parquet spark_catalog.default.item + + +Query: q42. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q42: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled - -Query: q43. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store_sales + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.item + + +Query: q43. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q43: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store_sales + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.store + Query: q44. Comet Exec: Enabled (CometHashAggregate, CometFilter, CometProject) Query: q44: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Window is not supported - -Query: q45. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (Project, BroadcastExchange)] + : : :- Project [COMET: Project is not native because the following children are not native (Filter)] + : : : +- Filter [COMET: Filter is not native because the following children are not native (Window)] + : : : +- Window [COMET: Window is not supported] + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : : +- Subquery + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- BroadcastExchange + : : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : : +- Filter [COMET: Filter is not native because the following children are not native (Window)] + : : +- Window [COMET: Window is not supported] + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : +- Subquery + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store_sales + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.item + + +Query: q45. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q45: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q46. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (Filter)] + +- Filter [COMET: Filter is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.item + + +Query: q46. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q46: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q47. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.household_demographics + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.customer_address + + +Query: q47. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q47: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Window is not supported - -Query: q48. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Project, Sort)] + :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : : +- Filter [COMET: Filter is not native because the following children are not native (Window)] + : : +- Window [COMET: Window is not supported] + : : +- Filter [COMET: Filter is not native because the following children are not native (Window)] + : : +- Window [COMET: Window is not supported] + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- Project [COMET: Project is not native because the following children are not native (Window)] + : +- Window [COMET: Window is not supported] + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Project [COMET: Project is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.store + + +Query: q48. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q48: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled - -Query: q49. Comet Exec: Enabled (CometFilter, CometProject) + HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer_demographics + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q49. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q49: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native -Window is not supported - -Query: q50. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + +- Union [COMET: Union is not enabled because the following children are not native (Project, Project, Project)] + :- Project [COMET: Project is not native because the following children are not native (Filter)] + : +- Filter [COMET: Filter is not native because the following children are not native (Window)] + : +- Window [COMET: Window is not supported] + : +- Sort [COMET: Sort is not native because the following children are not native (Window)] + : +- Window [COMET: Window is not supported] + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from DecimalType(7,2) to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + :- Project [COMET: Project is not native because the following children are not native (Filter)] + : +- Filter [COMET: Filter is not native because the following children are not native (Window)] + : +- Window [COMET: Window is not supported] + : +- Sort [COMET: Sort is not native because the following children are not native (Window)] + : +- Window [COMET: Window is not supported] + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from DecimalType(7,2) to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- Project [COMET: Project is not native because the following children are not native (Filter)] + +- Filter [COMET: Filter is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: Comet does not guarantee correct results for cast from DecimalType(7,2) to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q50. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q50: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q51. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_returns + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q51. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q51: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Window is not supported - -Query: q52. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Filter [COMET: Filter is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- Project [COMET: Project is not native because the following children are not native (Window)] + : +- Window [COMET: Window is not supported] + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Project [COMET: Project is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan parquet spark_catalog.default.store_sales + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q52. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q52: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled - -Query: q53. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store_sales + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.item + + +Query: q53. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q53: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled -Window is not supported - -Query: q54. Comet Exec: Enabled (CometFilter, CometProject, CometUnion) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (Filter)] + +- Filter [COMET: Filter is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.store + + +Query: q54. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject, CometUnion) Query: q54: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q55. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + +- HashAggregate + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometUnion + : : : : : : : :- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : +- Subquery + : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : +- CometProject + : : : : +- CometFilter + : : : : : :- Subquery + : : : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometHashAggregate + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : +- Subquery + : : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- BroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store + +- BroadcastExchange + +- CometProject + +- CometFilter + : :- Subquery + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- Subquery + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q55. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q55: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled - -Query: q56. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store_sales + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.item + + +Query: q56. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q56: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native - -Query: q57. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + +- Union [COMET: Union is not enabled because the following children are not native (HashAggregate, HashAggregate, HashAggregate)] + :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.item + + +Query: q57. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q57: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Window is not supported - -Query: q58. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Project, Sort)] + :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : : +- Filter [COMET: Filter is not native because the following children are not native (Window)] + : : +- Window [COMET: Window is not supported] + : : +- Filter [COMET: Filter is not native because the following children are not native (Window)] + : : +- Window [COMET: Window is not supported] + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.call_center + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- Project [COMET: Project is not native because the following children are not native (Window)] + : +- Window [COMET: Window is not supported] + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.call_center + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Project [COMET: Project is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.catalog_sales + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.call_center + + +Query: q58. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q58: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Project, Sort)] + :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (Filter)] + : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometScan parquet spark_catalog.default.date_dim + : +- Sort [COMET: Sort is not native because the following children are not native (Filter)] + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- Subquery + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometScan parquet spark_catalog.default.date_dim + +- Sort [COMET: Sort is not native because the following children are not native (Filter)] + +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : +- Subquery + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometScan parquet spark_catalog.default.date_dim + Query: q59. Comet Exec: Enabled (CometFilter, CometProject) Query: q59: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -ObjectHashAggregate is not supported -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q60. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- HashAggregate + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- BroadcastExchange + : : : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : +- Subquery + : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- BroadcastExchange + : : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : +- Subquery + : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q60. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q60: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native - -Query: q61. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + +- Union [COMET: Union is not enabled because the following children are not native (HashAggregate, HashAggregate, HashAggregate)] + :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.item + + +Query: q61. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q61: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -ObjectHashAggregate is not supported -SortMergeJoin disabled because not all child plans are native - -Query: q62. Comet Exec: Enabled (CometFilter, CometProject) + Project [COMET: Project is not native because the following children are not native (BroadcastNestedLoopJoin)] ++- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not native because the following children are not native (HashAggregate, BroadcastExchange)] + :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.promotion + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- BroadcastExchange + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.item + + +Query: q62. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q62: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled - -Query: q63. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.warehouse + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.ship_mode + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.web_site + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q63. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q63: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled -Window is not supported - -Query: q64. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (Filter)] + +- Filter [COMET: Filter is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.store + + +Query: q64. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q64: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native - -Query: q65. Comet Exec: Enabled (CometFilter, CometProject) + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin + : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : +- BroadcastHashJoin + : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : : : : : : : : : : : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : : : : : : : : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : : : : : : : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : : : : : : : : : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : : : : :- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : : : : : : : : : : : : : : : : :- Subquery + : : : : : : : : : : : : : : : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : : : : : : : : : : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : : : : : : : : : : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : : : +- Subquery + : : : : : : : : : : : : : : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : : : : : : : : : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : : : : : : : : : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Project)] + : : : : : : : : : : : : : : : : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : : : : : : : : : : : : : : : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : : : : : : : : : : : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : : : : : : : : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : : : : : : : : : : : +- CometHashAggregate + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : :- CometFilter + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : : : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns + : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store + : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer + : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : : : : : : : : +- BroadcastExchange + : : : : : : : : : +- CometProject + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan parquet spark_catalog.default.promotion + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics + : : : : : : +- BroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.household_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.customer_address + : : : : +- BroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer_address + : : : +- BroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.income_band + : : +- BroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.income_band + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin + : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : +- BroadcastHashJoin + : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : : : : : : : : : : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : : : : : : : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : : : : : : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : : : : : : : : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : : : :- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : : : : : : : : : : : : : : : :- Subquery + : : : : : : : : : : : : : : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : : : : : : : : : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : : : : : : : : : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : : +- Subquery + : : : : : : : : : : : : : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : : : : : : : : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : : : : : : : : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Project)] + : : : : : : : : : : : : : : : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : : : : : : : : : : : : : : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : : : : : : : : : : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : : : : : : : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : : : : : : : : : : +- CometHashAggregate + : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : :- CometFilter + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns + : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store + : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer + : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : : : : : +- BroadcastExchange + : : : : : : : : : +- CometProject + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.promotion + : : : : : : +- BroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.household_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.household_demographics + : : : : +- BroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer_address + : : : +- BroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.income_band + : +- BroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.income_band + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.item + + +Query: q65. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q65: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q66. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.store + : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- Sort [COMET: Sort is not native because the following children are not native (Filter)] + +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan parquet spark_catalog.default.store_sales + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q66. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q66: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native - -Query: q67. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + +- Union [COMET: Union is not enabled because the following children are not native (HashAggregate, HashAggregate)] + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: ] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.warehouse + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.time_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.ship_mode + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: ] + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.warehouse + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.time_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.ship_mode + + +Query: q67. Comet Exec: Enabled (CometExpand, CometBroadcastHashJoin, CometFilter, CometProject) Query: q67: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled -Window is not supported - -Query: q68. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Filter [COMET: Filter is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: ] + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.item + + +Query: q68. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q68: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q69. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.household_demographics + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.customer_address + + +Query: q69. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q69: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q70. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (SortMergeJoin, Sort)] + : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (SortMergeJoin, Sort)] + : : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.customer + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.customer_demographics + + +Query: q70. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q70: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Window is not supported - -Query: q71. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Expand)] + +- Expand [COMET: Expand is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Project)] + :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store + +- Project [COMET: Project is not native because the following children are not native (Filter)] + +- Filter [COMET: Filter is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q71. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject, CometUnion) Query: q71: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -Union disabled because not all child plans are native - -Query: q72. Comet Exec: Enabled (CometFilter, CometProject) + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometUnion + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.time_dim + + +Query: q72. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q72: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -ObjectHashAggregate is not supported -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q73. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometProject + : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : :- CometProject + : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometScan parquet spark_catalog.default.inventory + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan parquet spark_catalog.default.warehouse + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.household_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.promotion + +- BroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.catalog_returns + + +Query: q73. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q73: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native - -Query: q74. Comet Exec: Enabled (CometFilter, CometProject) + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.household_demographics + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.customer + + +Query: q74. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q74: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q75. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Project, Sort)] + :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (SortMergeJoin, Sort)] + : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.customer + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer + : +- CometFilter + : +- CometScan parquet spark_catalog.default.web_sales + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q75. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject, CometUnion) Query: q75: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native - -Query: q76. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometUnion + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.catalog_returns + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store_returns + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.web_returns + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometUnion + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.catalog_returns + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store_returns + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.web_returns + + +Query: q76. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject, CometUnion) Query: q76: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native - -Query: q77. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometUnion + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.catalog_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q77. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q77: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native - -Query: q78. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Expand)] + +- Expand [COMET: Expand is not native because the following children are not native (Union)] + +- Union [COMET: Union is not enabled because the following children are not native (Project, Project, Project)] + :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : +- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_returns + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store + :- Project [COMET: Project is not native because the following children are not native (BroadcastNestedLoopJoin)] + : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not native because the following children are not native (HashAggregate, BroadcastExchange)] + : :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + :- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.web_page + +- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.web_page + + +Query: q78. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q78: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q79. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Project, Sort)] + :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_returns + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- Sort [COMET: Sort is not native because the following children are not native (Filter)] + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- Sort [COMET: Sort is not native because the following children are not native (Filter)] + +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.catalog_sales + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.catalog_returns + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q79. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q79: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q80. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.household_demographics + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.customer + + +Query: q80. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q80: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -ObjectHashAggregate is not supported -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native - -Query: q81. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Expand)] + +- Expand [COMET: Expand is not native because the following children are not native (Union)] + +- Union [COMET: Union is not enabled because the following children are not native (HashAggregate, HashAggregate, HashAggregate)] + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from DecimalType(7,2) to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_returns + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.promotion + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from DecimalType(7,2) to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.catalog_returns + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_page + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.promotion + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: Comet does not guarantee correct results for cast from DecimalType(7,2) to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.web_site + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.promotion + + +Query: q81. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q81: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q82. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (Project, BroadcastExchange)] + : :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.catalog_returns + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer_address + : : +- Sort [COMET: Sort is not native because the following children are not native (Filter)] + : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : +- Subquery + : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer_address + : +- CometScan parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.customer_address + + +Query: q82. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q82: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q83. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.inventory + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.store_sales + + +Query: q83. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q83: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q84. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Project, Sort)] + :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_returns + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q84. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q84: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q85. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: concat is not supported] + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer_address + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.household_demographics + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.income_band + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.store_returns + + +Query: q85. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q85: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q86. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.web_returns + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_page + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.reason + + +Query: q86. Comet Exec: Enabled (CometHashAggregate, CometExpand, CometBroadcastHashJoin, CometFilter, CometProject) Query: q86: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled -Window is not supported - -Query: q87. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.item + + +Query: q87. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q87: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native - -Query: q88. Comet Exec: Enabled (CometFilter, CometProject) +HashAggregate ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (SortMergeJoin, Sort)] + :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.customer + + +Query: q88. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q88: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled - -Query: q89. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) + BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not native because the following children are not native (BroadcastNestedLoopJoin, BroadcastExchange)] +:- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not native because the following children are not native (BroadcastNestedLoopJoin, BroadcastExchange)] +: :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not native because the following children are not native (BroadcastNestedLoopJoin, BroadcastExchange)] +: : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not native because the following children are not native (BroadcastNestedLoopJoin, BroadcastExchange)] +: : : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not native because the following children are not native (BroadcastNestedLoopJoin, BroadcastExchange)] +: : : : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not native because the following children are not native (BroadcastNestedLoopJoin, BroadcastExchange)] +: : : : : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not native because the following children are not native (HashAggregate, BroadcastExchange)] +: : : : : : :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : : : : : : +- CometHashAggregate +: : : : : : : +- CometProject +: : : : : : : +- CometBroadcastHashJoin +: : : : : : : :- CometProject +: : : : : : : : +- CometBroadcastHashJoin +: : : : : : : : :- CometProject +: : : : : : : : : +- CometBroadcastHashJoin +: : : : : : : : : :- CometFilter +: : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales +: : : : : : : : : +- CometBroadcastExchange +: : : : : : : : : +- CometProject +: : : : : : : : : +- CometFilter +: : : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics +: : : : : : : : +- CometBroadcastExchange +: : : : : : : : +- CometProject +: : : : : : : : +- CometFilter +: : : : : : : : +- CometScan parquet spark_catalog.default.time_dim +: : : : : : : +- CometBroadcastExchange +: : : : : : : +- CometProject +: : : : : : : +- CometFilter +: : : : : : : +- CometScan parquet spark_catalog.default.store +: : : : : : +- BroadcastExchange +: : : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : : : : : +- CometHashAggregate +: : : : : : +- CometProject +: : : : : : +- CometBroadcastHashJoin +: : : : : : :- CometProject +: : : : : : : +- CometBroadcastHashJoin +: : : : : : : :- CometProject +: : : : : : : : +- CometBroadcastHashJoin +: : : : : : : : :- CometFilter +: : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales +: : : : : : : : +- CometBroadcastExchange +: : : : : : : : +- CometProject +: : : : : : : : +- CometFilter +: : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics +: : : : : : : +- CometBroadcastExchange +: : : : : : : +- CometProject +: : : : : : : +- CometFilter +: : : : : : : +- CometScan parquet spark_catalog.default.time_dim +: : : : : : +- CometBroadcastExchange +: : : : : : +- CometProject +: : : : : : +- CometFilter +: : : : : : +- CometScan parquet spark_catalog.default.store +: : : : : +- BroadcastExchange +: : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : : : : +- CometHashAggregate +: : : : : +- CometProject +: : : : : +- CometBroadcastHashJoin +: : : : : :- CometProject +: : : : : : +- CometBroadcastHashJoin +: : : : : : :- CometProject +: : : : : : : +- CometBroadcastHashJoin +: : : : : : : :- CometFilter +: : : : : : : : +- CometScan parquet spark_catalog.default.store_sales +: : : : : : : +- CometBroadcastExchange +: : : : : : : +- CometProject +: : : : : : : +- CometFilter +: : : : : : : +- CometScan parquet spark_catalog.default.household_demographics +: : : : : : +- CometBroadcastExchange +: : : : : : +- CometProject +: : : : : : +- CometFilter +: : : : : : +- CometScan parquet spark_catalog.default.time_dim +: : : : : +- CometBroadcastExchange +: : : : : +- CometProject +: : : : : +- CometFilter +: : : : : +- CometScan parquet spark_catalog.default.store +: : : : +- BroadcastExchange +: : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : : : +- CometHashAggregate +: : : : +- CometProject +: : : : +- CometBroadcastHashJoin +: : : : :- CometProject +: : : : : +- CometBroadcastHashJoin +: : : : : :- CometProject +: : : : : : +- CometBroadcastHashJoin +: : : : : : :- CometFilter +: : : : : : : +- CometScan parquet spark_catalog.default.store_sales +: : : : : : +- CometBroadcastExchange +: : : : : : +- CometProject +: : : : : : +- CometFilter +: : : : : : +- CometScan parquet spark_catalog.default.household_demographics +: : : : : +- CometBroadcastExchange +: : : : : +- CometProject +: : : : : +- CometFilter +: : : : : +- CometScan parquet spark_catalog.default.time_dim +: : : : +- CometBroadcastExchange +: : : : +- CometProject +: : : : +- CometFilter +: : : : +- CometScan parquet spark_catalog.default.store +: : : +- BroadcastExchange +: : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : : +- CometHashAggregate +: : : +- CometProject +: : : +- CometBroadcastHashJoin +: : : :- CometProject +: : : : +- CometBroadcastHashJoin +: : : : :- CometProject +: : : : : +- CometBroadcastHashJoin +: : : : : :- CometFilter +: : : : : : +- CometScan parquet spark_catalog.default.store_sales +: : : : : +- CometBroadcastExchange +: : : : : +- CometProject +: : : : : +- CometFilter +: : : : : +- CometScan parquet spark_catalog.default.household_demographics +: : : : +- CometBroadcastExchange +: : : : +- CometProject +: : : : +- CometFilter +: : : : +- CometScan parquet spark_catalog.default.time_dim +: : : +- CometBroadcastExchange +: : : +- CometProject +: : : +- CometFilter +: : : +- CometScan parquet spark_catalog.default.store +: : +- BroadcastExchange +: : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: : +- CometHashAggregate +: : +- CometProject +: : +- CometBroadcastHashJoin +: : :- CometProject +: : : +- CometBroadcastHashJoin +: : : :- CometProject +: : : : +- CometBroadcastHashJoin +: : : : :- CometFilter +: : : : : +- CometScan parquet spark_catalog.default.store_sales +: : : : +- CometBroadcastExchange +: : : : +- CometProject +: : : : +- CometFilter +: : : : +- CometScan parquet spark_catalog.default.household_demographics +: : : +- CometBroadcastExchange +: : : +- CometProject +: : : +- CometFilter +: : : +- CometScan parquet spark_catalog.default.time_dim +: : +- CometBroadcastExchange +: : +- CometProject +: : +- CometFilter +: : +- CometScan parquet spark_catalog.default.store +: +- BroadcastExchange +: +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] +: +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] +: +- CometHashAggregate +: +- CometProject +: +- CometBroadcastHashJoin +: :- CometProject +: : +- CometBroadcastHashJoin +: : :- CometProject +: : : +- CometBroadcastHashJoin +: : : :- CometFilter +: : : : +- CometScan parquet spark_catalog.default.store_sales +: : : +- CometBroadcastExchange +: : : +- CometProject +: : : +- CometFilter +: : : +- CometScan parquet spark_catalog.default.household_demographics +: : +- CometBroadcastExchange +: : +- CometProject +: : +- CometFilter +: : +- CometScan parquet spark_catalog.default.time_dim +: +- CometBroadcastExchange +: +- CometProject +: +- CometFilter +: +- CometScan parquet spark_catalog.default.store ++- BroadcastExchange + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.household_demographics + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.time_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.store + + +Query: q89. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q89: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled -Window is not supported - -Query: q90. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (Filter)] + +- Filter [COMET: Filter is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.store + + +Query: q90. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q90: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled - -Query: q91. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) + Project [COMET: Project is not native because the following children are not native (BroadcastNestedLoopJoin)] ++- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not native because the following children are not native (HashAggregate, BroadcastExchange)] + :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.household_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.time_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.web_page + +- BroadcastExchange + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.household_demographics + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.time_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.web_page + + +Query: q91. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q91: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -ObjectHashAggregate is not supported -SortMergeJoin disabled because not all child plans are native - -Query: q92. Comet Exec: Enabled (CometFilter, CometProject) + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.call_center + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.catalog_returns + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.household_demographics + + +Query: q92. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q92: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native - -Query: q93. Comet Exec: Enabled (CometFilter, CometProject) +HashAggregate ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : +- Subquery + : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- BroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- Sort [COMET: Sort is not native because the following children are not native (Filter)] + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q93. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q93: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -ObjectHashAggregate is not supported -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q94. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: ] + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometScan parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.reason + + +Query: q94. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q94: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -ObjectHashAggregate is not supported -SortMergeJoin disabled because not all child plans are native - -Query: q95. Comet Exec: Enabled (CometFilter, CometProject) +HashAggregate ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : +- CometBroadcastExchange + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- BroadcastExchange + : : : +- CometScan parquet spark_catalog.default.web_returns + : : +- BroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.web_site + + +Query: q95. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q95: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -ObjectHashAggregate is not supported -SortMergeJoin disabled because not all child plans are native - -Query: q96. Comet Exec: Enabled (CometFilter, CometProject) +HashAggregate ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (SortMergeJoin, Sort)] + : : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_returns + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- BroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.web_site + + +Query: q96. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q96: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled - -Query: q97. Comet Exec: Enabled (CometFilter, CometProject) + HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.household_demographics + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.time_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.store + + +Query: q97. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q97: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native - -Query: q98. Comet Exec: Enabled (CometFilter, CometProject) +HashAggregate ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + :- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan parquet spark_catalog.default.catalog_sales + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q98. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q98: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -Window is not supported - -Query: q99. Comet Exec: Enabled (CometFilter, CometProject) + Project [COMET: Project is not native because the following children are not native (Sort)] ++- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Project [COMET: Project is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q99. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q99: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled - -Query: q5a-v2.7. Comet Exec: Enabled (CometFilter, CometProject, CometUnion) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.warehouse + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.ship_mode + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.call_center + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q5a-v2.7. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject, CometUnion) Query: q5a-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + +- Union [COMET: Union is not enabled because the following children are not native (HashAggregate, HashAggregate, HashAggregate)] + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + : +- Union [COMET: Union is not enabled because the following children are not native (HashAggregate, HashAggregate, HashAggregate)] + : :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.catalog_page + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_returns + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.web_site + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + : +- Union [COMET: Union is not enabled because the following children are not native (HashAggregate, HashAggregate, HashAggregate)] + : :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.catalog_page + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_returns + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.web_site + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + +- Union [COMET: Union is not enabled because the following children are not native (HashAggregate, HashAggregate, HashAggregate)] + :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_returns + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store + :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_returns + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.catalog_page + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometUnion + : : :- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_returns + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.web_site + Query: q6-v2.7. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q6-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q10a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer_address + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- Subquery + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometScan parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + :- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- BroadcastExchange + +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.item + + +Query: q10a-v2.7. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject, CometUnion) Query: q10a-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native - -Query: q11-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (SortMergeJoin, Sort)] + : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.customer_demographics + + +Query: q11-v2.7. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q11-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q12-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Project, Sort)] + :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (SortMergeJoin, Sort)] + : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.customer + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer + : +- CometFilter + : +- CometScan parquet spark_catalog.default.web_sales + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q12-v2.7. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q12-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled -Window is not supported - -Query: q14-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q14-v2.7. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q14-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native - -Query: q14a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + :- Sort [COMET: Sort is not native because the following children are not native (Filter)] + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : +- Subquery + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (SortMergeJoin, BroadcastExchange)] + : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : : +- Subquery + : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : : +- Subquery + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- BroadcastExchange + : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- BroadcastExchange + : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : : +- Subquery + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometScan parquet spark_catalog.default.date_dim + +- Sort [COMET: Sort is not native because the following children are not native (Filter)] + +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- Subquery + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : +- CometUnion + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (SortMergeJoin, BroadcastExchange)] + : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : +- Subquery + : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- BroadcastExchange + : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- BroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- BroadcastExchange + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometProject + +- CometFilter + : +- Subquery + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q14a-v2.7. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q14a-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native - -Query: q18a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + +- Union [COMET: Union is not enabled because the following children are not native (HashAggregate, HashAggregate, HashAggregate, HashAggregate, HashAggregate)] + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + : +- Union [COMET: Union is not enabled because the following children are not native (Filter, Filter, Filter)] + : :- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : +- Subquery + : : : +- HashAggregate + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (SortMergeJoin, BroadcastExchange)] + : : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : : : +- Subquery + : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- BroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin + : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : : +- BroadcastExchange + : : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : : +- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- BroadcastExchange + : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : :- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : +- Subquery + : : : +- HashAggregate + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (SortMergeJoin, BroadcastExchange)] + : : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : : : +- Subquery + : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- BroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin + : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : : +- BroadcastExchange + : : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : : +- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- BroadcastExchange + : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : +- Subquery + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (SortMergeJoin, BroadcastExchange)] + : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : : +- Subquery + : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- BroadcastExchange + : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- BroadcastExchange + : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + : +- Union [COMET: Union is not enabled because the following children are not native (Filter, Filter, Filter)] + : :- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : +- Subquery + : : : +- HashAggregate + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (SortMergeJoin, BroadcastExchange)] + : : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : : : +- Subquery + : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- BroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin + : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : : +- BroadcastExchange + : : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : : +- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- BroadcastExchange + : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : :- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : +- Subquery + : : : +- HashAggregate + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (SortMergeJoin, BroadcastExchange)] + : : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : : : +- Subquery + : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- BroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin + : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : : +- BroadcastExchange + : : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : : +- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- BroadcastExchange + : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : +- Subquery + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (SortMergeJoin, BroadcastExchange)] + : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : : +- Subquery + : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- BroadcastExchange + : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- BroadcastExchange + : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + : +- Union [COMET: Union is not enabled because the following children are not native (Filter, Filter, Filter)] + : :- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : +- Subquery + : : : +- HashAggregate + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (SortMergeJoin, BroadcastExchange)] + : : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : : : +- Subquery + : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- BroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin + : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : : +- BroadcastExchange + : : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : : +- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- BroadcastExchange + : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : :- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : +- Subquery + : : : +- HashAggregate + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (SortMergeJoin, BroadcastExchange)] + : : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : : : +- Subquery + : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- BroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin + : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : : +- BroadcastExchange + : : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : : +- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- BroadcastExchange + : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : +- Subquery + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (SortMergeJoin, BroadcastExchange)] + : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : : +- Subquery + : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- BroadcastExchange + : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- BroadcastExchange + : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + : +- Union [COMET: Union is not enabled because the following children are not native (Filter, Filter, Filter)] + : :- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : +- Subquery + : : : +- HashAggregate + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (SortMergeJoin, BroadcastExchange)] + : : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : : : +- Subquery + : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- BroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin + : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : : +- BroadcastExchange + : : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : : +- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- BroadcastExchange + : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : :- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : +- Subquery + : : : +- HashAggregate + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (SortMergeJoin, BroadcastExchange)] + : : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- BroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin + : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : : +- BroadcastExchange + : : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : : +- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- BroadcastExchange + : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : +- Subquery + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (SortMergeJoin, BroadcastExchange)] + : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- BroadcastExchange + : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- BroadcastExchange + : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + +- Union [COMET: Union is not enabled because the following children are not native (Filter, Filter, Filter)] + :- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : +- Subquery + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (SortMergeJoin, BroadcastExchange)] + : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- BroadcastExchange + : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- BroadcastExchange + : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + :- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : +- Subquery + : : +- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (SortMergeJoin, BroadcastExchange)] + : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- BroadcastExchange + : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- BroadcastExchange + : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- Subquery + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : +- CometUnion + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (SortMergeJoin, BroadcastExchange)] + : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- BroadcastExchange + : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- BroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- BroadcastExchange + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q18a-v2.7. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q18a-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native - -Query: q20-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Union [COMET: Union is not enabled because the following children are not native (HashAggregate, HashAggregate, HashAggregate, HashAggregate, HashAggregate)] + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : +- Project [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true, Comet does not guarantee correct results for cast from DecimalType(7,2) to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : +- Project [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true, Comet does not guarantee correct results for cast from DecimalType(7,2) to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : +- Project [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true, Comet does not guarantee correct results for cast from DecimalType(7,2) to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : +- Project [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true, Comet does not guarantee correct results for cast from DecimalType(7,2) to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true, Comet does not guarantee correct results for cast from DecimalType(7,2) to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.item + + +Query: q20-v2.7. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q20-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled -Window is not supported - -Query: q22-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.catalog_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q22-v2.7. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q22-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled - -Query: q22a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Expand)] + +- Expand [COMET: Expand is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastNestedLoopJoin)] + +- BroadcastNestedLoopJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.inventory + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- BroadcastExchange + +- CometScan parquet spark_catalog.default.warehouse + + +Query: q22a-v2.7. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q22a-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native - -Query: q24-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Union [COMET: Union is not enabled because the following children are not native (HashAggregate, HashAggregate, HashAggregate, HashAggregate, HashAggregate)] + :- HashAggregate + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.inventory + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.warehouse + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.inventory + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.warehouse + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.inventory + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.warehouse + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.inventory + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.warehouse + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.inventory + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.warehouse + + +Query: q24-v2.7. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q24-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -ObjectHashAggregate is not supported -SortMergeJoin disabled because not all child plans are native - -Query: q27a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- Subquery + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_returns + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer_address + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.customer_address + + +Query: q27a-v2.7. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q27a-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native - -Query: q34-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Union [COMET: Union is not enabled because the following children are not native (HashAggregate, HashAggregate, HashAggregate)] + :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.item + + +Query: q34-v2.7. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q34-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native - -Query: q35-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.household_demographics + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.customer + + +Query: q35-v2.7. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q35-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q35a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (Filter)] + : : +- Filter [COMET: Filter is not native because the following children are not native (SortMergeJoin)] + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (SortMergeJoin, Sort)] + : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (SortMergeJoin, Sort)] + : : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.customer + : : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.customer_demographics + + +Query: q35a-v2.7. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject, CometUnion) Query: q35a-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native - -Query: q36a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (SortMergeJoin, Sort)] + : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.customer_demographics + + +Query: q36a-v2.7. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q36a-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native -Window is not supported - -Query: q47-v2.7. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + +- Union [COMET: Union is not enabled because the following children are not native (HashAggregate, HashAggregate, HashAggregate)] + :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.store + + +Query: q47-v2.7. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q47-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Window is not supported - -Query: q49-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Project, Sort)] + :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : : +- Filter [COMET: Filter is not native because the following children are not native (Window)] + : : +- Window [COMET: Window is not supported] + : : +- Filter [COMET: Filter is not native because the following children are not native (Window)] + : : +- Window [COMET: Window is not supported] + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- Project [COMET: Project is not native because the following children are not native (Window)] + : +- Window [COMET: Window is not supported] + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Project [COMET: Project is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.store + + +Query: q49-v2.7. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q49-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native -Window is not supported - -Query: q51a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + +- Union [COMET: Union is not enabled because the following children are not native (Project, Project, Project)] + :- Project [COMET: Project is not native because the following children are not native (Filter)] + : +- Filter [COMET: Filter is not native because the following children are not native (Window)] + : +- Window [COMET: Window is not supported] + : +- Sort [COMET: Sort is not native because the following children are not native (Window)] + : +- Window [COMET: Window is not supported] + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from DecimalType(7,2) to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + :- Project [COMET: Project is not native because the following children are not native (Filter)] + : +- Filter [COMET: Filter is not native because the following children are not native (Window)] + : +- Window [COMET: Window is not supported] + : +- Sort [COMET: Sort is not native because the following children are not native (Window)] + : +- Window [COMET: Window is not supported] + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from DecimalType(7,2) to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- Project [COMET: Project is not native because the following children are not native (Filter)] + +- Filter [COMET: Filter is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: Comet does not guarantee correct results for cast from DecimalType(7,2) to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q51a-v2.7. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q51a-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Window is not supported - -Query: q57-v2.7. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + +- HashAggregate + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Window, Project)] + :- Window [COMET: Window is not supported] + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : +- Filter [COMET: Filter is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : : +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Project [COMET: Project is not native because the following children are not native (Window)] + : : : +- Window [COMET: Window is not supported] + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Project [COMET: Project is not native because the following children are not native (Window)] + : : +- Window [COMET: Window is not supported] + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Project [COMET: Project is not native because the following children are not native (Window)] + : : +- Window [COMET: Window is not supported] + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- Project [COMET: Project is not native because the following children are not native (Window)] + : +- Window [COMET: Window is not supported] + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- Project [COMET: Project is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Project [COMET: Project is not native because the following children are not native (Filter)] + +- Filter [COMET: Filter is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Project [COMET: Project is not native because the following children are not native (Window)] + : : +- Window [COMET: Window is not supported] + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- Project [COMET: Project is not native because the following children are not native (Window)] + : +- Window [COMET: Window is not supported] + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- Project [COMET: Project is not native because the following children are not native (Window)] + : +- Window [COMET: Window is not supported] + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Project [COMET: Project is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan parquet spark_catalog.default.store_sales + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q57-v2.7. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q57-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Window is not supported - -Query: q64-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Project, Sort)] + :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : : +- Filter [COMET: Filter is not native because the following children are not native (Window)] + : : +- Window [COMET: Window is not supported] + : : +- Filter [COMET: Filter is not native because the following children are not native (Window)] + : : +- Window [COMET: Window is not supported] + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.item + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.call_center + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- Project [COMET: Project is not native because the following children are not native (Window)] + : +- Window [COMET: Window is not supported] + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.call_center + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Project [COMET: Project is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.catalog_sales + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.call_center + + +Query: q64-v2.7. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q64-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native - -Query: q67a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin + : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : +- BroadcastHashJoin + : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : : : : : : : : : : : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : : : : : : : : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : : : : : : : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : : : : : : : : : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : : : : :- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : : : : : : : : : : : : : : : : :- Subquery + : : : : : : : : : : : : : : : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : : : : : : : : : : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : : : : : : : : : : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : : : +- Subquery + : : : : : : : : : : : : : : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : : : : : : : : : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : : : : : : : : : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Project)] + : : : : : : : : : : : : : : : : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : : : : : : : : : : : : : : : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : : : : : : : : : : : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : : : : : : : : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : : : : : : : : : : : +- CometHashAggregate + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : :- CometFilter + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : : : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns + : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store + : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer + : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : : : : : : : : +- BroadcastExchange + : : : : : : : : : +- CometProject + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan parquet spark_catalog.default.promotion + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics + : : : : : : +- BroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.household_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.customer_address + : : : : +- BroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer_address + : : : +- BroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.income_band + : : +- BroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.income_band + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : +- BroadcastHashJoin + : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : +- BroadcastHashJoin + : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : +- BroadcastHashJoin + : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : +- BroadcastHashJoin + : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : : : : : : : : : : : : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : : : : : : : : : : : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : : : : : : : : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : : : : : : : : : : : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : : : :- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : : : : : : : : : : : : : : : :- Subquery + : : : : : : : : : : : : : : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : : : : : : : : : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : : : : : : : : : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : : +- Subquery + : : : : : : : : : : : : : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : : : : : : : : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : : : : : : : : : : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : +- Sort [COMET: Sort is not native because the following children are not native (Project)] + : : : : : : : : : : : : : : : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : : : : : : : : : : : : : : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : : : : : : : : : : : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : : : : : : : : : : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : : : : : : : : : : : : +- CometHashAggregate + : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : :- CometFilter + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns + : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store + : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer + : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : : : : : : +- BroadcastExchange + : : : : : : : : : +- CometProject + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.promotion + : : : : : : +- BroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.household_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.household_demographics + : : : : +- BroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer_address + : : : +- BroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.income_band + : +- BroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.income_band + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.item + + +Query: q67a-v2.7. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q67a-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native -Window is not supported - -Query: q70a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Filter [COMET: Filter is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Union [COMET: Union is not enabled because the following children are not native (HashAggregate, HashAggregate, HashAggregate, HashAggregate, HashAggregate, HashAggregate, HashAggregate, HashAggregate, HashAggregate)] + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: ] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: ] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: ] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: ] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: ] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: ] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: ] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: ] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: ] + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.item + + +Query: q70a-v2.7. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q70a-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native -Window is not supported - -Query: q72-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + +- Union [COMET: Union is not enabled because the following children are not native (HashAggregate, HashAggregate, HashAggregate)] + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Project)] + : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : +- Filter [COMET: Filter is not native because the following children are not native (Window)] + : +- Window [COMET: Window is not supported] + : +- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Project)] + : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : +- Filter [COMET: Filter is not native because the following children are not native (Window)] + : +- Window [COMET: Window is not supported] + : +- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Project)] + :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store + +- Project [COMET: Project is not native because the following children are not native (Filter)] + +- Filter [COMET: Filter is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q72-v2.7. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q72-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -ObjectHashAggregate is not supported -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q74-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometProject + : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : :- CometProject + : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometScan parquet spark_catalog.default.inventory + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan parquet spark_catalog.default.warehouse + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.household_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.promotion + +- BroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.catalog_returns + + +Query: q74-v2.7. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q74-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q75-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Project, Sort)] + :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (SortMergeJoin, Sort)] + : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.customer + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.customer + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.customer + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.customer + : +- CometFilter + : +- CometScan parquet spark_catalog.default.web_sales + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q75-v2.7. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject, CometUnion) Query: q75-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native - -Query: q77a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometUnion + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.catalog_returns + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store_returns + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.web_returns + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometUnion + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.catalog_returns + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store_returns + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.web_returns + + +Query: q77a-v2.7. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q77a-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native - -Query: q78-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + +- Union [COMET: Union is not enabled because the following children are not native (HashAggregate, HashAggregate, HashAggregate)] + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + : +- Union [COMET: Union is not enabled because the following children are not native (Project, Project, Project)] + : :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.store + : : +- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : :- Project [COMET: Project is not native because the following children are not native (BroadcastNestedLoopJoin)] + : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not native because the following children are not native (HashAggregate, BroadcastExchange)] + : : :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_returns + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.web_page + : +- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_returns + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.web_page + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + : +- Union [COMET: Union is not enabled because the following children are not native (Project, Project, Project)] + : :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.store + : : +- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : :- Project [COMET: Project is not native because the following children are not native (BroadcastNestedLoopJoin)] + : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not native because the following children are not native (HashAggregate, BroadcastExchange)] + : : :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_returns + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.web_page + : +- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_returns + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.web_page + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + +- Union [COMET: Union is not enabled because the following children are not native (Project, Project, Project)] + :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.store + : +- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_returns + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.store + :- Project [COMET: Project is not native because the following children are not native (BroadcastNestedLoopJoin)] + : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not native because the following children are not native (HashAggregate, BroadcastExchange)] + : :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + :- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.web_page + +- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.web_page + + +Query: q78-v2.7. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q78-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled - -Query: q80a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Project, Sort)] + :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (HashAggregate)] + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.store_returns + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- Sort [COMET: Sort is not native because the following children are not native (Filter)] + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- Sort [COMET: Sort is not native because the following children are not native (Filter)] + +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.catalog_sales + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet spark_catalog.default.catalog_returns + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + + +Query: q80a-v2.7. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q80a-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -ObjectHashAggregate is not supported -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native - -Query: q86a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + +- Union [COMET: Union is not enabled because the following children are not native (HashAggregate, HashAggregate, HashAggregate)] + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + : +- Union [COMET: Union is not enabled because the following children are not native (HashAggregate, HashAggregate, HashAggregate)] + : :- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from DecimalType(7,2) to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_returns + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.promotion + : :- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from DecimalType(7,2) to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_page + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.promotion + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from DecimalType(7,2) to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_returns + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_site + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.promotion + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + : +- Union [COMET: Union is not enabled because the following children are not native (HashAggregate, HashAggregate, HashAggregate)] + : :- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from DecimalType(7,2) to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_returns + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.promotion + : :- HashAggregate + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from DecimalType(7,2) to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.catalog_page + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.promotion + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from DecimalType(7,2) to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_returns + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_site + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.promotion + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + +- Union [COMET: Union is not enabled because the following children are not native (HashAggregate, HashAggregate, HashAggregate)] + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from DecimalType(7,2) to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.store_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.store_returns + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.promotion + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from DecimalType(7,2) to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.catalog_returns + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.catalog_page + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.promotion + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: Comet does not guarantee correct results for cast from DecimalType(7,2) to DecimalType(12,2) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan parquet spark_catalog.default.web_sales + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet spark_catalog.default.web_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.web_site + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.promotion + + +Query: q86a-v2.7. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q86a-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -TakeOrderedAndProject requires shuffle to be enabled -Union disabled because not all child plans are native -Window is not supported - -Query: q98-v2.7. Comet Exec: Enabled (CometFilter, CometProject) + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Union)] + +- Union [COMET: Union is not enabled because the following children are not native (HashAggregate, HashAggregate, HashAggregate)] + :- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + :- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.item + + +Query: q98-v2.7. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q98-v2.7: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -Window is not supported + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Project [COMET: Project is not native because the following children are not native (Window)] + +- Window [COMET: Window is not supported] + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet spark_catalog.default.date_dim + diff --git a/spark/inspections/CometTPCHQueriesList-results.txt b/spark/inspections/CometTPCHQueriesList-results.txt index ce896ba82..892b6515a 100644 --- a/spark/inspections/CometTPCHQueriesList-results.txt +++ b/spark/inspections/CometTPCHQueriesList-results.txt @@ -1,135 +1,679 @@ -Query: q1 TPCH Snappy. Comet Exec: Enabled (CometHashAggregate, CometProject) +Query: q1 TPCH Snappy. Comet Exec: Enabled (CometHashAggregate, CometFilter, CometProject) Query: q1 TPCH Snappy: ExplainInfo: -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometScan parquet -Query: q2 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) + +Query: q2 TPCH Snappy. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q2 TPCH Snappy: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -ObjectHashAggregate is not supported -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (Project, BroadcastExchange)] + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet + : : : : +- CometFilter + : : : : +- CometScan parquet + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan parquet + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet + : : +- BroadcastExchange + : : +- CometFilter + : : +- CometScan parquet + : +- BroadcastExchange + : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : +- Subquery + : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet + : +- CometScan parquet + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet + -Query: q3 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) +Query: q3 TPCH Snappy. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q3 TPCH Snappy: ExplainInfo: -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet + : +- CometFilter + : +- CometScan parquet + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometProject + +- CometFilter + +- CometScan parquet + -Query: q4 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) +Query: q4 TPCH Snappy. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q4 TPCH Snappy: ExplainInfo: -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometScan parquet + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet -Query: q5 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) + +Query: q5 TPCH Snappy. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q5 TPCH Snappy: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -ObjectHashAggregate is not supported -SortMergeJoin disabled because not all child plans are native + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (Project, BroadcastExchange)] + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin + : : :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometFilter + : : : +- CometScan parquet + : : +- BroadcastExchange + : : +- CometFilter + : : +- CometScan parquet + : +- BroadcastExchange + : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : +- Subquery + : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet + : +- CometScan parquet + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet + Query: q6 TPCH Snappy. Comet Exec: Enabled (CometHashAggregate, CometFilter, CometProject) Query: q6 TPCH Snappy: ExplainInfo: -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled + HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometScan parquet + -Query: q7 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) +Query: q7 TPCH Snappy. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q7 TPCH Snappy: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -ObjectHashAggregate is not supported -SortMergeJoin disabled because not all child plans are native + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet + : : : : +- CometFilter + : : : : +- CometScan parquet + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet -Query: q8 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) + +Query: q8 TPCH Snappy. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q8 TPCH Snappy: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -ObjectHashAggregate is not supported -SortMergeJoin disabled because not all child plans are native + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan parquet + : : : : : : +- CometFilter + : : : : : : +- CometScan parquet + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan parquet + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan parquet + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet + -Query: q9 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) +Query: q9 TPCH Snappy. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q9 TPCH Snappy: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan parquet + : : : : +- CometFilter + : : : : +- CometScan parquet + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometFilter + : : +- CometScan parquet + : +- BroadcastExchange + : +- CometFilter + : +- CometScan parquet + +- BroadcastExchange + +- CometFilter + +- CometScan parquet + -Query: q10 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) +Query: q10 TPCH Snappy. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q10 TPCH Snappy: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan parquet + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet + : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometProject + : +- CometFilter + : +- CometScan parquet + +- BroadcastExchange + +- CometFilter + +- CometScan parquet -Query: q11 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) + +Query: q11 TPCH Snappy. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q11 TPCH Snappy: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -ObjectHashAggregate is not supported -SortMergeJoin disabled because not all child plans are native + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- Subquery + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan parquet + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: Comet does not guarantee correct results for cast from IntegerType to DecimalType(10,0) with timezone Some(America/Los_Angeles) and evalMode LEGACY (No overflow check). To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet + -Query: q12 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) +Query: q12 TPCH Snappy. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q12 TPCH Snappy: ExplainInfo: -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometBroadcastExchange + : +- CometFilter + : +- CometScan parquet + +- CometProject + +- CometFilter + +- CometScan parquet + -Query: q13 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) +Query: q13 TPCH Snappy. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q13 TPCH Snappy: ExplainInfo: -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometScan parquet + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet -Query: q14 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) + +Query: q14 TPCH Snappy. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q14 TPCH Snappy: ExplainInfo: -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native + HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometScan parquet + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet + Query: q15 TPCH Snappy. Comet Exec: Enabled (CometHashAggregate, CometFilter, CometProject) Query: q15 TPCH Snappy: ExplainInfo: -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- BroadcastExchange + : +- CometFilter + : +- CometScan parquet + +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- Subquery + : +- HashAggregate + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometProject + : +- CometFilter + : +- CometScan parquet + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometScan parquet + Query: q16 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) Query: q16 TPCH Snappy: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- BroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet + +- BroadcastExchange + +- CometFilter + +- CometScan parquet -Query: q17 TPCH Snappy. Comet Exec: Enabled (CometHashAggregate, CometFilter, CometProject) + +Query: q17 TPCH Snappy. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q17 TPCH Snappy: ExplainInfo: -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native +HashAggregate ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan parquet + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet + +- Sort [COMET: Sort is not native because the following children are not native (Filter)] + +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometFilter + +- CometScan parquet + -Query: q18 TPCH Snappy. Comet Exec: Enabled (CometHashAggregate, CometFilter, CometProject) +Query: q18 TPCH Snappy. Comet Exec: Enabled (CometHashAggregate, CometFilter) Query: q18 TPCH Snappy: ExplainInfo: -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- BroadcastExchange + : : +- CometFilter + : : +- CometScan parquet + : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + : :- CometFilter + : : +- CometScan parquet + : +- BroadcastExchange + : +- Project [COMET: Project is not native because the following children are not native (Filter)] + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometScan parquet + +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange)] + :- CometFilter + : +- CometScan parquet + +- BroadcastExchange + +- Project [COMET: Project is not native because the following children are not native (Filter)] + +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometScan parquet + -Query: q19 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) +Query: q19 TPCH Snappy. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q19 TPCH Snappy: ExplainInfo: -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native + HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometScan parquet + +- CometBroadcastExchange + +- CometFilter + +- CometScan parquet -Query: q20 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) + +Query: q20 TPCH Snappy. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q20 TPCH Snappy: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -ObjectHashAggregate is not supported -SortMergeJoin disabled because not all child plans are native + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (Filter, BroadcastExchange)] + : :- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : +- Subquery + : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan parquet + : : +- CometScan parquet + : +- BroadcastExchange + : +- Project [COMET: Project is not native because the following children are not native (SortMergeJoin)] + : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan parquet + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet + : +- Sort [COMET: Sort is not native because the following children are not native (Filter)] + : +- Filter [COMET: Filter is not native because the following children are not native (HashAggregate)] + : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : +- CometHashAggregate + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometScan parquet + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet + Query: q21 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) Query: q21 TPCH Snappy: ExplainInfo: -BroadcastExchange is not supported -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -ObjectHashAggregate is not supported -SortMergeJoin disabled because not all child plans are native -TakeOrderedAndProject requires shuffle to be enabled + TakeOrderedAndProject [COMET: TakeOrderedAndProject requires shuffle to be enabled] ++- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : +- BroadcastHashJoin + : :- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + : : +- BroadcastHashJoin [COMET: BroadcastHashJoin is not native because the following children are not native (BroadcastExchange, SortMergeJoin)] + : : :- BroadcastExchange + : : : +- Filter [COMET: xxhash64 is disabled by default. Set spark.comet.xxhash64.enabled=true to enable it.] + : : : : +- Subquery + : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- ObjectHashAggregate [COMET: ObjectHashAggregate is not supported] + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet + : : : +- CometScan parquet + : : +- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (SortMergeJoin, Sort)] + : : :- SortMergeJoin [COMET: SortMergeJoin is not enabled because the following children are not native (Sort, Sort)] + : : : :- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan parquet + : : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : : +- CometScan parquet + : : +- Sort [COMET: Sort is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet + : +- BroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan parquet + +- BroadcastExchange + +- CometProject + +- CometFilter + +- CometScan parquet + -Query: q22 TPCH Snappy. Comet Exec: Enabled (CometProject) +Query: q22 TPCH Snappy. Comet Exec: Disabled Query: q22 TPCH Snappy: ExplainInfo: -Comet does not guarantee correct results for cast from DecimalType(12,2) to DecimalType(16,6) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -SortMergeJoin disabled because not all child plans are native + Sort [COMET: Sort is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate + +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Project)] + +- Project [COMET: Project is not native because the following children are not native (BroadcastHashJoin)] + +- BroadcastHashJoin + :- Filter [COMET: Comet does not guarantee correct results for cast from DecimalType(12,2) to DecimalType(16,6) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true] + : : +- Subquery + : : +- HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] + : : +- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometScan parquet + : +- CometScan parquet + +- BroadcastExchange + +- CometScan parquet + Query: q1 TPCH Extended Snappy. Comet Exec: Enabled (CometHashAggregate, CometFilter, CometProject) Query: q1 TPCH Extended Snappy: ExplainInfo: -Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled + HashAggregate [COMET: HashAggregate is not native because the following children are not native (Exchange)] ++- Exchange [COMET: Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled] + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometScan parquet + diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index ffb7de83b..fb1f62542 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -45,7 +45,8 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.comet.CometConf._ -import org.apache.comet.CometSparkSessionExtensions.{createMessage, getCometShuffleNotEnabledReason, isANSIEnabled, isCometBroadCastForceEnabled, isCometEnabled, isCometExecEnabled, isCometJVMShuffleMode, isCometNativeShuffleMode, isCometOperatorEnabled, isCometScan, isCometScanEnabled, isCometShuffleEnabled, isSchemaSupported, isSpark34Plus, isSpark40Plus, shouldApplyRowToColumnar, withInfo, withInfos} +import org.apache.comet.CometExplainInfo.getActualPlan +import org.apache.comet.CometSparkSessionExtensions.{createMessage, getCometBroadcastNotEnabledReason, getCometShuffleNotEnabledReason, isANSIEnabled, isCometBroadCastForceEnabled, isCometEnabled, isCometExecEnabled, isCometJVMShuffleMode, isCometNativeShuffleMode, isCometOperatorEnabled, isCometScan, isCometScanEnabled, isCometShuffleEnabled, isSchemaSupported, isSpark34Plus, isSpark40Plus, shouldApplyRowToColumnar, withInfo, withInfos} import org.apache.comet.parquet.{CometParquetScan, SupportsComet} import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.QueryPlanSerde @@ -222,6 +223,19 @@ class CometSparkSessionExtensions private def isCometNative(op: SparkPlan): Boolean = op.isInstanceOf[CometNativeExec] + private def explainChildNotNative(op: SparkPlan): String = { + var nonNatives: Seq[String] = Seq() + val actualOp = getActualPlan(op) + actualOp.children.foreach { + case p: SparkPlan => + if (!isCometNative(p)) { + nonNatives = nonNatives :+ getActualPlan(p).nodeName + } + case _ => + } + nonNatives.mkString("(", ", ", ")") + } + // spotless:off /** * Tries to transform a Spark physical plan into a Comet plan. @@ -292,6 +306,10 @@ class CometSparkSessionExtensions op, op.children.map(_.asInstanceOf[CometNativeExec].nativeOp): _*) } else { + withInfo( + op, + s"${op.nodeName} is not native because the following children are not native " + + s"${explainChildNotNative(op)}") None } } @@ -445,7 +463,10 @@ class CometSparkSessionExtensions op case op: ShuffledHashJoinExec if !op.children.forall(isCometNative(_)) => - withInfo(op, "ShuffleHashJoin disabled because not all child plans are native") + withInfo( + op, + "ShuffleHashJoin disabled because the following children are not native " + + s"${explainChildNotNative(op)}") op case op: BroadcastHashJoinExec @@ -469,10 +490,6 @@ class CometSparkSessionExtensions op } - case op: BroadcastHashJoinExec if !isCometOperatorEnabled(conf, "broadcast_hash_join") => - withInfo(op, "BroadcastHashJoin is not enabled") - op - case op: SortMergeJoinExec if isCometOperatorEnabled(conf, "sort_merge_join") && op.children.forall(isCometNative(_)) => @@ -492,12 +509,25 @@ class CometSparkSessionExtensions case None => op } + + case op: SortMergeJoinExec + if isCometOperatorEnabled(conf, "sort_merge_join") && + !op.children.forall(isCometNative(_)) => + withInfo( + op, + "SortMergeJoin is not enabled because the following children are not native " + + s"${explainChildNotNative(op)}") + op + case op: SortMergeJoinExec if !isCometOperatorEnabled(conf, "sort_merge_join") => withInfo(op, "SortMergeJoin is not enabled") op case op: SortMergeJoinExec if !op.children.forall(isCometNative(_)) => - withInfo(op, "SortMergeJoin disabled because not all child plans are native") + withInfo( + op, + "SortMergeJoin is not enabled because the following children are not native " + + s"${explainChildNotNative(op)}") op case c @ CoalesceExec(numPartitions, child) @@ -516,7 +546,10 @@ class CometSparkSessionExtensions c case op: CoalesceExec if !op.children.forall(isCometNative(_)) => - withInfo(op, "Coalesce disabled because not all child plans are native") + withInfo( + op, + "Coalesce is not enabled because the following children are not native " + + s"${explainChildNotNative(op)}") op case s: TakeOrderedAndProjectExec @@ -569,7 +602,10 @@ class CometSparkSessionExtensions u case op: UnionExec if !op.children.forall(isCometNative(_)) => - withInfo(op, "Union disabled because not all child plans are native") + withInfo( + op, + "Union is not enabled because the following children are not native " + + s"${explainChildNotNative(op)}") op // For AQE broadcast stage on a Comet broadcast exchange @@ -606,14 +642,18 @@ class CometSparkSessionExtensions if (isCometNative(newPlan) || isCometBroadCastForceEnabled(conf)) { newPlan } else { - if (!isCometOperatorEnabled( - conf, - "broadcastExchangeExec") && !isCometBroadCastForceEnabled(conf)) { - withInfo(plan, "Native Broadcast is not enabled") + if (isCometNative(newPlan)) { + val reason = + getCometBroadcastNotEnabledReason(conf).getOrElse("no reason available") + withInfo(plan, s"Broadcast is not enabled: $reason") } plan } } else { + withInfo( + plan, + s"${plan.nodeName} is not native because the following children are not native " + + s"${explainChildNotNative(plan)}") plan } @@ -621,7 +661,14 @@ class CometSparkSessionExtensions // child BroadcastExchange has been applied, otherwise that transform // never gets applied case op: BroadcastHashJoinExec if !op.children.forall(isCometNative(_)) => - withInfo(op, "BroadcastHashJoin disabled because not all child plans are native") + withInfo( + op, + "BroadcastHashJoin is not enabled because the following children are not native " + + s"${explainChildNotNative(op)}") + op + + case op: BroadcastHashJoinExec if !isCometOperatorEnabled(conf, "broadcast_hash_join") => + withInfo(op, "BroadcastHashJoin is not enabled") op // For AQE shuffle stage on a Comet shuffle exchange @@ -714,6 +761,9 @@ class CometSparkSessionExtensions case op => // An operator that is not supported by Comet op match { + // Broadcast exchange exec is transformed by the parent node. We include + // this case specially here so we do not add a misleading 'info' message + case _: BroadcastExchangeExec => op case _: CometExec | _: CometBroadcastExchangeExec | _: CometShuffleExchangeExec => op case o => withInfo(o, s"${o.nodeName} is not supported") @@ -945,6 +995,20 @@ object CometSparkSessionExtensions extends Logging { COMET_EXEC_BROADCAST_FORCE_ENABLED.get(conf) } + private[comet] def getCometBroadcastNotEnabledReason(conf: SQLConf): Option[String] = { + val operator = "broadcastExchangeExec" + if (!isCometOperatorEnabled(conf, "broadcastExchangeExec") && !isCometBroadCastForceEnabled( + conf)) { + Some( + s"$COMET_EXEC_CONFIG_PREFIX.$operator.enabled is not specified and " + + s"${COMET_EXEC_BROADCAST_FORCE_ENABLED.key} is not specified") + } else if (!isSpark34Plus) { + Some("Native broadcast requires Spark 3.4 or newer") + } else { + None + } + } + private[comet] def isCometShuffleEnabled(conf: SQLConf): Boolean = COMET_EXEC_SHUFFLE_ENABLED.get(conf) && isCometShuffleManagerEnabled(conf) && // TODO: AQE coalesce partitions feature causes Comet shuffle memory leak. diff --git a/spark/src/main/scala/org/apache/comet/ExtendedExplainInfo.scala b/spark/src/main/scala/org/apache/comet/ExtendedExplainInfo.scala index d7ef4e9f3..755c34571 100644 --- a/spark/src/main/scala/org/apache/comet/ExtendedExplainInfo.scala +++ b/spark/src/main/scala/org/apache/comet/ExtendedExplainInfo.scala @@ -26,22 +26,18 @@ import org.apache.spark.sql.catalyst.trees.{TreeNode, TreeNodeTag} import org.apache.spark.sql.execution.{InputAdapter, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, QueryStageExec} +import org.apache.comet.CometExplainInfo.getActualPlan + class ExtendedExplainInfo extends ExtendedExplainGenerator { override def title: String = "Comet" override def generateExtendedInfo(plan: SparkPlan): String = { - val info = extensionInfo(plan) - info.toSeq.sorted.mkString("\n").trim - } - - private def getActualPlan(node: TreeNode[_]): TreeNode[_] = { - node match { - case p: AdaptiveSparkPlanExec => getActualPlan(p.executedPlan) - case p: InputAdapter => getActualPlan(p.child) - case p: QueryStageExec => getActualPlan(p.plan) - case p: WholeStageCodegenExec => getActualPlan(p.child) - case p => p + if (CometConf.COMET_EXPLAIN_VERBOSE_ENABLED.get()) { + generateVerboseExtendedInfo(plan) + } else { + val info = extensionInfo(plan) + info.toSeq.sorted.mkString("\n").trim } } @@ -81,8 +77,89 @@ class ExtendedExplainInfo extends ExtendedExplainGenerator { } ordered.reverse } + + // generates the extended info in a verbose manner, printing each node along with the + // extended information in a tree display + def generateVerboseExtendedInfo(plan: SparkPlan): String = { + val outString = new StringBuilder() + generateTreeString(getActualPlan(plan), 0, Seq(), 0, outString) + outString.toString() + } + + // Simplified generateTreeString from Spark TreeNode. Appends explain info to the node if any + def generateTreeString( + node: TreeNode[_], + depth: Int, + lastChildren: Seq[Boolean], + indent: Int, + outString: StringBuilder): Unit = { + outString.append(" " * indent) + if (depth > 0) { + lastChildren.init.foreach { isLast => + outString.append(if (isLast) " " else ": ") + } + outString.append(if (lastChildren.last) "+- " else ":- ") + } + + val tagValue = node.getTagValue(CometExplainInfo.EXTENSION_INFO) + val str = if (tagValue.nonEmpty) { + s" ${node.nodeName} [COMET: ${tagValue.get.mkString(", ")}]" + } else { + node.nodeName + } + outString.append(str) + outString.append("\n") + + val innerChildrenLocal = node.innerChildren + if (innerChildrenLocal.nonEmpty) { + innerChildrenLocal.init.foreach { + case c @ (_: TreeNode[_]) => + generateTreeString( + getActualPlan(c), + depth + 2, + lastChildren :+ node.children.isEmpty :+ false, + indent, + outString) + case _ => + } + generateTreeString( + getActualPlan(innerChildrenLocal.last), + depth + 2, + lastChildren :+ node.children.isEmpty :+ true, + indent, + outString) + } + if (node.children.nonEmpty) { + node.children.init.foreach { + case c @ (_: TreeNode[_]) => + generateTreeString( + getActualPlan(c), + depth + 1, + lastChildren :+ false, + indent, + outString) + case _ => + } + node.children.last match { + case c @ (_: TreeNode[_]) => + generateTreeString(getActualPlan(c), depth + 1, lastChildren :+ true, indent, outString) + case _ => + } + } + } } object CometExplainInfo { val EXTENSION_INFO = new TreeNodeTag[Set[String]]("CometExtensionInfo") + + def getActualPlan(node: TreeNode[_]): TreeNode[_] = { + node match { + case p: AdaptiveSparkPlanExec => getActualPlan(p.executedPlan) + case p: InputAdapter => getActualPlan(p.child) + case p: QueryStageExec => getActualPlan(p.plan) + case p: WholeStageCodegenExec => getActualPlan(p.child) + case p => p + } + } + } diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index 8dbfb71b3..e80549988 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -1543,6 +1543,7 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { ( s"SELECT sum(c0), sum(c2) from $table group by c1", Set( + "HashAggregate is not native because the following children are not native (AQEShuffleRead)", "Comet shuffle is not enabled: spark.comet.exec.shuffle.enabled is not enabled", "AQEShuffleRead is not supported")), ( @@ -1554,8 +1555,10 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { "Comet shuffle is not enabled: spark.comet.exec.shuffle.enabled is not enabled", "AQEShuffleRead is not supported", "make_interval is not supported", - "BroadcastExchange is not supported", - "BroadcastHashJoin disabled because not all child plans are native"))) + "HashAggregate is not native because the following children are not native (AQEShuffleRead)", + "Project is not native because the following children are not native (BroadcastHashJoin)", + "BroadcastHashJoin is not enabled because the following children are not native" + + " (BroadcastExchange, Project)"))) .foreach(test => { val qry = test._1 val expected = test._2 diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTPCQueryListBase.scala b/spark/src/test/scala/org/apache/spark/sql/CometTPCQueryListBase.scala index 6a5832060..15b1745c7 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTPCQueryListBase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTPCQueryListBase.scala @@ -88,6 +88,7 @@ trait CometTPCQueryListBase CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true", + CometConf.COMET_EXPLAIN_VERBOSE_ENABLED.key -> "true", // Lower bloom filter thresholds to allows us to simulate the plan produced at larger scale. "spark.sql.optimizer.runtime.bloomFilter.creationSideThreshold" -> "1MB", "spark.sql.optimizer.runtime.bloomFilter.applicationSideScanSizeThreshold" -> "1MB") { From 8d0720431ebe579f8d6d40c85aa92cfa9fb98acc Mon Sep 17 00:00:00 2001 From: Emil Ejbyfeldt Date: Wed, 3 Jul 2024 21:15:44 +0200 Subject: [PATCH 07/16] feat: IsNaN expression in Comet (#612) * Support IsNaN expression in Comet * Document that IsNaN is supported * Update spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala Co-authored-by: Liang-Chi Hsieh * Fix whitespace --------- Co-authored-by: Liang-Chi Hsieh --- .../datafusion/expressions/scalar_funcs.rs | 52 ++++++++++++++++++- docs/source/user-guide/expressions.md | 1 + .../apache/comet/serde/QueryPlanSerde.scala | 7 +++ .../apache/comet/CometExpressionSuite.scala | 16 ++++++ 4 files changed, 75 insertions(+), 1 deletion(-) diff --git a/core/src/execution/datafusion/expressions/scalar_funcs.rs b/core/src/execution/datafusion/expressions/scalar_funcs.rs index e55425642..d47172dfa 100644 --- a/core/src/execution/datafusion/expressions/scalar_funcs.rs +++ b/core/src/execution/datafusion/expressions/scalar_funcs.rs @@ -24,7 +24,7 @@ use arrow::{ }, datatypes::{validate_decimal_precision, Decimal128Type, Int64Type}, }; -use arrow_array::{Array, ArrowNativeTypeOp, Decimal128Array}; +use arrow_array::{Array, ArrowNativeTypeOp, BooleanArray, Decimal128Array}; use arrow_schema::DataType; use datafusion::{ execution::FunctionRegistry, @@ -129,6 +129,10 @@ pub fn create_comet_physical_fun( let func = Arc::new(spark_chr); make_comet_scalar_udf!("chr", func, without data_type) } + "isnan" => { + let func = Arc::new(spark_isnan); + make_comet_scalar_udf!("isnan", func, without data_type) + } sha if sha2_functions.contains(&sha) => { // Spark requires hex string as the result of sha2 functions, we have to wrap the // result of digest functions as hex string @@ -634,3 +638,49 @@ fn spark_decimal_div( let result = result.with_data_type(DataType::Decimal128(p3, s3)); Ok(ColumnarValue::Array(Arc::new(result))) } + +fn spark_isnan(args: &[ColumnarValue]) -> Result { + fn set_nulls_to_false(is_nan: BooleanArray) -> ColumnarValue { + match is_nan.nulls() { + Some(nulls) => { + let is_not_null = nulls.inner(); + ColumnarValue::Array(Arc::new(BooleanArray::new( + is_nan.values() & is_not_null, + None, + ))) + } + None => ColumnarValue::Array(Arc::new(is_nan)), + } + } + let value = &args[0]; + match value { + ColumnarValue::Array(array) => match array.data_type() { + DataType::Float64 => { + let array = array.as_any().downcast_ref::().unwrap(); + let is_nan = BooleanArray::from_unary(array, |x| x.is_nan()); + Ok(set_nulls_to_false(is_nan)) + } + DataType::Float32 => { + let array = array.as_any().downcast_ref::().unwrap(); + let is_nan = BooleanArray::from_unary(array, |x| x.is_nan()); + Ok(set_nulls_to_false(is_nan)) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function isnan", + other, + ))), + }, + ColumnarValue::Scalar(a) => match a { + ScalarValue::Float64(a) => Ok(ColumnarValue::Scalar(ScalarValue::Boolean(Some( + a.map(|x| x.is_nan()).unwrap_or(false), + )))), + ScalarValue::Float32(a) => Ok(ColumnarValue::Scalar(ScalarValue::Boolean(Some( + a.map(|x| x.is_nan()).unwrap_or(false), + )))), + _ => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function isnan", + value.data_type(), + ))), + }, + } +} diff --git a/docs/source/user-guide/expressions.md b/docs/source/user-guide/expressions.md index 14b6f18d0..1d4ce3736 100644 --- a/docs/source/user-guide/expressions.md +++ b/docs/source/user-guide/expressions.md @@ -119,6 +119,7 @@ The following Spark expressions are currently available. Any known compatibility | Cos | | | Exp | | | Floor | | +| IsNaN | | | Log | log(0) will produce `-Infinity` unlike Spark which returns `null` | | Log2 | log2(0) will produce `-Infinity` unlike Spark which returns `null` | | Log10 | log10(0) will produce `-Infinity` unlike Spark which returns `null` | 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 3465315d1..4c4530db3 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -1476,6 +1476,13 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim None } + case IsNaN(child) => + val childExpr = exprToProtoInternal(child, inputs) + val optExpr = + scalarExprToProtoWithReturnType("isnan", BooleanType, childExpr) + + optExprWithInfo(optExpr, expr, child) + case SortOrder(child, direction, nullOrdering, _) => val childExpr = exprToProtoInternal(child, inputs) diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index e80549988..498a305ed 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -1722,4 +1722,20 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } } + + test("isnan") { + Seq("true", "false").foreach { dictionary => + withSQLConf("parquet.enable.dictionary" -> dictionary) { + withParquetTable( + Seq(Some(1.0), Some(Double.NaN), None).map(i => Tuple1(i)), + "tbl", + withDictionary = dictionary.toBoolean) { + checkSparkAnswerAndOperator("SELECT isnan(_1), isnan(cast(_1 as float)) FROM tbl") + // Use inside a nullable statement to make sure isnan has correct behavior for null input + checkSparkAnswerAndOperator( + "SELECT CASE WHEN (_1 > 0) THEN NULL ELSE isnan(_1) END FROM tbl") + } + } + } + } } From 9fff887c21b2ce1661c65ff25669a171246cb75b Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 3 Jul 2024 21:58:50 -0700 Subject: [PATCH 08/16] build: Upgrade to 2.13.14 for scala-2.13 profile (#626) --- pom.xml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 0195e9292..043c11e89 100644 --- a/pom.xml +++ b/pom.xml @@ -591,8 +591,9 @@ under the License. scala-2.13 - 2.13.8 + 2.13.14 2.13 + 4.9.5 From b3977cd6e663b03d1dd63cb8d3f6c66f19ad0795 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 4 Jul 2024 09:18:40 -0600 Subject: [PATCH 09/16] chore: Rename shuffle write metric (#624) * Rename shuffle write metric * introduce constants for shuffle native write metric --- .../apache/spark/sql/comet/CometCollectLimitExec.scala | 5 +++-- .../sql/comet/CometTakeOrderedAndProjectExec.scala | 5 +++-- .../org/apache/spark/sql/comet/CometWindowExec.scala | 5 +++-- .../execution/shuffle/CometShuffleExchangeExec.scala | 10 +++++++--- 4 files changed, 16 insertions(+), 9 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala index dd4855126..d2c9158ee 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala @@ -23,6 +23,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.comet.execution.shuffle.{CometShuffledBatchRDD, CometShuffleExchangeExec} +import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec.{METRIC_NATIVE_TIME_DESCRIPTION, METRIC_NATIVE_TIME_NAME} import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, UnaryExecNode, UnsafeRowSerializer} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics, SQLShuffleReadMetricsReporter, SQLShuffleWriteMetricsReporter} import org.apache.spark.sql.vectorized.ColumnarBatch @@ -51,8 +52,8 @@ case class CometCollectLimitExec( SQLShuffleReadMetricsReporter.createShuffleReadMetrics(sparkContext) override lazy val metrics: Map[String, SQLMetric] = Map( "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"), - "shuffleReadElapsedCompute" -> - SQLMetrics.createNanoTimingMetric(sparkContext, "shuffle read elapsed compute at native"), + METRIC_NATIVE_TIME_NAME -> + SQLMetrics.createNanoTimingMetric(sparkContext, METRIC_NATIVE_TIME_DESCRIPTION), "numPartitions" -> SQLMetrics.createMetric( sparkContext, "number of partitions")) ++ readMetrics ++ writeMetrics diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala index 26ec401ed..6e9bfe424 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala @@ -24,6 +24,7 @@ import org.apache.spark.serializer.Serializer import org.apache.spark.sql.catalyst.expressions.{Attribute, NamedExpression, SortOrder} import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.comet.execution.shuffle.{CometShuffledBatchRDD, CometShuffleExchangeExec} +import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec.{METRIC_NATIVE_TIME_DESCRIPTION, METRIC_NATIVE_TIME_NAME} import org.apache.spark.sql.execution.{SparkPlan, TakeOrderedAndProjectExec, UnaryExecNode, UnsafeRowSerializer} import org.apache.spark.sql.execution.metric.{SQLMetrics, SQLShuffleReadMetricsReporter, SQLShuffleWriteMetricsReporter} import org.apache.spark.sql.execution.metric.SQLMetric @@ -55,8 +56,8 @@ case class CometTakeOrderedAndProjectExec( SQLShuffleReadMetricsReporter.createShuffleReadMetrics(sparkContext) override lazy val metrics: Map[String, SQLMetric] = Map( "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"), - "shuffleReadElapsedCompute" -> - SQLMetrics.createNanoTimingMetric(sparkContext, "shuffle read elapsed compute at native"), + METRIC_NATIVE_TIME_NAME -> + SQLMetrics.createNanoTimingMetric(sparkContext, METRIC_NATIVE_TIME_DESCRIPTION), "numPartitions" -> SQLMetrics.createMetric( sparkContext, "number of partitions")) ++ readMetrics ++ writeMetrics diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala index 9a1232f0c..9685e75e1 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala @@ -25,6 +25,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, SortOrder, WindowExpression} import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.comet.CometWindowExec.getNativePlan +import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec.{METRIC_NATIVE_TIME_DESCRIPTION, METRIC_NATIVE_TIME_NAME} import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics, SQLShuffleReadMetricsReporter, SQLShuffleWriteMetricsReporter} import org.apache.spark.sql.vectorized.ColumnarBatch @@ -59,8 +60,8 @@ case class CometWindowExec( SQLShuffleReadMetricsReporter.createShuffleReadMetrics(sparkContext) override lazy val metrics: Map[String, SQLMetric] = Map( "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"), - "shuffleReadElapsedCompute" -> - SQLMetrics.createNanoTimingMetric(sparkContext, "shuffle read elapsed compute at native"), + METRIC_NATIVE_TIME_NAME -> + SQLMetrics.createNanoTimingMetric(sparkContext, METRIC_NATIVE_TIME_DESCRIPTION), "numPartitions" -> SQLMetrics.createMetric( sparkContext, "number of partitions")) ++ readMetrics ++ writeMetrics diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala index aabe3c350..cec64d7a8 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala @@ -39,6 +39,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.comet.{CometExec, CometMetricNode, CometPlan} +import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec.{METRIC_NATIVE_TIME_DESCRIPTION, METRIC_NATIVE_TIME_NAME} import org.apache.spark.sql.comet.shims.ShimCometShuffleWriteProcessor import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.exchange.{ENSURE_REQUIREMENTS, ShuffleExchangeLike, ShuffleOrigin} @@ -77,8 +78,8 @@ case class CometShuffleExchangeExec( SQLShuffleReadMetricsReporter.createShuffleReadMetrics(sparkContext) override lazy val metrics: Map[String, SQLMetric] = Map( "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"), - "shuffleReadElapsedCompute" -> - SQLMetrics.createNanoTimingMetric(sparkContext, "shuffle read elapsed compute at native"), + METRIC_NATIVE_TIME_NAME -> + SQLMetrics.createNanoTimingMetric(sparkContext, METRIC_NATIVE_TIME_DESCRIPTION), "numPartitions" -> SQLMetrics.createMetric( sparkContext, "number of partitions")) ++ readMetrics ++ writeMetrics @@ -220,6 +221,9 @@ case class CometShuffleExchangeExec( } object CometShuffleExchangeExec extends ShimCometShuffleExchangeExec { + val METRIC_NATIVE_TIME_NAME = "shuffleNativeTotalTime" + val METRIC_NATIVE_TIME_DESCRIPTION = "shuffle native code time" + def prepareShuffleDependency( rdd: RDD[ColumnarBatch], outputAttributes: Seq[Attribute], @@ -479,7 +483,7 @@ class CometShuffleWriteProcessor( // Maps native metrics to SQL metrics val nativeSQLMetrics = Map( "output_rows" -> metrics(SQLShuffleWriteMetricsReporter.SHUFFLE_RECORDS_WRITTEN), - "elapsed_compute" -> metrics("shuffleReadElapsedCompute")) + "elapsed_compute" -> metrics("shuffleNativeTotalTime")) val nativeMetrics = CometMetricNode(nativeSQLMetrics) // Getting rid of the fake partitionId From d8fef2b93f3b5bc5e6f259fc8bf44adc1122d2a3 Mon Sep 17 00:00:00 2001 From: Emil Ejbyfeldt Date: Fri, 5 Jul 2024 18:00:56 +0200 Subject: [PATCH 10/16] feat: Add support for CreateNamedStruct (#620) * Add support for CreateNamedStruct * Exclude HashJoins using struct keys as this is currently unsupported in datafusion * Add CreateNamedStruct to docs * Add message --- .../expressions/create_named_struct.rs | 127 ++++++++++++++++++ .../execution/datafusion/expressions/mod.rs | 1 + core/src/execution/datafusion/planner.rs | 11 +- core/src/execution/proto/expr.proto | 6 + docs/source/user-guide/expressions.md | 1 + .../apache/comet/serde/QueryPlanSerde.scala | 24 ++++ .../apache/comet/CometExpressionSuite.scala | 15 +++ 7 files changed, 184 insertions(+), 1 deletion(-) create mode 100644 core/src/execution/datafusion/expressions/create_named_struct.rs diff --git a/core/src/execution/datafusion/expressions/create_named_struct.rs b/core/src/execution/datafusion/expressions/create_named_struct.rs new file mode 100644 index 000000000..17d763cca --- /dev/null +++ b/core/src/execution/datafusion/expressions/create_named_struct.rs @@ -0,0 +1,127 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::{ + any::Any, + fmt::{Display, Formatter}, + hash::{Hash, Hasher}, + sync::Arc, +}; + +use arrow::record_batch::RecordBatch; +use arrow_array::StructArray; +use arrow_schema::{DataType, Schema}; +use datafusion::logical_expr::ColumnarValue; +use datafusion_common::{DataFusionError, Result as DataFusionResult}; +use datafusion_physical_expr::PhysicalExpr; + +use crate::execution::datafusion::expressions::utils::down_cast_any_ref; + +#[derive(Debug, Hash)] +pub struct CreateNamedStruct { + values: Vec>, + data_type: DataType, +} + +impl CreateNamedStruct { + pub fn new(values: Vec>, data_type: DataType) -> Self { + Self { values, data_type } + } +} + +impl PhysicalExpr for CreateNamedStruct { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, _input_schema: &Schema) -> DataFusionResult { + Ok(self.data_type.clone()) + } + + fn nullable(&self, _input_schema: &Schema) -> DataFusionResult { + Ok(false) + } + + fn evaluate(&self, batch: &RecordBatch) -> DataFusionResult { + let values = self + .values + .iter() + .map(|expr| expr.evaluate(batch)) + .collect::>>()?; + let arrays = ColumnarValue::values_to_arrays(&values)?; + let fields = match &self.data_type { + DataType::Struct(fields) => fields, + _ => { + return Err(DataFusionError::Internal(format!( + "Expected struct data type, got {:?}", + self.data_type + ))) + } + }; + Ok(ColumnarValue::Array(Arc::new(StructArray::new( + fields.clone(), + arrays, + None, + )))) + } + + fn children(&self) -> Vec<&Arc> { + self.values.iter().collect() + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> datafusion_common::Result> { + Ok(Arc::new(CreateNamedStruct::new( + children.clone(), + self.data_type.clone(), + ))) + } + + fn dyn_hash(&self, state: &mut dyn Hasher) { + let mut s = state; + self.values.hash(&mut s); + self.data_type.hash(&mut s); + self.hash(&mut s); + } +} + +impl Display for CreateNamedStruct { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!( + f, + "CreateNamedStruct [values: {:?}, data_type: {:?}]", + self.values, self.data_type + ) + } +} + +impl PartialEq for CreateNamedStruct { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::() + .map(|x| { + self.values + .iter() + .zip(x.values.iter()) + .all(|(a, b)| a.eq(b)) + && self.data_type.eq(&x.data_type) + }) + .unwrap_or(false) + } +} diff --git a/core/src/execution/datafusion/expressions/mod.rs b/core/src/execution/datafusion/expressions/mod.rs index d91e25980..385857f1e 100644 --- a/core/src/execution/datafusion/expressions/mod.rs +++ b/core/src/execution/datafusion/expressions/mod.rs @@ -33,6 +33,7 @@ pub mod avg_decimal; pub mod bloom_filter_might_contain; pub mod correlation; pub mod covariance; +pub mod create_named_struct; pub mod negative; pub mod stats; pub mod stddev; diff --git a/core/src/execution/datafusion/planner.rs b/core/src/execution/datafusion/planner.rs index a969c3baa..8d7d24654 100644 --- a/core/src/execution/datafusion/planner.rs +++ b/core/src/execution/datafusion/planner.rs @@ -104,7 +104,7 @@ use crate::{ }, }; -use super::expressions::{abs::CometAbsFunc, EvalMode}; +use super::expressions::{abs::CometAbsFunc, create_named_struct::CreateNamedStruct, EvalMode}; // For clippy error on type_complexity. type ExecResult = Result; @@ -591,6 +591,15 @@ impl PhysicalPlanner { value_expr, )?)) } + ExprStruct::CreateNamedStruct(expr) => { + let values = expr + .values + .iter() + .map(|expr| self.create_expr(expr, input_schema.clone())) + .collect::, _>>()?; + let data_type = to_arrow_datatype(expr.datatype.as_ref().unwrap()); + Ok(Arc::new(CreateNamedStruct::new(values, data_type))) + } expr => Err(ExecutionError::GeneralError(format!( "Not implemented: {:?}", expr diff --git a/core/src/execution/proto/expr.proto b/core/src/execution/proto/expr.proto index 6b66a307a..56518d9ee 100644 --- a/core/src/execution/proto/expr.proto +++ b/core/src/execution/proto/expr.proto @@ -77,6 +77,7 @@ message Expr { Subquery subquery = 50; UnboundReference unbound = 51; BloomFilterMightContain bloom_filter_might_contain = 52; + CreateNamedStruct create_named_struct = 53; } } @@ -486,6 +487,11 @@ message BloomFilterMightContain { Expr value = 2; } +message CreateNamedStruct { + repeated Expr values = 1; + DataType datatype = 2; +} + enum SortDirection { Ascending = 0; Descending = 1; diff --git a/docs/source/user-guide/expressions.md b/docs/source/user-guide/expressions.md index 1d4ce3736..07323382c 100644 --- a/docs/source/user-guide/expressions.md +++ b/docs/source/user-guide/expressions.md @@ -191,3 +191,4 @@ The following Spark expressions are currently available. Any known compatibility | ScalarSubquery | | | Coalesce | | | NormalizeNaNAndZero | | +| CreateNamedStruct | | 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 4c4530db3..65de37c83 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -2272,6 +2272,25 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim scalarExprToProtoWithReturnType(algorithm, StringType, childExpr) } + case struct @ CreateNamedStruct(_) => + val valExprs = struct.valExprs.map(exprToProto(_, inputs, binding)) + val dataType = serializeDataType(struct.dataType) + + if (valExprs.forall(_.isDefined) && dataType.isDefined) { + val structBuilder = ExprOuterClass.CreateNamedStruct.newBuilder() + structBuilder.addAllValues(valExprs.map(_.get).asJava) + structBuilder.setDatatype(dataType.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setCreateNamedStruct(structBuilder) + .build()) + } else { + withInfo(expr, "unsupported arguments for CreateNamedStruct", struct.valExprs: _*) + None + } + case _ => withInfo(expr, s"${expr.prettyName} is not supported", expr.children: _*) None @@ -2633,6 +2652,11 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim return None } + if ((join.leftKeys ++ join.rightKeys).exists(_.dataType.isInstanceOf[StructType])) { + withInfo(join, "Unsupported struct data type in join keys") + return None + } + if (join.buildSide == BuildRight && join.joinType == LeftAnti) { withInfo(join, "BuildRight with LeftAnti is not supported") return None diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index 498a305ed..cb2069ed2 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -1738,4 +1738,19 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } } + + test("named_struct") { + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "test.parquet") + makeParquetFileAllTypes(path, dictionaryEnabled = dictionaryEnabled, 10000) + withParquetTable(path.toString, "tbl") { + checkSparkAnswerAndOperator("SELECT named_struct('a', _1, 'b', _2) FROM tbl") + checkSparkAnswerAndOperator("SELECT named_struct('a', _1, 'b', 2) FROM tbl") + checkSparkAnswerAndOperator( + "SELECT named_struct('a', named_struct('b', _1, 'c', _2)) FROM tbl") + } + } + } + } } From eff2897c98b3c820142514af888a126928f7f9b2 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 5 Jul 2024 15:28:56 -0600 Subject: [PATCH 11/16] minor: replace .downcast_ref::().is_some() with .is::() (#635) --- core/src/execution/datafusion/planner.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/execution/datafusion/planner.rs b/core/src/execution/datafusion/planner.rs index 8d7d24654..40515c0c4 100644 --- a/core/src/execution/datafusion/planner.rs +++ b/core/src/execution/datafusion/planner.rs @@ -916,7 +916,7 @@ impl PhysicalPlanner { // the data corruption. Note that we only need to copy the input batch // if the child operator is `ScanExec`, because other operators after `ScanExec` // will create new arrays for the output batch. - let child = if child.as_any().downcast_ref::().is_some() { + let child = if child.as_any().is::() { Arc::new(CopyExec::new(child)) } else { child @@ -1609,10 +1609,10 @@ impl From for DataFusionError { /// modification. This is used to determine if we need to copy the input batch to avoid /// data corruption from reusing the input batch. fn can_reuse_input_batch(op: &Arc) -> bool { - op.as_any().downcast_ref::().is_some() - || op.as_any().downcast_ref::().is_some() - || op.as_any().downcast_ref::().is_some() - || op.as_any().downcast_ref::().is_some() + op.as_any().is::() + || op.as_any().is::() + || op.as_any().is::() + || op.as_any().is::() } /// Collects the indices of the columns in the input schema that are used in the expression From 335146eab34f70df56a70061c79df2448e8eca82 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 5 Jul 2024 14:53:32 -0700 Subject: [PATCH 12/16] test: Copy Spark TPCDSQueryTestSuite to CometTPCDSQueryTestSuite (#628) --- .../spark/sql/CometSQLQueryTestHelper.scala | 108 ++++++++ .../spark/sql/CometTPCDSQuerySuite.scala | 2 +- .../spark/sql/CometTPCDSQueryTestSuite.scala | 230 ++++++++++++++++++ 3 files changed, 339 insertions(+), 1 deletion(-) create mode 100644 spark/src/test/scala/org/apache/spark/sql/CometSQLQueryTestHelper.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/CometTPCDSQueryTestSuite.scala diff --git a/spark/src/test/scala/org/apache/spark/sql/CometSQLQueryTestHelper.scala b/spark/src/test/scala/org/apache/spark/sql/CometSQLQueryTestHelper.scala new file mode 100644 index 000000000..bf5ed4396 --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/CometSQLQueryTestHelper.scala @@ -0,0 +1,108 @@ +/* + * 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 + +import scala.util.control.NonFatal + +import org.apache.spark.{SparkException, SparkThrowable} +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.execution.HiveResult.hiveResultString +import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.execution.command.{DescribeColumnCommand, DescribeCommandBase} +import org.apache.spark.sql.types.StructType + +trait CometSQLQueryTestHelper { + + private val notIncludedMsg = "[not included in comparison]" + private val clsName = this.getClass.getCanonicalName + protected val emptySchema: String = StructType(Seq.empty).catalogString + + protected def replaceNotIncludedMsg(line: String): String = { + line + .replaceAll("#\\d+", "#x") + .replaceAll("plan_id=\\d+", "plan_id=x") + .replaceAll(s"Location.*$clsName/", s"Location $notIncludedMsg/{warehouse_dir}/") + .replaceAll(s"file:[^\\s,]*$clsName", s"file:$notIncludedMsg/{warehouse_dir}") + .replaceAll("Created By.*", s"Created By $notIncludedMsg") + .replaceAll("Created Time.*", s"Created Time $notIncludedMsg") + .replaceAll("Last Access.*", s"Last Access $notIncludedMsg") + .replaceAll("Partition Statistics\t\\d+", s"Partition Statistics\t$notIncludedMsg") + .replaceAll("\\*\\(\\d+\\) ", "*") // remove the WholeStageCodegen codegenStageIds + } + + /** Executes a query and returns the result as (schema of the output, normalized output). */ + protected def getNormalizedResult(session: SparkSession, sql: String): (String, Seq[String]) = { + // Returns true if the plan is supposed to be sorted. + def isSorted(plan: LogicalPlan): Boolean = plan match { + case _: Join | _: Aggregate | _: Generate | _: Sample | _: Distinct => false + case _: DescribeCommandBase | _: DescribeColumnCommand | _: DescribeRelation | + _: DescribeColumn => + true + case PhysicalOperation(_, _, Sort(_, true, _)) => true + case _ => plan.children.iterator.exists(isSorted) + } + + val df = session.sql(sql) + val schema = df.schema.catalogString + // Get answer, but also get rid of the #1234 expression ids that show up in explain plans + val answer = SQLExecution.withNewExecutionId(df.queryExecution, Some(sql)) { + hiveResultString(df.queryExecution.executedPlan).map(replaceNotIncludedMsg) + } + + // If the output is not pre-sorted, sort it. + if (isSorted(df.queryExecution.analyzed)) (schema, answer) else (schema, answer.sorted) + } + + /** + * This method handles exceptions occurred during query execution as they may need special care + * to become comparable to the expected output. + * + * @param result + * a function that returns a pair of schema and output + */ + protected def handleExceptions(result: => (String, Seq[String])): (String, Seq[String]) = { + try { + result + } catch { + case e: SparkThrowable with Throwable if e.getErrorClass != null => + (emptySchema, Seq(e.getClass.getName, e.getMessage)) + case a: AnalysisException => + // Do not output the logical plan tree which contains expression IDs. + // Also implement a crude way of masking expression IDs in the error message + // with a generic pattern "###". + val msg = a.getMessage + (emptySchema, Seq(a.getClass.getName, msg.replaceAll("#\\d+", "#x"))) + case s: SparkException if s.getCause != null => + // For a runtime exception, it is hard to match because its message contains + // information of stage, task ID, etc. + // To make result matching simpler, here we match the cause of the exception if it exists. + s.getCause match { + case e: SparkThrowable with Throwable if e.getErrorClass != null => + (emptySchema, Seq(e.getClass.getName, e.getMessage)) + case cause => + (emptySchema, Seq(cause.getClass.getName, cause.getMessage)) + } + case NonFatal(e) => + // If there is an exception, put the exception class followed by the message. + (emptySchema, Seq(e.getClass.getName, e.getMessage)) + } + } +} 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 53186b131..3e0f64522 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala @@ -145,7 +145,7 @@ class CometTPCDSQuerySuite override val tpcdsQueries: Seq[String] = tpcdsAllQueries.filterNot(excludedTpcdsQueries.contains) } - with TPCDSQueryTestSuite + with CometTPCDSQueryTestSuite with ShimCometTPCDSQuerySuite { override def sparkConf: SparkConf = { val conf = super.sparkConf diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQueryTestSuite.scala b/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQueryTestSuite.scala new file mode 100644 index 000000000..c2b853515 --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQueryTestSuite.scala @@ -0,0 +1,230 @@ +/* + * 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 + +import java.io.File +import java.nio.file.{Files, Paths} + +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.catalyst.util.{fileToString, resourceToString, stringToFile} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.TestSparkSession + +/** + * Because we need to modify some methods of Spark `TPCDSQueryTestSuite` but they are private, we + * copy Spark `TPCDSQueryTestSuite`. + */ +class CometTPCDSQueryTestSuite extends QueryTest with TPCDSBase with CometSQLQueryTestHelper { + + private val tpcdsDataPath = sys.env.get("SPARK_TPCDS_DATA") + + private val regenGoldenFiles: Boolean = System.getenv("SPARK_GENERATE_GOLDEN_FILES") == "1" + + // To make output results deterministic + override protected def sparkConf: SparkConf = super.sparkConf + .set(SQLConf.SHUFFLE_PARTITIONS.key, "1") + + protected override def createSparkSession: TestSparkSession = { + new TestSparkSession(new SparkContext("local[1]", this.getClass.getSimpleName, sparkConf)) + } + + // We use SF=1 table data here, so we cannot use SF=100 stats + protected override val injectStats: Boolean = false + + if (tpcdsDataPath.nonEmpty) { + val nonExistentTables = tableNames.filterNot { tableName => + Files.exists(Paths.get(s"${tpcdsDataPath.get}/$tableName")) + } + if (nonExistentTables.nonEmpty) { + fail( + s"Non-existent TPCDS table paths found in ${tpcdsDataPath.get}: " + + nonExistentTables.mkString(", ")) + } + } + + protected val baseResourcePath: String = { + // use the same way as `SQLQueryTestSuite` to get the resource path + getWorkspaceFilePath( + "sql", + "core", + "src", + "test", + "resources", + "tpcds-query-results").toFile.getAbsolutePath + } + + override def createTable( + spark: SparkSession, + tableName: String, + format: String = "parquet", + options: scala.Seq[String]): Unit = { + spark.sql(s""" + |CREATE TABLE `$tableName` (${tableColumns(tableName)}) + |USING $format + |LOCATION '${tpcdsDataPath.get}/$tableName' + |${options.mkString("\n")} + """.stripMargin) + } + + private def runQuery(query: String, goldenFile: File, conf: Map[String, String]): Unit = { + // This is `sortMergeJoinConf != conf` in Spark, i.e., it sorts results for other joins + // than sort merge join. But in some queries DataFusion sort returns correct results + // in terms of required sorting columns, but the results are not same as Spark in terms of + // order of irrelevant columns. So, we need to sort the results for all joins. + val shouldSortResults = true + withSQLConf(conf.toSeq: _*) { + try { + val (schema, output) = handleExceptions(getNormalizedResult(spark, query)) + val queryString = query.trim + val outputString = output.mkString("\n").replaceAll("\\s+$", "") + if (regenGoldenFiles) { + val goldenOutput = { + s"-- Automatically generated by ${getClass.getSimpleName}\n\n" + + "-- !query schema\n" + + schema + "\n" + + "-- !query output\n" + + outputString + + "\n" + } + val parent = goldenFile.getParentFile + if (!parent.exists()) { + assert(parent.mkdirs(), "Could not create directory: " + parent) + } + stringToFile(goldenFile, goldenOutput) + } + + // Read back the golden file. + val (expectedSchema, expectedOutput) = { + val goldenOutput = fileToString(goldenFile) + val segments = goldenOutput.split("-- !query.*\n") + + // query has 3 segments, plus the header + assert( + segments.size == 3, + s"Expected 3 blocks in result file but got ${segments.size}. " + + "Try regenerate the result files.") + + (segments(1).trim, segments(2).replaceAll("\\s+$", "")) + } + + val notMatchedSchemaOutput = if (schema == emptySchema) { + // There might be exception. See `handleExceptions`. + s"Schema did not match\n$queryString\nOutput/Exception: $outputString" + } else { + s"Schema did not match\n$queryString" + } + + assertResult(expectedSchema, notMatchedSchemaOutput) { + schema + } + if (shouldSortResults) { + val expectSorted = expectedOutput + .split("\n") + .sorted + .map(_.trim) + .mkString("\n") + .replaceAll("\\s+$", "") + val outputSorted = output.sorted.map(_.trim).mkString("\n").replaceAll("\\s+$", "") + assertResult(expectSorted, s"Result did not match\n$queryString") { + outputSorted + } + } else { + assertResult(expectedOutput, s"Result did not match\n$queryString") { + outputString + } + } + } catch { + case e: Throwable => + val configs = conf.map { case (k, v) => + s"$k=$v" + } + throw new Exception(s"${e.getMessage}\nError using configs:\n${configs.mkString("\n")}") + } + } + } + + val sortMergeJoinConf: Map[String, String] = Map( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.PREFER_SORTMERGEJOIN.key -> "true") + + val broadcastHashJoinConf: Map[String, String] = Map( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10485760") + + val shuffledHashJoinConf: Map[String, String] = Map( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + "spark.sql.join.forceApplyShuffledHashJoin" -> "true") + + val allJoinConfCombinations: Seq[Map[String, String]] = + Seq(sortMergeJoinConf, broadcastHashJoinConf, shuffledHashJoinConf) + + val joinConfs: Seq[Map[String, String]] = if (regenGoldenFiles) { + require( + !sys.env.contains("SPARK_TPCDS_JOIN_CONF"), + "'SPARK_TPCDS_JOIN_CONF' cannot be set together with 'SPARK_GENERATE_GOLDEN_FILES'") + Seq(sortMergeJoinConf) + } else { + sys.env + .get("SPARK_TPCDS_JOIN_CONF") + .map { s => + val p = new java.util.Properties() + p.load(new java.io.StringReader(s)) + Seq(p.asScala.toMap) + } + .getOrElse(allJoinConfCombinations) + } + + assert(joinConfs.nonEmpty) + joinConfs.foreach(conf => + require( + allJoinConfCombinations.contains(conf), + s"Join configurations [$conf] should be one of $allJoinConfCombinations")) + + if (tpcdsDataPath.nonEmpty) { + tpcdsQueries.foreach { name => + val queryString = resourceToString( + s"tpcds/$name.sql", + classLoader = Thread.currentThread().getContextClassLoader) + test(name) { + val goldenFile = new File(s"$baseResourcePath/v1_4", s"$name.sql.out") + joinConfs.foreach { conf => + System.gc() // Workaround for GitHub Actions memory limitation, see also SPARK-37368 + runQuery(queryString, goldenFile, conf) + } + } + } + + tpcdsQueriesV2_7_0.foreach { name => + val queryString = resourceToString( + s"tpcds-v2.7.0/$name.sql", + classLoader = Thread.currentThread().getContextClassLoader) + test(s"$name-v2.7") { + val goldenFile = new File(s"$baseResourcePath/v2_7", s"$name.sql.out") + joinConfs.foreach { conf => + System.gc() // SPARK-37368 + runQuery(queryString, goldenFile, conf) + } + } + } + } else { + ignore("skipped because env `SPARK_TPCDS_DATA` is not set") {} + } +} From a8433b5f359f2abe1ebdc57a98457991e9303402 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 8 Jul 2024 09:42:34 -0600 Subject: [PATCH 13/16] chore: Convert Rust project into a workspace (#637) * convert into workspace project * update GitHub actions * update Makefile * fix regression * update target path * update protobuf path in pom.xml * update more paths --- .github/actions/java-test/action.yaml | 2 +- .github/actions/rust-test/action.yaml | 8 +- .github/workflows/benchmark-tpch.yml | 6 +- .github/workflows/benchmark.yml | 6 +- .gitignore | 2 +- Makefile | 36 +- common/pom.xml | 4 +- native/Cargo.lock | 3301 +++++++++++++++++ native/Cargo.toml | 32 + {core => native/core}/Cargo.lock | 0 {core => native/core}/Cargo.toml | 0 {core => native/core}/benches/bit_util.rs | 0 .../core}/benches/cast_from_string.rs | 0 {core => native/core}/benches/cast_numeric.rs | 0 {core => native/core}/benches/common.rs | 0 {core => native/core}/benches/hash.rs | 0 .../core}/benches/parquet_decode.rs | 0 {core => native/core}/benches/parquet_read.rs | 0 {core => native/core}/benches/perf.rs | 0 {core => native/core}/benches/row_columnar.rs | 0 .../core}/benches/shuffle_writer.rs | 0 {core => native/core}/build.rs | 0 {core => native/core}/rustfmt.toml | 0 {core => native/core}/src/common/bit.rs | 0 {core => native/core}/src/common/buffer.rs | 0 {core => native/core}/src/common/mod.rs | 0 {core => native/core}/src/data_type.rs | 0 {core => native/core}/src/errors.rs | 2 +- .../execution/datafusion/expressions/abs.rs | 0 .../execution/datafusion/expressions/avg.rs | 0 .../datafusion/expressions/avg_decimal.rs | 0 .../datafusion/expressions/bitwise_not.rs | 0 .../expressions/bloom_filter_might_contain.rs | 0 .../execution/datafusion/expressions/cast.rs | 0 .../datafusion/expressions/checkoverflow.rs | 0 .../datafusion/expressions/correlation.rs | 0 .../datafusion/expressions/covariance.rs | 0 .../expressions/create_named_struct.rs | 0 .../datafusion/expressions/if_expr.rs | 0 .../execution/datafusion/expressions/mod.rs | 0 .../datafusion/expressions/negative.rs | 0 .../datafusion/expressions/normalize_nan.rs | 0 .../datafusion/expressions/scalar_funcs.rs | 0 .../expressions/scalar_funcs/chr.rs | 0 .../scalar_funcs/hash_expressions.rs | 0 .../expressions/scalar_funcs/hex.rs | 0 .../expressions/scalar_funcs/unhex.rs | 0 .../execution/datafusion/expressions/stats.rs | 0 .../datafusion/expressions/stddev.rs | 0 .../datafusion/expressions/strings.rs | 0 .../datafusion/expressions/subquery.rs | 0 .../datafusion/expressions/sum_decimal.rs | 0 .../datafusion/expressions/temporal.rs | 0 .../datafusion/expressions/unbound.rs | 0 .../execution/datafusion/expressions/utils.rs | 0 .../datafusion/expressions/variance.rs | 0 .../datafusion/expressions/xxhash64.rs | 0 .../core}/src/execution/datafusion/mod.rs | 0 .../execution/datafusion/operators/expand.rs | 0 .../src/execution/datafusion/operators/mod.rs | 0 .../core}/src/execution/datafusion/planner.rs | 0 .../execution/datafusion/shuffle_writer.rs | 0 .../src/execution/datafusion/spark_hash.rs | 0 .../src/execution/datafusion/util/mod.rs | 0 .../datafusion/util/spark_bit_array.rs | 0 .../datafusion/util/spark_bloom_filter.rs | 0 .../core}/src/execution/jni_api.rs | 0 .../core}/src/execution/kernels/hash.rs | 0 .../core}/src/execution/kernels/mod.rs | 0 .../core}/src/execution/kernels/strings.rs | 0 .../core}/src/execution/kernels/temporal.rs | 0 .../core}/src/execution/memory_pool.rs | 0 .../core}/src/execution/metrics/mod.rs | 0 .../core}/src/execution/metrics/utils.rs | 0 {core => native/core}/src/execution/mod.rs | 0 .../core}/src/execution/operators/copy.rs | 0 .../core}/src/execution/operators/mod.rs | 0 .../core}/src/execution/operators/scan.rs | 0 .../core}/src/execution/proto/expr.proto | 0 .../core}/src/execution/proto/operator.proto | 0 .../src/execution/proto/partitioning.proto | 0 {core => native/core}/src/execution/serde.rs | 0 .../core}/src/execution/shuffle/list.rs | 0 .../core}/src/execution/shuffle/map.rs | 0 .../core}/src/execution/shuffle/mod.rs | 0 .../core}/src/execution/shuffle/row.rs | 0 {core => native/core}/src/execution/sort.rs | 0 .../core}/src/execution/timezone.rs | 0 {core => native/core}/src/execution/utils.rs | 0 .../core}/src/jvm_bridge/batch_iterator.rs | 0 .../core}/src/jvm_bridge/comet_exec.rs | 0 .../core}/src/jvm_bridge/comet_metric_node.rs | 0 .../jvm_bridge/comet_task_memory_manager.rs | 0 {core => native/core}/src/jvm_bridge/mod.rs | 0 {core => native/core}/src/lib.rs | 0 .../core}/src/parquet/compression.rs | 0 .../core}/src/parquet/data_type.rs | 0 {core => native/core}/src/parquet/mod.rs | 0 .../core}/src/parquet/mutable_vector.rs | 0 .../core}/src/parquet/read/column.rs | 0 .../core}/src/parquet/read/levels.rs | 0 {core => native/core}/src/parquet/read/mod.rs | 0 .../core}/src/parquet/read/values.rs | 0 .../core}/src/parquet/util/bit_packing.rs | 0 .../core}/src/parquet/util/buffer.rs | 0 {core => native/core}/src/parquet/util/jni.rs | 0 .../core}/src/parquet/util/jni_buffer.rs | 0 .../core}/src/parquet/util/memory.rs | 0 {core => native/core}/src/parquet/util/mod.rs | 0 .../src/parquet/util/test_common/file_util.rs | 0 .../core}/src/parquet/util/test_common/mod.rs | 0 .../src/parquet/util/test_common/page_util.rs | 0 .../src/parquet/util/test_common/rand_gen.rs | 0 {core => native/core}/testdata/backtrace.txt | 0 {core => native/core}/testdata/stacktrace.txt | 0 pom.xml | 4 +- spark/pom.xml | 2 +- 117 files changed, 3369 insertions(+), 36 deletions(-) create mode 100644 native/Cargo.lock create mode 100644 native/Cargo.toml rename {core => native/core}/Cargo.lock (100%) rename {core => native/core}/Cargo.toml (100%) rename {core => native/core}/benches/bit_util.rs (100%) rename {core => native/core}/benches/cast_from_string.rs (100%) rename {core => native/core}/benches/cast_numeric.rs (100%) rename {core => native/core}/benches/common.rs (100%) rename {core => native/core}/benches/hash.rs (100%) rename {core => native/core}/benches/parquet_decode.rs (100%) rename {core => native/core}/benches/parquet_read.rs (100%) rename {core => native/core}/benches/perf.rs (100%) rename {core => native/core}/benches/row_columnar.rs (100%) rename {core => native/core}/benches/shuffle_writer.rs (100%) rename {core => native/core}/build.rs (100%) rename {core => native/core}/rustfmt.toml (100%) rename {core => native/core}/src/common/bit.rs (100%) rename {core => native/core}/src/common/buffer.rs (100%) rename {core => native/core}/src/common/mod.rs (100%) rename {core => native/core}/src/data_type.rs (100%) rename {core => native/core}/src/errors.rs (99%) rename {core => native/core}/src/execution/datafusion/expressions/abs.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/avg.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/avg_decimal.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/bitwise_not.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/bloom_filter_might_contain.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/cast.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/checkoverflow.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/correlation.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/covariance.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/create_named_struct.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/if_expr.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/mod.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/negative.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/normalize_nan.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/scalar_funcs.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/scalar_funcs/chr.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/scalar_funcs/hash_expressions.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/scalar_funcs/hex.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/scalar_funcs/unhex.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/stats.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/stddev.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/strings.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/subquery.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/sum_decimal.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/temporal.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/unbound.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/utils.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/variance.rs (100%) rename {core => native/core}/src/execution/datafusion/expressions/xxhash64.rs (100%) rename {core => native/core}/src/execution/datafusion/mod.rs (100%) rename {core => native/core}/src/execution/datafusion/operators/expand.rs (100%) rename {core => native/core}/src/execution/datafusion/operators/mod.rs (100%) rename {core => native/core}/src/execution/datafusion/planner.rs (100%) rename {core => native/core}/src/execution/datafusion/shuffle_writer.rs (100%) rename {core => native/core}/src/execution/datafusion/spark_hash.rs (100%) rename {core => native/core}/src/execution/datafusion/util/mod.rs (100%) rename {core => native/core}/src/execution/datafusion/util/spark_bit_array.rs (100%) rename {core => native/core}/src/execution/datafusion/util/spark_bloom_filter.rs (100%) rename {core => native/core}/src/execution/jni_api.rs (100%) rename {core => native/core}/src/execution/kernels/hash.rs (100%) rename {core => native/core}/src/execution/kernels/mod.rs (100%) rename {core => native/core}/src/execution/kernels/strings.rs (100%) rename {core => native/core}/src/execution/kernels/temporal.rs (100%) rename {core => native/core}/src/execution/memory_pool.rs (100%) rename {core => native/core}/src/execution/metrics/mod.rs (100%) rename {core => native/core}/src/execution/metrics/utils.rs (100%) rename {core => native/core}/src/execution/mod.rs (100%) rename {core => native/core}/src/execution/operators/copy.rs (100%) rename {core => native/core}/src/execution/operators/mod.rs (100%) rename {core => native/core}/src/execution/operators/scan.rs (100%) rename {core => native/core}/src/execution/proto/expr.proto (100%) rename {core => native/core}/src/execution/proto/operator.proto (100%) rename {core => native/core}/src/execution/proto/partitioning.proto (100%) rename {core => native/core}/src/execution/serde.rs (100%) rename {core => native/core}/src/execution/shuffle/list.rs (100%) rename {core => native/core}/src/execution/shuffle/map.rs (100%) rename {core => native/core}/src/execution/shuffle/mod.rs (100%) rename {core => native/core}/src/execution/shuffle/row.rs (100%) rename {core => native/core}/src/execution/sort.rs (100%) rename {core => native/core}/src/execution/timezone.rs (100%) rename {core => native/core}/src/execution/utils.rs (100%) rename {core => native/core}/src/jvm_bridge/batch_iterator.rs (100%) rename {core => native/core}/src/jvm_bridge/comet_exec.rs (100%) rename {core => native/core}/src/jvm_bridge/comet_metric_node.rs (100%) rename {core => native/core}/src/jvm_bridge/comet_task_memory_manager.rs (100%) rename {core => native/core}/src/jvm_bridge/mod.rs (100%) rename {core => native/core}/src/lib.rs (100%) rename {core => native/core}/src/parquet/compression.rs (100%) rename {core => native/core}/src/parquet/data_type.rs (100%) rename {core => native/core}/src/parquet/mod.rs (100%) rename {core => native/core}/src/parquet/mutable_vector.rs (100%) rename {core => native/core}/src/parquet/read/column.rs (100%) rename {core => native/core}/src/parquet/read/levels.rs (100%) rename {core => native/core}/src/parquet/read/mod.rs (100%) rename {core => native/core}/src/parquet/read/values.rs (100%) rename {core => native/core}/src/parquet/util/bit_packing.rs (100%) rename {core => native/core}/src/parquet/util/buffer.rs (100%) rename {core => native/core}/src/parquet/util/jni.rs (100%) rename {core => native/core}/src/parquet/util/jni_buffer.rs (100%) rename {core => native/core}/src/parquet/util/memory.rs (100%) rename {core => native/core}/src/parquet/util/mod.rs (100%) rename {core => native/core}/src/parquet/util/test_common/file_util.rs (100%) rename {core => native/core}/src/parquet/util/test_common/mod.rs (100%) rename {core => native/core}/src/parquet/util/test_common/page_util.rs (100%) rename {core => native/core}/src/parquet/util/test_common/rand_gen.rs (100%) rename {core => native/core}/testdata/backtrace.txt (100%) rename {core => native/core}/testdata/stacktrace.txt (100%) diff --git a/.github/actions/java-test/action.yaml b/.github/actions/java-test/action.yaml index e1efd9fce..caf3f6bba 100644 --- a/.github/actions/java-test/action.yaml +++ b/.github/actions/java-test/action.yaml @@ -33,7 +33,7 @@ runs: - name: Run Cargo build shell: bash run: | - cd core + cd native cargo build - name: Cache Maven dependencies diff --git a/.github/actions/rust-test/action.yaml b/.github/actions/rust-test/action.yaml index bf0d0ba15..b543bcf00 100644 --- a/.github/actions/rust-test/action.yaml +++ b/.github/actions/rust-test/action.yaml @@ -22,19 +22,19 @@ runs: - name: Check Cargo fmt shell: bash run: | - cd core + cd native cargo fmt --all -- --check --color=never - name: Check Cargo clippy shell: bash run: | - cd core + cd native cargo clippy --color=never -- -D warnings - name: Check compilation shell: bash run: | - cd core + cd native cargo check --benches - name: Cache Maven dependencies @@ -56,5 +56,5 @@ runs: - name: Run Cargo test shell: bash run: | - cd core + cd native RUST_BACKTRACE=1 cargo test diff --git a/.github/workflows/benchmark-tpch.yml b/.github/workflows/benchmark-tpch.yml index f4c547a8f..3d4fbb750 100644 --- a/.github/workflows/benchmark-tpch.yml +++ b/.github/workflows/benchmark-tpch.yml @@ -76,8 +76,8 @@ jobs: with: name: libcomet-${{ github.run_id }} path: | - core/target/release/libcomet.so - core/target/release/libcomet.dylib + native/target/release/libcomet.so + native/target/release/libcomet.dylib retention-days: 1 # remove the artifact after 1 day, only valid for this workflow overwrite: true - name: Generate TPC-H (SF=1) table data @@ -119,7 +119,7 @@ jobs: uses: actions/download-artifact@v4 with: name: libcomet-${{ github.run_id }} - path: core/target/release + path: native/target/release - name: Run TPC-H queries run: | SPARK_HOME=`pwd` SPARK_TPCH_DATA=`pwd`/tpch/sf1_parquet ./mvnw -B -Prelease -Dsuites=org.apache.spark.sql.CometTPCHQuerySuite test diff --git a/.github/workflows/benchmark.yml b/.github/workflows/benchmark.yml index 023b6a685..de1ad5773 100644 --- a/.github/workflows/benchmark.yml +++ b/.github/workflows/benchmark.yml @@ -83,8 +83,8 @@ jobs: with: name: libcomet-${{ github.run_id }} path: | - core/target/release/libcomet.so - core/target/release/libcomet.dylib + native/target/release/libcomet.so + native/target/release/libcomet.dylib retention-days: 1 # remove the artifact after 1 day, only valid for this workflow overwrite: true - name: Build tpcds-kit @@ -134,7 +134,7 @@ jobs: uses: actions/download-artifact@v4 with: name: libcomet-${{ github.run_id }} - path: core/target/release + path: native/target/release - name: Run TPC-DS queries (Sort merge join) if: matrix.join == 'sort_merge' run: | diff --git a/.gitignore b/.gitignore index 1c247dd9a..8bdcd51d3 100644 --- a/.gitignore +++ b/.gitignore @@ -8,7 +8,7 @@ derby.log metastore_db/ spark-warehouse/ dependency-reduced-pom.xml -core/src/execution/generated +native/core/src/execution/generated prebuild .flattened-pom.xml rat.txt diff --git a/Makefile b/Makefile index 573a7f955..a52524511 100644 --- a/Makefile +++ b/Makefile @@ -20,11 +20,11 @@ all: core jvm core: - cd core && cargo build + cd native && cargo build test-rust: # We need to compile CometException so that the cargo test can pass ./mvnw compile -pl common -DskipTests $(PROFILES) - cd core && cargo build && \ + cd native && cargo build && \ RUST_BACKTRACE=1 cargo test jvm: ./mvnw clean package -DskipTests $(PROFILES) @@ -32,24 +32,24 @@ test-jvm: core SPARK_HOME=`pwd` COMET_CONF_DIR=$(shell pwd)/conf RUST_BACKTRACE=1 ./mvnw verify $(PROFILES) test: test-rust test-jvm clean: - cd core && cargo clean + cd native && cargo clean ./mvnw clean $(PROFILES) rm -rf .dist bench: - cd core && RUSTFLAGS="-Ctarget-cpu=native" cargo bench $(filter-out $@,$(MAKECMDGOALS)) + cd native && RUSTFLAGS="-Ctarget-cpu=native" cargo bench $(filter-out $@,$(MAKECMDGOALS)) format: - cd core && cargo fmt + cd native && cargo fmt ./mvnw compile test-compile scalafix:scalafix -Psemanticdb $(PROFILES) ./mvnw spotless:apply $(PROFILES) core-amd64: rustup target add x86_64-apple-darwin - cd core && RUSTFLAGS="-Ctarget-cpu=skylake -Ctarget-feature=-prefer-256-bit" CC=o64-clang CXX=o64-clang++ cargo build --target x86_64-apple-darwin --release + cd native && RUSTFLAGS="-Ctarget-cpu=skylake -Ctarget-feature=-prefer-256-bit" CC=o64-clang CXX=o64-clang++ cargo build --target x86_64-apple-darwin --release mkdir -p common/target/classes/org/apache/comet/darwin/x86_64 - cp core/target/x86_64-apple-darwin/release/libcomet.dylib common/target/classes/org/apache/comet/darwin/x86_64 - cd core && RUSTFLAGS="-Ctarget-cpu=haswell -Ctarget-feature=-prefer-256-bit" cargo build --release + cp native/target/x86_64-apple-darwin/release/libcomet.dylib common/target/classes/org/apache/comet/darwin/x86_64 + cd native && RUSTFLAGS="-Ctarget-cpu=haswell -Ctarget-feature=-prefer-256-bit" cargo build --release mkdir -p common/target/classes/org/apache/comet/linux/amd64 - cp core/target/release/libcomet.so common/target/classes/org/apache/comet/linux/amd64 + cp native/target/release/libcomet.so common/target/classes/org/apache/comet/linux/amd64 jar -cf common/target/comet-native-x86_64.jar \ -C common/target/classes/org/apache/comet darwin \ -C common/target/classes/org/apache/comet linux @@ -57,12 +57,12 @@ core-amd64: core-arm64: rustup target add aarch64-apple-darwin - cd core && RUSTFLAGS="-Ctarget-cpu=apple-m1" CC=arm64-apple-darwin21.4-clang CXX=arm64-apple-darwin21.4-clang++ CARGO_FEATURE_NEON=1 cargo build --target aarch64-apple-darwin --release + cd native && RUSTFLAGS="-Ctarget-cpu=apple-m1" CC=arm64-apple-darwin21.4-clang CXX=arm64-apple-darwin21.4-clang++ CARGO_FEATURE_NEON=1 cargo build --target aarch64-apple-darwin --release mkdir -p common/target/classes/org/apache/comet/darwin/aarch64 - cp core/target/aarch64-apple-darwin/release/libcomet.dylib common/target/classes/org/apache/comet/darwin/aarch64 - cd core && RUSTFLAGS="-Ctarget-cpu=native" cargo build --release + cp native/target/aarch64-apple-darwin/release/libcomet.dylib common/target/classes/org/apache/comet/darwin/aarch64 + cd native && RUSTFLAGS="-Ctarget-cpu=native" cargo build --release mkdir -p common/target/classes/org/apache/comet/linux/aarch64 - cp core/target/release/libcomet.so common/target/classes/org/apache/comet/linux/aarch64 + cp native/target/release/libcomet.so common/target/classes/org/apache/comet/linux/aarch64 jar -cf common/target/comet-native-aarch64.jar \ -C common/target/classes/org/apache/comet darwin \ -C common/target/classes/org/apache/comet linux @@ -70,15 +70,15 @@ core-arm64: release-linux: clean rustup target add aarch64-apple-darwin x86_64-apple-darwin - cd core && RUSTFLAGS="-Ctarget-cpu=apple-m1" CC=arm64-apple-darwin21.4-clang CXX=arm64-apple-darwin21.4-clang++ CARGO_FEATURE_NEON=1 cargo build --target aarch64-apple-darwin --release - cd core && RUSTFLAGS="-Ctarget-cpu=skylake -Ctarget-feature=-prefer-256-bit" CC=o64-clang CXX=o64-clang++ cargo build --target x86_64-apple-darwin --release - cd core && RUSTFLAGS="-Ctarget-cpu=native -Ctarget-feature=-prefer-256-bit" cargo build --release + cd native && RUSTFLAGS="-Ctarget-cpu=apple-m1" CC=arm64-apple-darwin21.4-clang CXX=arm64-apple-darwin21.4-clang++ CARGO_FEATURE_NEON=1 cargo build --target aarch64-apple-darwin --release + cd native && RUSTFLAGS="-Ctarget-cpu=skylake -Ctarget-feature=-prefer-256-bit" CC=o64-clang CXX=o64-clang++ cargo build --target x86_64-apple-darwin --release + cd native && RUSTFLAGS="-Ctarget-cpu=native -Ctarget-feature=-prefer-256-bit" cargo build --release ./mvnw install -Prelease -DskipTests $(PROFILES) release: - cd core && RUSTFLAGS="-Ctarget-cpu=native" cargo build --release + cd native && RUSTFLAGS="-Ctarget-cpu=native" cargo build --release ./mvnw install -Prelease -DskipTests $(PROFILES) release-nogit: - cd core && RUSTFLAGS="-Ctarget-cpu=native" cargo build --features nightly --release + cd native && RUSTFLAGS="-Ctarget-cpu=native" cargo build --features nightly --release ./mvnw install -Prelease -DskipTests $(PROFILES) -Dmaven.gitcommitid.skip=true benchmark-%: clean release cd spark && COMET_CONF_DIR=$(shell pwd)/conf MAVEN_OPTS='-Xmx20g' ../mvnw exec:java -Dexec.mainClass="$*" -Dexec.classpathScope="test" -Dexec.cleanupDaemonThreads="false" -Dexec.args="$(filter-out $@,$(MAKECMDGOALS))" $(PROFILES) diff --git a/common/pom.xml b/common/pom.xml index b59d7b187..b912e8bd5 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -193,14 +193,14 @@ under the License. ${project.basedir}/src/main/resources - ${project.basedir}/../core/target/x86_64-apple-darwin/release + ${project.basedir}/../native/target/x86_64-apple-darwin/release libcomet.dylib org/apache/comet/darwin/x86_64 - ${project.basedir}/../core/target/aarch64-apple-darwin/release + ${project.basedir}/../native/target/aarch64-apple-darwin/release libcomet.dylib diff --git a/native/Cargo.lock b/native/Cargo.lock new file mode 100644 index 000000000..ccb6433d9 --- /dev/null +++ b/native/Cargo.lock @@ -0,0 +1,3301 @@ +# This file is automatically @generated by Cargo. +# It is not intended for manual editing. +version = 3 + +[[package]] +name = "addr2line" +version = "0.22.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6e4503c46a5c0c7844e948c9a4d6acd9f50cccb4de1c48eb9e291ea17470c678" +dependencies = [ + "gimli", +] + +[[package]] +name = "adler" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" + +[[package]] +name = "ahash" +version = "0.8.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e89da841a80418a9b391ebaea17f5c112ffaaa96f621d2c285b5174da76b9011" +dependencies = [ + "cfg-if", + "const-random", + "getrandom", + "once_cell", + "version_check", + "zerocopy", +] + +[[package]] +name = "aho-corasick" +version = "1.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8e60d3430d3a69478ad0993f19238d2df97c507009a52b3c10addcd7f6bcb916" +dependencies = [ + "memchr", +] + +[[package]] +name = "alloc-no-stdlib" +version = "2.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cc7bb162ec39d46ab1ca8c77bf72e890535becd1751bb45f64c597edb4c8c6b3" + +[[package]] +name = "alloc-stdlib" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "94fb8275041c72129eb51b7d0322c29b8387a0386127718b096429201a5d6ece" +dependencies = [ + "alloc-no-stdlib", +] + +[[package]] +name = "allocator-api2" +version = "0.2.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c6cb57a04249c6480766f7f7cef5467412af1490f8d1e243141daddada3264f" + +[[package]] +name = "android-tzdata" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e999941b234f3131b00bc13c22d06e8c5ff726d1b6318ac7eb276997bbb4fef0" + +[[package]] +name = "android_system_properties" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "819e7219dbd41043ac279b19830f2efc897156490d7fd6ea916720117ee66311" +dependencies = [ + "libc", +] + +[[package]] +name = "anes" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4b46cbb362ab8752921c97e041f5e366ee6297bd428a31275b9fcf1e380f7299" + +[[package]] +name = "anstyle" +version = "1.0.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "038dfcf04a5feb68e9c60b21c9625a54c2c0616e79b72b0fd87075a056ae1d1b" + +[[package]] +name = "anyhow" +version = "1.0.86" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b3d1d046238990b9cf5bcde22a3fb3584ee5cf65fb2765f454ed428c7a0063da" + +[[package]] +name = "arc-swap" +version = "1.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "69f7f8c3906b62b754cd5326047894316021dcfe5a194c8ea52bdd94934a3457" + +[[package]] +name = "arrayref" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6b4930d2cb77ce62f89ee5d5289b4ac049559b1c45539271f5ed4fdc7db34545" + +[[package]] +name = "arrayvec" +version = "0.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" + +[[package]] +name = "arrow" +version = "52.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +dependencies = [ + "arrow-arith", + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-csv", + "arrow-data", + "arrow-ipc", + "arrow-json", + "arrow-ord", + "arrow-row", + "arrow-schema", + "arrow-select", + "arrow-string", +] + +[[package]] +name = "arrow-arith" +version = "52.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "chrono", + "half", + "num", +] + +[[package]] +name = "arrow-array" +version = "52.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +dependencies = [ + "ahash", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "chrono", + "chrono-tz 0.9.0", + "half", + "hashbrown", + "num", +] + +[[package]] +name = "arrow-buffer" +version = "52.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +dependencies = [ + "bytes", + "half", + "num", +] + +[[package]] +name = "arrow-cast" +version = "52.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "arrow-select", + "atoi", + "base64", + "chrono", + "comfy-table", + "half", + "lexical-core", + "num", + "ryu", +] + +[[package]] +name = "arrow-csv" +version = "52.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-schema", + "chrono", + "csv", + "csv-core", + "lazy_static", + "lexical-core", + "regex", +] + +[[package]] +name = "arrow-data" +version = "52.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +dependencies = [ + "arrow-buffer", + "arrow-schema", + "half", + "num", +] + +[[package]] +name = "arrow-ipc" +version = "52.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-schema", + "flatbuffers", + "lz4_flex", +] + +[[package]] +name = "arrow-json" +version = "52.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-schema", + "chrono", + "half", + "indexmap", + "lexical-core", + "num", + "serde", + "serde_json", +] + +[[package]] +name = "arrow-ord" +version = "52.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "arrow-select", + "half", + "num", +] + +[[package]] +name = "arrow-row" +version = "52.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +dependencies = [ + "ahash", + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "half", + "hashbrown", +] + +[[package]] +name = "arrow-schema" +version = "52.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +dependencies = [ + "bitflags 2.6.0", +] + +[[package]] +name = "arrow-select" +version = "52.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +dependencies = [ + "ahash", + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "num", +] + +[[package]] +name = "arrow-string" +version = "52.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "arrow-select", + "memchr", + "num", + "regex", + "regex-syntax", +] + +[[package]] +name = "assertables" +version = "7.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c24e9d990669fbd16806bff449e4ac644fd9b1fca014760087732fe4102f131" + +[[package]] +name = "async-trait" +version = "0.1.80" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c6fa2087f2753a7da8cc1c0dbfcf89579dd57458e36769de5ac750b4671737ca" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.68", +] + +[[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.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c4b4d0bd25bd0b74681c0ad21497610ce1b7c91b1022cd21c80c6fbdd9476b0" + +[[package]] +name = "backtrace" +version = "0.3.73" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5cc23269a4f8976d0a4d2e7109211a419fe30e8d88d677cd60b6bc79c5732e0a" +dependencies = [ + "addr2line", + "cc", + "cfg-if", + "libc", + "miniz_oxide", + "object", + "rustc-demangle", +] + +[[package]] +name = "base64" +version = "0.22.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72b3254f16251a8381aa12e40e3c4d2f0199f8c6508fbecb9d91f575e0fbb8c6" + +[[package]] +name = "bitflags" +version = "1.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" + +[[package]] +name = "bitflags" +version = "2.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b048fb63fd8b5923fc5aa7b340d8e156aec7ec02f0c78fa8a6ddc2613f6f71de" + +[[package]] +name = "blake2" +version = "0.10.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "46502ad458c9a52b69d4d4d32775c788b7a1b85e8bc9d482d92250fc0e3f8efe" +dependencies = [ + "digest", +] + +[[package]] +name = "blake3" +version = "1.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "30cca6d3674597c30ddf2c587bf8d9d65c9a84d2326d941cc79c9842dfe0ef52" +dependencies = [ + "arrayref", + "arrayvec", + "cc", + "cfg-if", + "constant_time_eq", +] + +[[package]] +name = "block-buffer" +version = "0.10.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3078c7629b62d3f0439517fa394996acacc5cbc91c5a20d8c658e77abd503a71" +dependencies = [ + "generic-array", +] + +[[package]] +name = "brotli" +version = "3.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d640d25bc63c50fb1f0b545ffd80207d2e10a4c965530809b40ba3386825c391" +dependencies = [ + "alloc-no-stdlib", + "alloc-stdlib", + "brotli-decompressor", +] + +[[package]] +name = "brotli-decompressor" +version = "2.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4e2e4afe60d7dd600fdd3de8d0f08c2b7ec039712e3b6137ff98b7004e82de4f" +dependencies = [ + "alloc-no-stdlib", + "alloc-stdlib", +] + +[[package]] +name = "bumpalo" +version = "3.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "79296716171880943b8470b5f8d03aa55eb2e645a4874bdbb28adb49162e012c" + +[[package]] +name = "bytemuck" +version = "1.16.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b236fc92302c97ed75b38da1f4917b5cdda4984745740f153a5d3059e48d725e" + +[[package]] +name = "byteorder" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" + +[[package]] +name = "bytes" +version = "1.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "514de17de45fdb8dc022b1a7975556c53c86f9f0aa5f534b98977b171857c2c9" + +[[package]] +name = "cast" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" + +[[package]] +name = "cc" +version = "1.0.104" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "74b6a57f98764a267ff415d50a25e6e166f3831a5071af4995296ea97d210490" +dependencies = [ + "jobserver", + "libc", + "once_cell", +] + +[[package]] +name = "cesu8" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6d43a04d8753f35258c91f8ec639f792891f748a1edbd759cf1dcea3382ad83c" + +[[package]] +name = "cfg-if" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" + +[[package]] +name = "chrono" +version = "0.4.38" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a21f936df1771bf62b77f047b726c4625ff2e8aa607c01ec06e5a05bd8463401" +dependencies = [ + "android-tzdata", + "iana-time-zone", + "num-traits", + "windows-targets 0.52.6", +] + +[[package]] +name = "chrono-tz" +version = "0.8.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d59ae0466b83e838b81a54256c39d5d7c20b9d7daa10510a242d9b75abd5936e" +dependencies = [ + "chrono", + "chrono-tz-build 0.2.1", + "phf", +] + +[[package]] +name = "chrono-tz" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "93698b29de5e97ad0ae26447b344c482a7284c737d9ddc5f9e52b74a336671bb" +dependencies = [ + "chrono", + "chrono-tz-build 0.3.0", + "phf", +] + +[[package]] +name = "chrono-tz-build" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "433e39f13c9a060046954e0592a8d0a4bcb1040125cbf91cb8ee58964cfb350f" +dependencies = [ + "parse-zoneinfo", + "phf", + "phf_codegen", +] + +[[package]] +name = "chrono-tz-build" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c088aee841df9c3041febbb73934cfc39708749bf96dc827e3359cd39ef11b1" +dependencies = [ + "parse-zoneinfo", + "phf", + "phf_codegen", +] + +[[package]] +name = "ciborium" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "42e69ffd6f0917f5c029256a24d0161db17cea3997d185db0d35926308770f0e" +dependencies = [ + "ciborium-io", + "ciborium-ll", + "serde", +] + +[[package]] +name = "ciborium-io" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "05afea1e0a06c9be33d539b876f1ce3692f4afea2cb41f740e7743225ed1c757" + +[[package]] +name = "ciborium-ll" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "57663b653d948a338bfb3eeba9bb2fd5fcfaecb9e199e87e1eda4d9e8b240fd9" +dependencies = [ + "ciborium-io", + "half", +] + +[[package]] +name = "clap" +version = "4.5.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "84b3edb18336f4df585bc9aa31dd99c036dfa5dc5e9a2939a722a188f3a8970d" +dependencies = [ + "clap_builder", +] + +[[package]] +name = "clap_builder" +version = "4.5.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c1c09dd5ada6c6c78075d6fd0da3f90d8080651e2d6cc8eb2f1aaa4034ced708" +dependencies = [ + "anstyle", + "clap_lex", +] + +[[package]] +name = "clap_lex" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4b82cf0babdbd58558212896d1a4272303a57bdb245c2bf1147185fb45640e70" + +[[package]] +name = "combine" +version = "4.6.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ba5a308b75df32fe02788e748662718f03fde005016435c444eea572398219fd" +dependencies = [ + "bytes", + "memchr", +] + +[[package]] +name = "comfy-table" +version = "7.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b34115915337defe99b2aff5c2ce6771e5fbc4079f4b506301f5cf394c8452f7" +dependencies = [ + "strum", + "strum_macros", + "unicode-width", +] + +[[package]] +name = "const-random" +version = "0.1.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87e00182fe74b066627d63b85fd550ac2998d4b0bd86bfed477a0ae4c7c71359" +dependencies = [ + "const-random-macro", +] + +[[package]] +name = "const-random-macro" +version = "0.1.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f9d839f2a20b0aee515dc581a6172f2321f96cab76c1a38a4c584a194955390e" +dependencies = [ + "getrandom", + "once_cell", + "tiny-keccak", +] + +[[package]] +name = "constant_time_eq" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f7144d30dcf0fafbce74250a3963025d8d52177934239851c917d29f1df280c2" + +[[package]] +name = "core-foundation-sys" +version = "0.8.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06ea2b9bc92be3c2baa9334a323ebca2d6f074ff852cd1d7b11064035cd3868f" + +[[package]] +name = "cpp_demangle" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7e8227005286ec39567949b33df9896bcadfa6051bccca2488129f108ca23119" +dependencies = [ + "cfg-if", +] + +[[package]] +name = "cpufeatures" +version = "0.2.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "53fe5e26ff1b7aef8bca9c6080520cfb8d9333c7568e1829cef191a9723e5504" +dependencies = [ + "libc", +] + +[[package]] +name = "crc32fast" +version = "1.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a97769d94ddab943e4510d138150169a2758b5ef3eb191a9ee688de3e23ef7b3" +dependencies = [ + "cfg-if", +] + +[[package]] +name = "criterion" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f2b12d017a929603d80db1831cd3a24082f8137ce19c69e6447f54f5fc8d692f" +dependencies = [ + "anes", + "cast", + "ciborium", + "clap", + "criterion-plot", + "is-terminal", + "itertools 0.10.5", + "num-traits", + "once_cell", + "oorandom", + "plotters", + "rayon", + "regex", + "serde", + "serde_derive", + "serde_json", + "tinytemplate", + "walkdir", +] + +[[package]] +name = "criterion-plot" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6b50826342786a51a89e2da3a28f1c32b06e387201bc2d19791f622c673706b1" +dependencies = [ + "cast", + "itertools 0.10.5", +] + +[[package]] +name = "crossbeam-deque" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "613f8cc01fe9cf1a3eb3d7f488fd2fa8388403e97039e2f73692932e291a770d" +dependencies = [ + "crossbeam-epoch", + "crossbeam-utils", +] + +[[package]] +name = "crossbeam-epoch" +version = "0.9.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5b82ac4a3c2ca9c3460964f020e1402edd5753411d7737aa39c3714ad1b5420e" +dependencies = [ + "crossbeam-utils", +] + +[[package]] +name = "crossbeam-utils" +version = "0.8.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "22ec99545bb0ed0ea7bb9b8e1e9122ea386ff8a48c0922e43f36d45ab09e0e80" + +[[package]] +name = "crunchy" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7a81dae078cea95a014a339291cec439d2f232ebe854a9d672b796c6afafa9b7" + +[[package]] +name = "crypto-common" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1bfb12502f3fc46cca1bb51ac28df9d618d813cdc3d2f25b9fe775a34af26bb3" +dependencies = [ + "generic-array", + "typenum", +] + +[[package]] +name = "csv" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ac574ff4d437a7b5ad237ef331c17ccca63c46479e5b5453eb8e10bb99a759fe" +dependencies = [ + "csv-core", + "itoa", + "ryu", + "serde", +] + +[[package]] +name = "csv-core" +version = "0.1.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5efa2b3d7902f4b634a20cae3c9c4e6209dc4779feb6863329607560143efa70" +dependencies = [ + "memchr", +] + +[[package]] +name = "dashmap" +version = "5.5.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "978747c1d849a7d2ee5e8adc0159961c48fb7e5db2f06af6723b80123bb53856" +dependencies = [ + "cfg-if", + "hashbrown", + "lock_api", + "once_cell", + "parking_lot_core", +] + +[[package]] +name = "datafusion" +version = "39.0.0" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +dependencies = [ + "ahash", + "arrow", + "arrow-array", + "arrow-ipc", + "arrow-schema", + "async-trait", + "bytes", + "chrono", + "dashmap", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions", + "datafusion-functions-aggregate", + "datafusion-optimizer", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "datafusion-sql", + "futures", + "glob", + "half", + "hashbrown", + "indexmap", + "itertools 0.12.1", + "log", + "num_cpus", + "object_store", + "parking_lot", + "paste", + "pin-project-lite", + "rand", + "sqlparser", + "tempfile", + "tokio", + "url", + "uuid", +] + +[[package]] +name = "datafusion-comet" +version = "0.1.0" +dependencies = [ + "ahash", + "arrow", + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "arrow-string", + "assertables", + "async-trait", + "brotli", + "byteorder", + "bytes", + "chrono", + "chrono-tz 0.8.6", + "crc32fast", + "criterion", + "datafusion", + "datafusion-common", + "datafusion-expr", + "datafusion-functions", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "flate2", + "futures", + "half", + "hashbrown", + "hex", + "itertools 0.11.0", + "jni", + "lazy_static", + "log", + "log4rs", + "lz4", + "mimalloc", + "num", + "once_cell", + "parking_lot", + "parquet", + "parquet-format", + "paste", + "pprof", + "prost 0.12.6", + "prost-build", + "rand", + "regex", + "serde", + "simd-adler32", + "snap", + "tempfile", + "thiserror", + "thrift 0.17.0", + "tokio", + "tokio-stream", + "twox-hash", + "unicode-segmentation", + "zstd", +] + +[[package]] +name = "datafusion-common" +version = "39.0.0" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +dependencies = [ + "ahash", + "arrow", + "arrow-array", + "arrow-buffer", + "arrow-schema", + "chrono", + "half", + "hashbrown", + "instant", + "libc", + "num_cpus", + "object_store", + "sqlparser", +] + +[[package]] +name = "datafusion-common-runtime" +version = "39.0.0" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +dependencies = [ + "tokio", +] + +[[package]] +name = "datafusion-execution" +version = "39.0.0" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +dependencies = [ + "arrow", + "chrono", + "dashmap", + "datafusion-common", + "datafusion-expr", + "futures", + "hashbrown", + "log", + "object_store", + "parking_lot", + "rand", + "tempfile", + "url", +] + +[[package]] +name = "datafusion-expr" +version = "39.0.0" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +dependencies = [ + "ahash", + "arrow", + "arrow-array", + "arrow-buffer", + "chrono", + "datafusion-common", + "paste", + "serde_json", + "sqlparser", + "strum", + "strum_macros", +] + +[[package]] +name = "datafusion-functions" +version = "39.0.0" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +dependencies = [ + "arrow", + "base64", + "blake2", + "blake3", + "chrono", + "datafusion-common", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "hashbrown", + "hex", + "itertools 0.12.1", + "log", + "md-5", + "rand", + "regex", + "sha2", + "unicode-segmentation", + "uuid", +] + +[[package]] +name = "datafusion-functions-aggregate" +version = "39.0.0" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +dependencies = [ + "ahash", + "arrow", + "arrow-schema", + "datafusion-common", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr-common", + "log", + "paste", + "sqlparser", +] + +[[package]] +name = "datafusion-optimizer" +version = "39.0.0" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +dependencies = [ + "arrow", + "async-trait", + "chrono", + "datafusion-common", + "datafusion-expr", + "datafusion-physical-expr", + "hashbrown", + "indexmap", + "itertools 0.12.1", + "log", + "regex-syntax", +] + +[[package]] +name = "datafusion-physical-expr" +version = "39.0.0" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +dependencies = [ + "ahash", + "arrow", + "arrow-array", + "arrow-buffer", + "arrow-ord", + "arrow-schema", + "arrow-string", + "base64", + "chrono", + "datafusion-common", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions-aggregate", + "datafusion-physical-expr-common", + "half", + "hashbrown", + "hex", + "indexmap", + "itertools 0.12.1", + "log", + "paste", + "petgraph", + "regex", +] + +[[package]] +name = "datafusion-physical-expr-common" +version = "39.0.0" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +dependencies = [ + "arrow", + "datafusion-common", + "datafusion-expr", + "rand", +] + +[[package]] +name = "datafusion-physical-plan" +version = "39.0.0" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +dependencies = [ + "ahash", + "arrow", + "arrow-array", + "arrow-buffer", + "arrow-ord", + "arrow-schema", + "async-trait", + "chrono", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions-aggregate", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "futures", + "half", + "hashbrown", + "indexmap", + "itertools 0.12.1", + "log", + "once_cell", + "parking_lot", + "pin-project-lite", + "rand", + "tokio", +] + +[[package]] +name = "datafusion-sql" +version = "39.0.0" +source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +dependencies = [ + "arrow", + "arrow-array", + "arrow-schema", + "datafusion-common", + "datafusion-expr", + "log", + "regex", + "sqlparser", + "strum", +] + +[[package]] +name = "debugid" +version = "0.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bef552e6f588e446098f6ba40d89ac146c8c7b64aade83c051ee00bb5d2bc18d" +dependencies = [ + "uuid", +] + +[[package]] +name = "derivative" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fcc3dd5e9e9c0b295d6e1e4d811fb6f157d5ffd784b8d202fc62eac8035a770b" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "destructure_traitobject" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3c877555693c14d2f84191cfd3ad8582790fc52b5e2274b40b59cf5f5cea25c7" + +[[package]] +name = "digest" +version = "0.10.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292" +dependencies = [ + "block-buffer", + "crypto-common", + "subtle", +] + +[[package]] +name = "doc-comment" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fea41bba32d969b513997752735605054bc0dfa92b4c56bf1189f2e174be7a10" + +[[package]] +name = "either" +version = "1.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "60b1af1c220855b6ceac025d3f6ecdd2b7c4894bfe9cd9bda4fbb4bc7c0d4cf0" + +[[package]] +name = "equivalent" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" + +[[package]] +name = "errno" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "534c5cf6194dfab3db3242765c03bbe257cf92f22b38f6bc0c58d59108a820ba" +dependencies = [ + "libc", + "windows-sys 0.52.0", +] + +[[package]] +name = "fastrand" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9fc0510504f03c51ada170672ac806f1f105a88aa97a5281117e1ddc3368e51a" + +[[package]] +name = "findshlibs" +version = "0.10.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "40b9e59cd0f7e0806cca4be089683ecb6434e602038df21fe6bf6711b2f07f64" +dependencies = [ + "cc", + "lazy_static", + "libc", + "winapi", +] + +[[package]] +name = "fixedbitset" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0ce7134b9999ecaf8bcd65542e436736ef32ddca1b3e06094cb6ec5755203b80" + +[[package]] +name = "flatbuffers" +version = "24.3.25" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8add37afff2d4ffa83bc748a70b4b1370984f6980768554182424ef71447c35f" +dependencies = [ + "bitflags 1.3.2", + "rustc_version", +] + +[[package]] +name = "flate2" +version = "1.0.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5f54427cfd1c7829e2a139fcefea601bf088ebca651d2bf53ebc600eac295dae" +dependencies = [ + "crc32fast", + "miniz_oxide", +] + +[[package]] +name = "fnv" +version = "1.0.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" + +[[package]] +name = "form_urlencoded" +version = "1.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e13624c2627564efccf4934284bdd98cbaa14e79b0b5a141218e507b3a823456" +dependencies = [ + "percent-encoding", +] + +[[package]] +name = "futures" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "645c6916888f6cb6350d2550b80fb63e734897a8498abe35cfb732b6487804b0" +dependencies = [ + "futures-channel", + "futures-core", + "futures-executor", + "futures-io", + "futures-sink", + "futures-task", + "futures-util", +] + +[[package]] +name = "futures-channel" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "eac8f7d7865dcb88bd4373ab671c8cf4508703796caa2b1985a9ca867b3fcb78" +dependencies = [ + "futures-core", + "futures-sink", +] + +[[package]] +name = "futures-core" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dfc6580bb841c5a68e9ef15c77ccc837b40a7504914d52e47b8b0e9bbda25a1d" + +[[package]] +name = "futures-executor" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a576fc72ae164fca6b9db127eaa9a9dda0d61316034f33a0a0d4eda41f02b01d" +dependencies = [ + "futures-core", + "futures-task", + "futures-util", +] + +[[package]] +name = "futures-io" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a44623e20b9681a318efdd71c299b6b222ed6f231972bfe2f224ebad6311f0c1" + +[[package]] +name = "futures-macro" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.68", +] + +[[package]] +name = "futures-sink" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9fb8e00e87438d937621c1c6269e53f536c14d3fbd6a042bb24879e57d474fb5" + +[[package]] +name = "futures-task" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "38d84fa142264698cdce1a9f9172cf383a0c82de1bddcf3092901442c4097004" + +[[package]] +name = "futures-util" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3d6401deb83407ab3da39eba7e33987a73c3df0c82b4bb5813ee871c19c41d48" +dependencies = [ + "futures-channel", + "futures-core", + "futures-io", + "futures-macro", + "futures-sink", + "futures-task", + "memchr", + "pin-project-lite", + "pin-utils", + "slab", +] + +[[package]] +name = "generic-array" +version = "0.14.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "85649ca51fd72272d7821adaf274ad91c288277713d9c18820d8499a7ff69e9a" +dependencies = [ + "typenum", + "version_check", +] + +[[package]] +name = "getrandom" +version = "0.2.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c4567c8db10ae91089c99af84c68c38da3ec2f087c3f82960bcdbf3656b6f4d7" +dependencies = [ + "cfg-if", + "libc", + "wasi", +] + +[[package]] +name = "gimli" +version = "0.29.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "40ecd4077b5ae9fd2e9e169b102c6c330d0605168eb0e8bf79952b256dbefffd" + +[[package]] +name = "glob" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" + +[[package]] +name = "half" +version = "2.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6dd08c532ae367adf81c312a4580bc67f1d0fe8bc9c460520283f4c0ff277888" +dependencies = [ + "cfg-if", + "crunchy", + "num-traits", +] + +[[package]] +name = "hashbrown" +version = "0.14.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e5274423e17b7c9fc20b6e7e208532f9b19825d82dfd615708b70edd83df41f1" +dependencies = [ + "ahash", + "allocator-api2", +] + +[[package]] +name = "heck" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6d621efb26863f0e9924c6ac577e8275e5e6b77455db64ffa6c65c904e9e132c" +dependencies = [ + "unicode-segmentation", +] + +[[package]] +name = "heck" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8" + +[[package]] +name = "heck" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2304e00983f87ffb38b55b444b5e3b60a884b5d30c0fca7d82fe33449bbe55ea" + +[[package]] +name = "hermit-abi" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d231dfb89cfffdbc30e7fc41579ed6066ad03abda9e567ccafae602b97ec5024" + +[[package]] +name = "hex" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f24254aa9a54b5c858eaee2f5bccdb46aaf0e486a595ed5fd8f86ba55232a70" + +[[package]] +name = "home" +version = "0.5.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e3d1354bf6b7235cb4a0576c2619fd4ed18183f689b12b006a0ee7329eeff9a5" +dependencies = [ + "windows-sys 0.52.0", +] + +[[package]] +name = "humantime" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" + +[[package]] +name = "iana-time-zone" +version = "0.1.60" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e7ffbb5a1b541ea2561f8c41c087286cc091e21e556a4f09a8f6cbf17b69b141" +dependencies = [ + "android_system_properties", + "core-foundation-sys", + "iana-time-zone-haiku", + "js-sys", + "wasm-bindgen", + "windows-core", +] + +[[package]] +name = "iana-time-zone-haiku" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f31827a206f56af32e590ba56d5d2d085f558508192593743f16b2306495269f" +dependencies = [ + "cc", +] + +[[package]] +name = "idna" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "634d9b1461af396cad843f47fdba5597a4f9e6ddd4bfb6ff5d85028c25cb12f6" +dependencies = [ + "unicode-bidi", + "unicode-normalization", +] + +[[package]] +name = "indexmap" +version = "2.2.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "168fb715dda47215e360912c096649d23d58bf392ac62f73919e831745e40f26" +dependencies = [ + "equivalent", + "hashbrown", +] + +[[package]] +name = "inferno" +version = "0.11.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "321f0f839cd44a4686e9504b0a62b4d69a50b62072144c71c68f5873c167b8d9" +dependencies = [ + "ahash", + "indexmap", + "is-terminal", + "itoa", + "log", + "num-format", + "once_cell", + "quick-xml", + "rgb", + "str_stack", +] + +[[package]] +name = "instant" +version = "0.1.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e0242819d153cba4b4b05a5a8f2a7e9bbf97b6055b2a002b395c96b5ff3c0222" +dependencies = [ + "cfg-if", + "js-sys", + "wasm-bindgen", + "web-sys", +] + +[[package]] +name = "integer-encoding" +version = "1.1.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "48dc51180a9b377fd75814d0cc02199c20f8e99433d6762f650d39cdbbd3b56f" + +[[package]] +name = "integer-encoding" +version = "3.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8bb03732005da905c88227371639bf1ad885cc712789c011c31c5fb3ab3ccf02" + +[[package]] +name = "is-terminal" +version = "0.4.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f23ff5ef2b80d608d61efee834934d862cd92461afc0560dedf493e4c033738b" +dependencies = [ + "hermit-abi", + "libc", + "windows-sys 0.52.0", +] + +[[package]] +name = "itertools" +version = "0.10.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b0fd2260e829bddf4cb6ea802289de2f86d6a7a690192fbe91b3f46e0f2c8473" +dependencies = [ + "either", +] + +[[package]] +name = "itertools" +version = "0.11.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b1c173a5686ce8bfa551b3563d0c2170bf24ca44da99c7ca4bfdab5418c3fe57" +dependencies = [ + "either", +] + +[[package]] +name = "itertools" +version = "0.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ba291022dbbd398a455acf126c1e341954079855bc60dfdda641363bd6922569" +dependencies = [ + "either", +] + +[[package]] +name = "itoa" +version = "1.0.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "49f1f14873335454500d59611f1cf4a4b0f786f9ac11f4312a78e4cf2566695b" + +[[package]] +name = "java-locator" +version = "0.1.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d2abecabd9961c5e01405a6426687fcf1bd94a269927137e4c3cc1a7419b93fd" +dependencies = [ + "glob", + "lazy_static", +] + +[[package]] +name = "jni" +version = "0.21.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1a87aa2bb7d2af34197c04845522473242e1aa17c12f4935d5856491a7fb8c97" +dependencies = [ + "cesu8", + "cfg-if", + "combine", + "java-locator", + "jni-sys", + "libloading", + "log", + "thiserror", + "walkdir", + "windows-sys 0.45.0", +] + +[[package]] +name = "jni-sys" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8eaf4bc02d17cbdd7ff4c7438cafcdf7fb9a4613313ad11b4f8fefe7d3fa0130" + +[[package]] +name = "jobserver" +version = "0.1.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d2b099aaa34a9751c5bf0878add70444e1ed2dd73f347be99003d4577277de6e" +dependencies = [ + "libc", +] + +[[package]] +name = "js-sys" +version = "0.3.69" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "29c15563dc2726973df627357ce0c9ddddbea194836909d655df6a75d2cf296d" +dependencies = [ + "wasm-bindgen", +] + +[[package]] +name = "lazy_static" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bbd2bcb4c963f2ddae06a2efc7e9f3591312473c50c6685e1f298068316e66fe" + +[[package]] +name = "lexical-core" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2cde5de06e8d4c2faabc400238f9ae1c74d5412d03a7bd067645ccbc47070e46" +dependencies = [ + "lexical-parse-float", + "lexical-parse-integer", + "lexical-util", + "lexical-write-float", + "lexical-write-integer", +] + +[[package]] +name = "lexical-parse-float" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "683b3a5ebd0130b8fb52ba0bdc718cc56815b6a097e28ae5a6997d0ad17dc05f" +dependencies = [ + "lexical-parse-integer", + "lexical-util", + "static_assertions", +] + +[[package]] +name = "lexical-parse-integer" +version = "0.8.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6d0994485ed0c312f6d965766754ea177d07f9c00c9b82a5ee62ed5b47945ee9" +dependencies = [ + "lexical-util", + "static_assertions", +] + +[[package]] +name = "lexical-util" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5255b9ff16ff898710eb9eb63cb39248ea8a5bb036bea8085b1a767ff6c4e3fc" +dependencies = [ + "static_assertions", +] + +[[package]] +name = "lexical-write-float" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "accabaa1c4581f05a3923d1b4cfd124c329352288b7b9da09e766b0668116862" +dependencies = [ + "lexical-util", + "lexical-write-integer", + "static_assertions", +] + +[[package]] +name = "lexical-write-integer" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e1b6f3d1f4422866b68192d62f77bc5c700bee84f3069f2469d7bc8c77852446" +dependencies = [ + "lexical-util", + "static_assertions", +] + +[[package]] +name = "libc" +version = "0.2.155" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "97b3888a4aecf77e811145cadf6eef5901f4782c53886191b2f693f24761847c" + +[[package]] +name = "libloading" +version = "0.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b67380fd3b2fbe7527a606e18729d21c6f3951633d0500574c4dc22d2d638b9f" +dependencies = [ + "cfg-if", + "winapi", +] + +[[package]] +name = "libm" +version = "0.2.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4ec2a862134d2a7d32d7983ddcdd1c4923530833c9f2ea1a44fc5fa473989058" + +[[package]] +name = "libmimalloc-sys" +version = "0.1.39" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "23aa6811d3bd4deb8a84dde645f943476d13b248d818edcf8ce0b2f37f036b44" +dependencies = [ + "cc", + "libc", +] + +[[package]] +name = "linux-raw-sys" +version = "0.4.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "78b3ae25bc7c8c38cec158d1f2757ee79e9b3740fbc7ccf0e59e4b08d793fa89" + +[[package]] +name = "lock_api" +version = "0.4.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "07af8b9cdd281b7915f413fa73f29ebd5d55d0d3f0155584dade1ff18cea1b17" +dependencies = [ + "autocfg", + "scopeguard", +] + +[[package]] +name = "log" +version = "0.4.22" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a7a70ba024b9dc04c27ea2f0c0548feb474ec5c54bba33a7f72f873a39d07b24" +dependencies = [ + "serde", +] + +[[package]] +name = "log-mdc" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a94d21414c1f4a51209ad204c1776a3d0765002c76c6abcb602a6f09f1e881c7" + +[[package]] +name = "log4rs" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0816135ae15bd0391cf284eab37e6e3ee0a6ee63d2ceeb659862bd8d0a984ca6" +dependencies = [ + "anyhow", + "arc-swap", + "chrono", + "derivative", + "fnv", + "humantime", + "libc", + "log", + "log-mdc", + "once_cell", + "parking_lot", + "rand", + "serde", + "serde-value", + "serde_json", + "serde_yaml", + "thiserror", + "thread-id", + "typemap-ors", + "winapi", +] + +[[package]] +name = "lz4" +version = "1.25.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d6eab492fe7f8651add23237ea56dbf11b3c4ff762ab83d40a47f11433421f91" +dependencies = [ + "libc", + "lz4-sys", +] + +[[package]] +name = "lz4-sys" +version = "1.9.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e9764018d143cc854c9f17f0b907de70f14393b1f502da6375dce70f00514eb3" +dependencies = [ + "cc", + "libc", +] + +[[package]] +name = "lz4_flex" +version = "0.11.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "75761162ae2b0e580d7e7c390558127e5f01b4194debd6221fd8c207fc80e3f5" +dependencies = [ + "twox-hash", +] + +[[package]] +name = "md-5" +version = "0.10.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d89e7ee0cfbedfc4da3340218492196241d89eefb6dab27de5df917a6d2e78cf" +dependencies = [ + "cfg-if", + "digest", +] + +[[package]] +name = "memchr" +version = "2.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "78ca9ab1a0babb1e7d5695e3530886289c18cf2f87ec19a575a0abdce112e3a3" + +[[package]] +name = "memmap2" +version = "0.9.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fe751422e4a8caa417e13c3ea66452215d7d63e19e604f4980461212f3ae1322" +dependencies = [ + "libc", +] + +[[package]] +name = "mimalloc" +version = "0.1.43" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68914350ae34959d83f732418d51e2427a794055d0b9529f48259ac07af65633" +dependencies = [ + "libmimalloc-sys", +] + +[[package]] +name = "miniz_oxide" +version = "0.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b8a240ddb74feaf34a79a7add65a741f3167852fba007066dcac1ca548d89c08" +dependencies = [ + "adler", +] + +[[package]] +name = "multimap" +version = "0.8.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e5ce46fe64a9d73be07dcbe690a38ce1b293be448fd8ce1e6c1b8062c9f72c6a" + +[[package]] +name = "nix" +version = "0.26.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "598beaf3cc6fdd9a5dfb1630c2800c7acd31df7aaf0f565796fba2b53ca1af1b" +dependencies = [ + "bitflags 1.3.2", + "cfg-if", + "libc", +] + +[[package]] +name = "num" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "35bd024e8b2ff75562e5f34e7f4905839deb4b22955ef5e73d2fea1b9813cb23" +dependencies = [ + "num-bigint", + "num-complex", + "num-integer", + "num-iter", + "num-rational", + "num-traits", +] + +[[package]] +name = "num-bigint" +version = "0.4.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a5e44f723f1133c9deac646763579fdb3ac745e418f2a7af9cd0c431da1f20b9" +dependencies = [ + "num-integer", + "num-traits", +] + +[[package]] +name = "num-complex" +version = "0.4.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "73f88a1307638156682bada9d7604135552957b7818057dcef22705b4d509495" +dependencies = [ + "num-traits", +] + +[[package]] +name = "num-format" +version = "0.4.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a652d9771a63711fd3c3deb670acfbe5c30a4072e664d7a3bf5a9e1056ac72c3" +dependencies = [ + "arrayvec", + "itoa", +] + +[[package]] +name = "num-integer" +version = "0.1.46" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7969661fd2958a5cb096e56c8e1ad0444ac2bbcd0061bd28660485a44879858f" +dependencies = [ + "num-traits", +] + +[[package]] +name = "num-iter" +version = "0.1.45" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1429034a0490724d0075ebb2bc9e875d6503c3cf69e235a8941aa757d83ef5bf" +dependencies = [ + "autocfg", + "num-integer", + "num-traits", +] + +[[package]] +name = "num-rational" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f83d14da390562dca69fc84082e73e548e1ad308d24accdedd2720017cb37824" +dependencies = [ + "num-bigint", + "num-integer", + "num-traits", +] + +[[package]] +name = "num-traits" +version = "0.2.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "071dfc062690e90b734c0b2273ce72ad0ffa95f0c74596bc250dcfd960262841" +dependencies = [ + "autocfg", + "libm", +] + +[[package]] +name = "num_cpus" +version = "1.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4161fcb6d602d4d2081af7c3a45852d875a03dd337a6bfdd6e06407b61342a43" +dependencies = [ + "hermit-abi", + "libc", +] + +[[package]] +name = "object" +version = "0.36.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "081b846d1d56ddfc18fdf1a922e4f6e07a11768ea1b92dec44e42b72712ccfce" +dependencies = [ + "memchr", +] + +[[package]] +name = "object_store" +version = "0.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fbebfd32c213ba1907fa7a9c9138015a8de2b43e30c5aa45b18f7deb46786ad6" +dependencies = [ + "async-trait", + "bytes", + "chrono", + "futures", + "humantime", + "itertools 0.12.1", + "parking_lot", + "percent-encoding", + "snafu", + "tokio", + "tracing", + "url", + "walkdir", +] + +[[package]] +name = "once_cell" +version = "1.19.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3fdb12b2476b595f9358c5161aa467c2438859caa136dec86c26fdd2efe17b92" + +[[package]] +name = "oorandom" +version = "11.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0ab1bc2a289d34bd04a330323ac98a1b4bc82c9d9fcb1e66b63caa84da26b575" + +[[package]] +name = "ordered-float" +version = "1.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3305af35278dd29f46fcdd139e0b1fbfae2153f0e5928b39b035542dd31e37b7" +dependencies = [ + "num-traits", +] + +[[package]] +name = "ordered-float" +version = "2.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68f19d67e5a2795c94e73e0bb1cc1a7edeb2e28efd39e2e1c9b7a40c1108b11c" +dependencies = [ + "num-traits", +] + +[[package]] +name = "parking_lot" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f1bf18183cf54e8d6059647fc3063646a1801cf30896933ec2311622cc4b9a27" +dependencies = [ + "lock_api", + "parking_lot_core", +] + +[[package]] +name = "parking_lot_core" +version = "0.9.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e401f977ab385c9e4e3ab30627d6f26d00e2c73eef317493c4ec6d468726cf8" +dependencies = [ + "cfg-if", + "libc", + "redox_syscall", + "smallvec", + "windows-targets 0.52.6", +] + +[[package]] +name = "parquet" +version = "52.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +dependencies = [ + "ahash", + "bytes", + "chrono", + "half", + "hashbrown", + "num", + "num-bigint", + "paste", + "seq-macro", + "thrift 0.17.0", + "twox-hash", +] + +[[package]] +name = "parquet-format" +version = "4.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1f0c06cdcd5460967c485f9c40a821746f5955ad81990533c7fae95dbd9bc0b5" +dependencies = [ + "thrift 0.13.0", +] + +[[package]] +name = "parse-zoneinfo" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1f2a05b18d44e2957b88f96ba460715e295bc1d7510468a2f3d3b44535d26c24" +dependencies = [ + "regex", +] + +[[package]] +name = "paste" +version = "1.0.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "57c0d7b74b563b49d38dae00a0c37d4d6de9b432382b2892f0574ddcae73fd0a" + +[[package]] +name = "percent-encoding" +version = "2.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e3148f5046208a5d56bcfc03053e3ca6334e51da8dfb19b6cdc8b306fae3283e" + +[[package]] +name = "petgraph" +version = "0.6.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b4c5cc86750666a3ed20bdaf5ca2a0344f9c67674cae0515bec2da16fbaa47db" +dependencies = [ + "fixedbitset", + "indexmap", +] + +[[package]] +name = "phf" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ade2d8b8f33c7333b51bcf0428d37e217e9f32192ae4772156f65063b8ce03dc" +dependencies = [ + "phf_shared", +] + +[[package]] +name = "phf_codegen" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e8d39688d359e6b34654d328e262234662d16cc0f60ec8dcbe5e718709342a5a" +dependencies = [ + "phf_generator", + "phf_shared", +] + +[[package]] +name = "phf_generator" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "48e4cc64c2ad9ebe670cb8fd69dd50ae301650392e81c05f9bfcb2d5bdbc24b0" +dependencies = [ + "phf_shared", + "rand", +] + +[[package]] +name = "phf_shared" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "90fcb95eef784c2ac79119d1dd819e162b5da872ce6f3c3abe1e8ca1c082f72b" +dependencies = [ + "siphasher", +] + +[[package]] +name = "pin-project-lite" +version = "0.2.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bda66fc9667c18cb2758a2ac84d1167245054bcf85d5d1aaa6923f45801bdd02" + +[[package]] +name = "pin-utils" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" + +[[package]] +name = "pkg-config" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d231b230927b5e4ad203db57bbcbee2802f6bce620b1e4a9024a07d94e2907ec" + +[[package]] +name = "plotters" +version = "0.3.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a15b6eccb8484002195a3e44fe65a4ce8e93a625797a063735536fd59cb01cf3" +dependencies = [ + "num-traits", + "plotters-backend", + "plotters-svg", + "wasm-bindgen", + "web-sys", +] + +[[package]] +name = "plotters-backend" +version = "0.3.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "414cec62c6634ae900ea1c56128dfe87cf63e7caece0852ec76aba307cebadb7" + +[[package]] +name = "plotters-svg" +version = "0.3.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "81b30686a7d9c3e010b84284bdd26a29f2138574f52f5eb6f794fc0ad924e705" +dependencies = [ + "plotters-backend", +] + +[[package]] +name = "pprof" +version = "0.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ef5c97c51bd34c7e742402e216abdeb44d415fbe6ae41d56b114723e953711cb" +dependencies = [ + "backtrace", + "cfg-if", + "findshlibs", + "inferno", + "libc", + "log", + "nix", + "once_cell", + "parking_lot", + "smallvec", + "symbolic-demangle", + "tempfile", + "thiserror", +] + +[[package]] +name = "ppv-lite86" +version = "0.2.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de" + +[[package]] +name = "proc-macro2" +version = "1.0.86" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5e719e8df665df0d1c8fbfd238015744736151d4445ec0836b8e628aae103b77" +dependencies = [ + "unicode-ident", +] + +[[package]] +name = "prost" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "444879275cb4fd84958b1a1d5420d15e6fcf7c235fe47f053c9c2a80aceb6001" +dependencies = [ + "bytes", + "prost-derive 0.9.0", +] + +[[package]] +name = "prost" +version = "0.12.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "deb1435c188b76130da55f17a466d252ff7b1418b2ad3e037d127b94e3411f29" +dependencies = [ + "bytes", + "prost-derive 0.12.6", +] + +[[package]] +name = "prost-build" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "62941722fb675d463659e49c4f3fe1fe792ff24fe5bbaa9c08cd3b98a1c354f5" +dependencies = [ + "bytes", + "heck 0.3.3", + "itertools 0.10.5", + "lazy_static", + "log", + "multimap", + "petgraph", + "prost 0.9.0", + "prost-types", + "regex", + "tempfile", + "which", +] + +[[package]] +name = "prost-derive" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f9cc1a3263e07e0bf68e96268f37665207b49560d98739662cdfaae215c720fe" +dependencies = [ + "anyhow", + "itertools 0.10.5", + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "prost-derive" +version = "0.12.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "81bddcdb20abf9501610992b6759a4c888aef7d1a7247ef75e2404275ac24af1" +dependencies = [ + "anyhow", + "itertools 0.12.1", + "proc-macro2", + "quote", + "syn 2.0.68", +] + +[[package]] +name = "prost-types" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "534b7a0e836e3c482d2693070f982e39e7611da9695d4d1f5a4b186b51faef0a" +dependencies = [ + "bytes", + "prost 0.9.0", +] + +[[package]] +name = "quick-xml" +version = "0.26.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f50b1c63b38611e7d4d7f68b82d3ad0cc71a2ad2e7f61fc10f1328d917c93cd" +dependencies = [ + "memchr", +] + +[[package]] +name = "quote" +version = "1.0.36" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0fa76aaf39101c457836aec0ce2316dbdc3ab723cdda1c6bd4e6ad4208acaca7" +dependencies = [ + "proc-macro2", +] + +[[package]] +name = "rand" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "34af8d1a0e25924bc5b7c43c079c942339d8f0a8b57c39049bef581b46327404" +dependencies = [ + "libc", + "rand_chacha", + "rand_core", +] + +[[package]] +name = "rand_chacha" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6c10a63a0fa32252be49d21e7709d4d4baf8d231c2dbce1eaa8141b9b127d88" +dependencies = [ + "ppv-lite86", + "rand_core", +] + +[[package]] +name = "rand_core" +version = "0.6.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ec0be4795e2f6a28069bec0b5ff3e2ac9bafc99e6a9a7dc3547996c5c816922c" +dependencies = [ + "getrandom", +] + +[[package]] +name = "rayon" +version = "1.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b418a60154510ca1a002a752ca9714984e21e4241e804d32555251faf8b78ffa" +dependencies = [ + "either", + "rayon-core", +] + +[[package]] +name = "rayon-core" +version = "1.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1465873a3dfdaa8ae7cb14b4383657caab0b3e8a0aa9ae8e04b044854c8dfce2" +dependencies = [ + "crossbeam-deque", + "crossbeam-utils", +] + +[[package]] +name = "redox_syscall" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c82cf8cff14456045f55ec4241383baeff27af886adb72ffb2162f99911de0fd" +dependencies = [ + "bitflags 2.6.0", +] + +[[package]] +name = "regex" +version = "1.10.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b91213439dad192326a0d7c6ee3955910425f441d7038e0d6933b0aec5c4517f" +dependencies = [ + "aho-corasick", + "memchr", + "regex-automata", + "regex-syntax", +] + +[[package]] +name = "regex-automata" +version = "0.4.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "38caf58cc5ef2fed281f89292ef23f6365465ed9a41b7a7754eb4e26496c92df" +dependencies = [ + "aho-corasick", + "memchr", + "regex-syntax", +] + +[[package]] +name = "regex-syntax" +version = "0.8.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7a66a03ae7c801facd77a29370b4faec201768915ac14a721ba36f20bc9c209b" + +[[package]] +name = "rgb" +version = "0.8.40" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a7439be6844e40133eda024efd85bf07f59d0dd2f59b10c00dd6cfb92cc5c741" +dependencies = [ + "bytemuck", +] + +[[package]] +name = "rustc-demangle" +version = "0.1.24" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "719b953e2095829ee67db738b3bfa9fa368c94900df327b3f07fe6e794d2fe1f" + +[[package]] +name = "rustc_version" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bfa0f585226d2e68097d4f95d113b15b83a82e819ab25717ec0590d9584ef366" +dependencies = [ + "semver", +] + +[[package]] +name = "rustix" +version = "0.38.34" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "70dc5ec042f7a43c4a73241207cecc9873a06d45debb38b329f8541d85c2730f" +dependencies = [ + "bitflags 2.6.0", + "errno", + "libc", + "linux-raw-sys", + "windows-sys 0.52.0", +] + +[[package]] +name = "rustversion" +version = "1.0.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "955d28af4278de8121b7ebeb796b6a45735dc01436d898801014aced2773a3d6" + +[[package]] +name = "ryu" +version = "1.0.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f3cb5ba0dc43242ce17de99c180e96db90b235b8a9fdc9543c96d2209116bd9f" + +[[package]] +name = "same-file" +version = "1.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "93fc1dc3aaa9bfed95e02e6eadabb4baf7e3078b0bd1b4d7b6b0b68378900502" +dependencies = [ + "winapi-util", +] + +[[package]] +name = "scopeguard" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" + +[[package]] +name = "semver" +version = "1.0.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "61697e0a1c7e512e84a621326239844a24d8207b4669b41bc18b32ea5cbf988b" + +[[package]] +name = "seq-macro" +version = "0.3.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" + +[[package]] +name = "serde" +version = "1.0.203" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7253ab4de971e72fb7be983802300c30b5a7f0c2e56fab8abfc6a214307c0094" +dependencies = [ + "serde_derive", +] + +[[package]] +name = "serde-value" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f3a1a3341211875ef120e117ea7fd5228530ae7e7036a779fdc9117be6b3282c" +dependencies = [ + "ordered-float 2.10.1", + "serde", +] + +[[package]] +name = "serde_derive" +version = "1.0.203" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "500cbc0ebeb6f46627f50f3f5811ccf6bf00643be300b4c3eabc0ef55dc5b5ba" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.68", +] + +[[package]] +name = "serde_json" +version = "1.0.120" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4e0d21c9a8cae1235ad58a00c11cb40d4b1e5c784f1ef2c537876ed6ffd8b7c5" +dependencies = [ + "itoa", + "ryu", + "serde", +] + +[[package]] +name = "serde_yaml" +version = "0.9.34+deprecated" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6a8b1a1a2ebf674015cc02edccce75287f1a0130d394307b36743c2f5d504b47" +dependencies = [ + "indexmap", + "itoa", + "ryu", + "serde", + "unsafe-libyaml", +] + +[[package]] +name = "sha2" +version = "0.10.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "793db75ad2bcafc3ffa7c68b215fee268f537982cd901d132f89c6343f3a3dc8" +dependencies = [ + "cfg-if", + "cpufeatures", + "digest", +] + +[[package]] +name = "simd-adler32" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d66dc143e6b11c1eddc06d5c423cfc97062865baf299914ab64caa38182078fe" + +[[package]] +name = "siphasher" +version = "0.3.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "38b58827f4464d87d377d175e90bf58eb00fd8716ff0a62f80356b5e61555d0d" + +[[package]] +name = "slab" +version = "0.4.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8f92a496fb766b417c996b9c5e57daf2f7ad3b0bebe1ccfca4856390e3d3bb67" +dependencies = [ + "autocfg", +] + +[[package]] +name = "smallvec" +version = "1.13.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3c5e1a9a646d36c3599cd173a41282daf47c44583ad367b8e6837255952e5c67" + +[[package]] +name = "snafu" +version = "0.7.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e4de37ad025c587a29e8f3f5605c00f70b98715ef90b9061a815b9e59e9042d6" +dependencies = [ + "doc-comment", + "snafu-derive", +] + +[[package]] +name = "snafu-derive" +version = "0.7.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "990079665f075b699031e9c08fd3ab99be5029b96f3b78dc0709e8f77e4efebf" +dependencies = [ + "heck 0.4.1", + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "snap" +version = "1.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1b6b67fb9a61334225b5b790716f609cd58395f895b3fe8b328786812a40bc3b" + +[[package]] +name = "sqlparser" +version = "0.47.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "295e9930cd7a97e58ca2a070541a3ca502b17f5d1fa7157376d0fabd85324f25" +dependencies = [ + "log", + "sqlparser_derive", +] + +[[package]] +name = "sqlparser_derive" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.68", +] + +[[package]] +name = "stable_deref_trait" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" + +[[package]] +name = "static_assertions" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a2eb9349b6444b326872e140eb1cf5e7c522154d69e7a0ffb0fb81c06b37543f" + +[[package]] +name = "str_stack" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9091b6114800a5f2141aee1d1b9d6ca3592ac062dc5decb3764ec5895a47b4eb" + +[[package]] +name = "strum" +version = "0.26.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8fec0f0aef304996cf250b31b5a10dee7980c85da9d759361292b8bca5a18f06" +dependencies = [ + "strum_macros", +] + +[[package]] +name = "strum_macros" +version = "0.26.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4c6bee85a5a24955dc440386795aa378cd9cf82acd5f764469152d2270e581be" +dependencies = [ + "heck 0.5.0", + "proc-macro2", + "quote", + "rustversion", + "syn 2.0.68", +] + +[[package]] +name = "subtle" +version = "2.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "13c2bddecc57b384dee18652358fb23172facb8a2c51ccc10d74c157bdea3292" + +[[package]] +name = "symbolic-common" +version = "12.9.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "71297dc3e250f7dbdf8adb99e235da783d690f5819fdeb4cce39d9cfb0aca9f1" +dependencies = [ + "debugid", + "memmap2", + "stable_deref_trait", + "uuid", +] + +[[package]] +name = "symbolic-demangle" +version = "12.9.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "424fa2c9bf2c862891b9cfd354a752751a6730fd838a4691e7f6c2c7957b9daf" +dependencies = [ + "cpp_demangle", + "rustc-demangle", + "symbolic-common", +] + +[[package]] +name = "syn" +version = "1.0.109" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72b64191b275b66ffe2469e8af2c1cfe3bafa67b529ead792a6d0160888b4237" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "syn" +version = "2.0.68" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "901fa70d88b9d6c98022e23b4136f9f3e54e4662c3bc1bd1d84a42a9a0f0c1e9" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "tempfile" +version = "3.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "85b77fafb263dd9d05cbeac119526425676db3784113aa9295c88498cbf8bff1" +dependencies = [ + "cfg-if", + "fastrand", + "rustix", + "windows-sys 0.52.0", +] + +[[package]] +name = "thiserror" +version = "1.0.61" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c546c80d6be4bc6a00c0f01730c08df82eaa7a7a61f11d656526506112cc1709" +dependencies = [ + "thiserror-impl", +] + +[[package]] +name = "thiserror-impl" +version = "1.0.61" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "46c3384250002a6d5af4d114f2845d37b57521033f30d5c3f46c4d70e1197533" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.68", +] + +[[package]] +name = "thread-id" +version = "4.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f0ec81c46e9eb50deaa257be2f148adf052d1fb7701cfd55ccfab2525280b70b" +dependencies = [ + "libc", + "winapi", +] + +[[package]] +name = "threadpool" +version = "1.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d050e60b33d41c19108b32cea32164033a9013fe3b46cbd4457559bfbf77afaa" +dependencies = [ + "num_cpus", +] + +[[package]] +name = "thrift" +version = "0.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c6d965454947cc7266d22716ebfd07b18d84ebaf35eec558586bbb2a8cb6b5b" +dependencies = [ + "byteorder", + "integer-encoding 1.1.7", + "log", + "ordered-float 1.1.1", + "threadpool", +] + +[[package]] +name = "thrift" +version = "0.17.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7e54bc85fc7faa8bc175c4bab5b92ba8d9a3ce893d0e9f42cc455c8ab16a9e09" +dependencies = [ + "byteorder", + "integer-encoding 3.0.4", + "log", + "ordered-float 2.10.1", + "threadpool", +] + +[[package]] +name = "tiny-keccak" +version = "2.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c9d3793400a45f954c52e73d068316d76b6f4e36977e3fcebb13a2721e80237" +dependencies = [ + "crunchy", +] + +[[package]] +name = "tinytemplate" +version = "1.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "be4d6b5f19ff7664e8c98d03e2139cb510db9b0a60b55f8e8709b689d939b6bc" +dependencies = [ + "serde", + "serde_json", +] + +[[package]] +name = "tinyvec" +version = "1.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ce6b6a2fb3a985e99cebfaefa9faa3024743da73304ca1c683a36429613d3d22" +dependencies = [ + "tinyvec_macros", +] + +[[package]] +name = "tinyvec_macros" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" + +[[package]] +name = "tokio" +version = "1.38.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ba4f4a02a7a80d6f274636f0aa95c7e383b912d41fe721a31f29e29698585a4a" +dependencies = [ + "backtrace", + "bytes", + "num_cpus", + "pin-project-lite", + "tokio-macros", +] + +[[package]] +name = "tokio-macros" +version = "2.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5f5ae998a069d4b5aba8ee9dad856af7d520c3699e6159b185c2acd48155d39a" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.68", +] + +[[package]] +name = "tokio-stream" +version = "0.1.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "267ac89e0bec6e691e5813911606935d77c476ff49024f98abcea3e7b15e37af" +dependencies = [ + "futures-core", + "pin-project-lite", + "tokio", +] + +[[package]] +name = "tracing" +version = "0.1.40" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c3523ab5a71916ccf420eebdf5521fcef02141234bbc0b8a49f2fdc4544364ef" +dependencies = [ + "pin-project-lite", + "tracing-attributes", + "tracing-core", +] + +[[package]] +name = "tracing-attributes" +version = "0.1.27" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.68", +] + +[[package]] +name = "tracing-core" +version = "0.1.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c06d3da6113f116aaee68e4d601191614c9053067f9ab7f6edbcb161237daa54" +dependencies = [ + "once_cell", +] + +[[package]] +name = "twox-hash" +version = "1.6.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "97fee6b57c6a41524a810daee9286c02d7752c4253064d0b05472833a438f675" +dependencies = [ + "cfg-if", + "rand", + "static_assertions", +] + +[[package]] +name = "typemap-ors" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a68c24b707f02dd18f1e4ccceb9d49f2058c2fb86384ef9972592904d7a28867" +dependencies = [ + "unsafe-any-ors", +] + +[[package]] +name = "typenum" +version = "1.17.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" + +[[package]] +name = "unicode-bidi" +version = "0.3.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "08f95100a766bf4f8f28f90d77e0a5461bbdb219042e7679bebe79004fed8d75" + +[[package]] +name = "unicode-ident" +version = "1.0.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3354b9ac3fae1ff6755cb6db53683adb661634f67557942dea4facebec0fee4b" + +[[package]] +name = "unicode-normalization" +version = "0.1.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a56d1686db2308d901306f92a263857ef59ea39678a5458e7cb17f01415101f5" +dependencies = [ + "tinyvec", +] + +[[package]] +name = "unicode-segmentation" +version = "1.11.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d4c87d22b6e3f4a18d4d40ef354e97c90fcb14dd91d7dc0aa9d8a1172ebf7202" + +[[package]] +name = "unicode-width" +version = "0.1.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0336d538f7abc86d282a4189614dfaa90810dfc2c6f6427eaf88e16311dd225d" + +[[package]] +name = "unsafe-any-ors" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e0a303d30665362d9680d7d91d78b23f5f899504d4f08b3c4cf08d055d87c0ad" +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.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "22784dbdf76fdde8af1aeda5622b546b422b6fc585325248a2bf9f5e41e94d6c" +dependencies = [ + "form_urlencoded", + "idna", + "percent-encoding", +] + +[[package]] +name = "uuid" +version = "1.9.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5de17fd2f7da591098415cff336e12965a28061ddace43b59cb3c430179c9439" +dependencies = [ + "getrandom", +] + +[[package]] +name = "version_check" +version = "0.9.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" + +[[package]] +name = "walkdir" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "29790946404f91d9c5d06f9874efddea1dc06c5efe94541a7d6863108e3a5e4b" +dependencies = [ + "same-file", + "winapi-util", +] + +[[package]] +name = "wasi" +version = "0.11.0+wasi-snapshot-preview1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" + +[[package]] +name = "wasm-bindgen" +version = "0.2.92" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4be2531df63900aeb2bca0daaaddec08491ee64ceecbee5076636a3b026795a8" +dependencies = [ + "cfg-if", + "wasm-bindgen-macro", +] + +[[package]] +name = "wasm-bindgen-backend" +version = "0.2.92" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "614d787b966d3989fa7bb98a654e369c762374fd3213d212cfc0251257e747da" +dependencies = [ + "bumpalo", + "log", + "once_cell", + "proc-macro2", + "quote", + "syn 2.0.68", + "wasm-bindgen-shared", +] + +[[package]] +name = "wasm-bindgen-macro" +version = "0.2.92" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a1f8823de937b71b9460c0c34e25f3da88250760bec0ebac694b49997550d726" +dependencies = [ + "quote", + "wasm-bindgen-macro-support", +] + +[[package]] +name = "wasm-bindgen-macro-support" +version = "0.2.92" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e94f17b526d0a461a191c78ea52bbce64071ed5c04c9ffe424dcb38f74171bb7" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.68", + "wasm-bindgen-backend", + "wasm-bindgen-shared", +] + +[[package]] +name = "wasm-bindgen-shared" +version = "0.2.92" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "af190c94f2773fdb3729c55b007a722abb5384da03bc0986df4c289bf5567e96" + +[[package]] +name = "web-sys" +version = "0.3.69" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77afa9a11836342370f4817622a2f0f418b134426d91a82dfb48f532d2ec13ef" +dependencies = [ + "js-sys", + "wasm-bindgen", +] + +[[package]] +name = "which" +version = "4.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87ba24419a2078cd2b0f2ede2691b6c66d8e47836da3b6db8265ebad47afbfc7" +dependencies = [ + "either", + "home", + "once_cell", + "rustix", +] + +[[package]] +name = "winapi" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c839a674fcd7a98952e593242ea400abe93992746761e38641405d28b00f419" +dependencies = [ + "winapi-i686-pc-windows-gnu", + "winapi-x86_64-pc-windows-gnu", +] + +[[package]] +name = "winapi-i686-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" + +[[package]] +name = "winapi-util" +version = "0.1.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4d4cc384e1e73b93bafa6fb4f1df8c41695c8a91cf9c4c64358067d15a7b6c6b" +dependencies = [ + "windows-sys 0.52.0", +] + +[[package]] +name = "winapi-x86_64-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" + +[[package]] +name = "windows-core" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "33ab640c8d7e35bf8ba19b884ba838ceb4fba93a4e8c65a9059d08afcfc683d9" +dependencies = [ + "windows-targets 0.52.6", +] + +[[package]] +name = "windows-sys" +version = "0.45.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "75283be5efb2831d37ea142365f009c02ec203cd29a3ebecbc093d52315b66d0" +dependencies = [ + "windows-targets 0.42.2", +] + +[[package]] +name = "windows-sys" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "282be5f36a8ce781fad8c8ae18fa3f9beff57ec1b52cb3de0789201425d9a33d" +dependencies = [ + "windows-targets 0.52.6", +] + +[[package]] +name = "windows-targets" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8e5180c00cd44c9b1c88adb3693291f1cd93605ded80c250a75d472756b4d071" +dependencies = [ + "windows_aarch64_gnullvm 0.42.2", + "windows_aarch64_msvc 0.42.2", + "windows_i686_gnu 0.42.2", + "windows_i686_msvc 0.42.2", + "windows_x86_64_gnu 0.42.2", + "windows_x86_64_gnullvm 0.42.2", + "windows_x86_64_msvc 0.42.2", +] + +[[package]] +name = "windows-targets" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9b724f72796e036ab90c1021d4780d4d3d648aca59e491e6b98e725b84e99973" +dependencies = [ + "windows_aarch64_gnullvm 0.52.6", + "windows_aarch64_msvc 0.52.6", + "windows_i686_gnu 0.52.6", + "windows_i686_gnullvm", + "windows_i686_msvc 0.52.6", + "windows_x86_64_gnu 0.52.6", + "windows_x86_64_gnullvm 0.52.6", + "windows_x86_64_msvc 0.52.6", +] + +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "597a5118570b68bc08d8d59125332c54f1ba9d9adeedeef5b99b02ba2b0698f8" + +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "32a4622180e7a0ec044bb555404c800bc9fd9ec262ec147edd5989ccd0c02cd3" + +[[package]] +name = "windows_aarch64_msvc" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e08e8864a60f06ef0d0ff4ba04124db8b0fb3be5776a5cd47641e942e58c4d43" + +[[package]] +name = "windows_aarch64_msvc" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09ec2a7bb152e2252b53fa7803150007879548bc709c039df7627cabbd05d469" + +[[package]] +name = "windows_i686_gnu" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c61d927d8da41da96a81f029489353e68739737d3beca43145c8afec9a31a84f" + +[[package]] +name = "windows_i686_gnu" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8e9b5ad5ab802e97eb8e295ac6720e509ee4c243f69d781394014ebfe8bbfa0b" + +[[package]] +name = "windows_i686_gnullvm" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0eee52d38c090b3caa76c563b86c3a4bd71ef1a819287c19d586d7334ae8ed66" + +[[package]] +name = "windows_i686_msvc" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "44d840b6ec649f480a41c8d80f9c65108b92d89345dd94027bfe06ac444d1060" + +[[package]] +name = "windows_i686_msvc" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "240948bc05c5e7c6dabba28bf89d89ffce3e303022809e73deaefe4f6ec56c66" + +[[package]] +name = "windows_x86_64_gnu" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8de912b8b8feb55c064867cf047dda097f92d51efad5b491dfb98f6bbb70cb36" + +[[package]] +name = "windows_x86_64_gnu" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "147a5c80aabfbf0c7d901cb5895d1de30ef2907eb21fbbab29ca94c5b08b1a78" + +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "26d41b46a36d453748aedef1486d5c7a85db22e56aff34643984ea85514e94a3" + +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "24d5b23dc417412679681396f2b49f3de8c1473deb516bd34410872eff51ed0d" + +[[package]] +name = "windows_x86_64_msvc" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9aec5da331524158c6d1a4ac0ab1541149c0b9505fde06423b02f5ef0106b9f0" + +[[package]] +name = "windows_x86_64_msvc" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" + +[[package]] +name = "zerocopy" +version = "0.7.35" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1b9b4fd18abc82b8136838da5d50bae7bdea537c574d8dc1a34ed098d6c166f0" +dependencies = [ + "zerocopy-derive", +] + +[[package]] +name = "zerocopy-derive" +version = "0.7.35" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.68", +] + +[[package]] +name = "zstd" +version = "0.11.2+zstd.1.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "20cc960326ece64f010d2d2107537f26dc589a6573a316bd5b1dba685fa5fde4" +dependencies = [ + "zstd-safe", +] + +[[package]] +name = "zstd-safe" +version = "5.0.2+zstd.1.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1d2a5585e04f9eea4b2a3d1eca508c4dee9592a89ef6f450c11719da0726f4db" +dependencies = [ + "libc", + "zstd-sys", +] + +[[package]] +name = "zstd-sys" +version = "2.0.12+zstd.1.5.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0a4e40c320c3cb459d9a9ff6de98cff88f4751ee9275d140e2be94a2b74e4c13" +dependencies = [ + "cc", + "pkg-config", +] diff --git a/native/Cargo.toml b/native/Cargo.toml new file mode 100644 index 000000000..79d393fdb --- /dev/null +++ b/native/Cargo.toml @@ -0,0 +1,32 @@ +# 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. + +[workspace] +members = ["core"] + +[workspace.package] +version = "0.1.0" +homepage = "https://datafusion.apache.org/comet" +repository = "https://github.com/apache/datafusion-comet" +authors = ["Apache DataFusion "] +description = "Apache DataFusion Comet: High performance accelerator for Apache Spark" +readme = "README.md" +license = "Apache-2.0" +edition = "2021" + +# Comet uses the same minimum Rust version as DataFusion +rust-version = "1.75" diff --git a/core/Cargo.lock b/native/core/Cargo.lock similarity index 100% rename from core/Cargo.lock rename to native/core/Cargo.lock diff --git a/core/Cargo.toml b/native/core/Cargo.toml similarity index 100% rename from core/Cargo.toml rename to native/core/Cargo.toml diff --git a/core/benches/bit_util.rs b/native/core/benches/bit_util.rs similarity index 100% rename from core/benches/bit_util.rs rename to native/core/benches/bit_util.rs diff --git a/core/benches/cast_from_string.rs b/native/core/benches/cast_from_string.rs similarity index 100% rename from core/benches/cast_from_string.rs rename to native/core/benches/cast_from_string.rs diff --git a/core/benches/cast_numeric.rs b/native/core/benches/cast_numeric.rs similarity index 100% rename from core/benches/cast_numeric.rs rename to native/core/benches/cast_numeric.rs diff --git a/core/benches/common.rs b/native/core/benches/common.rs similarity index 100% rename from core/benches/common.rs rename to native/core/benches/common.rs diff --git a/core/benches/hash.rs b/native/core/benches/hash.rs similarity index 100% rename from core/benches/hash.rs rename to native/core/benches/hash.rs diff --git a/core/benches/parquet_decode.rs b/native/core/benches/parquet_decode.rs similarity index 100% rename from core/benches/parquet_decode.rs rename to native/core/benches/parquet_decode.rs diff --git a/core/benches/parquet_read.rs b/native/core/benches/parquet_read.rs similarity index 100% rename from core/benches/parquet_read.rs rename to native/core/benches/parquet_read.rs diff --git a/core/benches/perf.rs b/native/core/benches/perf.rs similarity index 100% rename from core/benches/perf.rs rename to native/core/benches/perf.rs diff --git a/core/benches/row_columnar.rs b/native/core/benches/row_columnar.rs similarity index 100% rename from core/benches/row_columnar.rs rename to native/core/benches/row_columnar.rs diff --git a/core/benches/shuffle_writer.rs b/native/core/benches/shuffle_writer.rs similarity index 100% rename from core/benches/shuffle_writer.rs rename to native/core/benches/shuffle_writer.rs diff --git a/core/build.rs b/native/core/build.rs similarity index 100% rename from core/build.rs rename to native/core/build.rs diff --git a/core/rustfmt.toml b/native/core/rustfmt.toml similarity index 100% rename from core/rustfmt.toml rename to native/core/rustfmt.toml diff --git a/core/src/common/bit.rs b/native/core/src/common/bit.rs similarity index 100% rename from core/src/common/bit.rs rename to native/core/src/common/bit.rs diff --git a/core/src/common/buffer.rs b/native/core/src/common/buffer.rs similarity index 100% rename from core/src/common/buffer.rs rename to native/core/src/common/buffer.rs diff --git a/core/src/common/mod.rs b/native/core/src/common/mod.rs similarity index 100% rename from core/src/common/mod.rs rename to native/core/src/common/mod.rs diff --git a/core/src/data_type.rs b/native/core/src/data_type.rs similarity index 100% rename from core/src/data_type.rs rename to native/core/src/data_type.rs diff --git a/core/src/errors.rs b/native/core/src/errors.rs similarity index 99% rename from core/src/errors.rs rename to native/core/src/errors.rs index b6f4d0889..7b0b57448 100644 --- a/core/src/errors.rs +++ b/native/core/src/errors.rs @@ -551,7 +551,7 @@ mod tests { INIT.call_once(|| { // Add common classes to the classpath in so that we can find CometException let mut common_classes = PathBuf::from(env!("CARGO_MANIFEST_DIR")); - common_classes.push("../common/target/classes"); + common_classes.push("../../common/target/classes"); let mut class_path = common_classes .as_path() .to_str() diff --git a/core/src/execution/datafusion/expressions/abs.rs b/native/core/src/execution/datafusion/expressions/abs.rs similarity index 100% rename from core/src/execution/datafusion/expressions/abs.rs rename to native/core/src/execution/datafusion/expressions/abs.rs diff --git a/core/src/execution/datafusion/expressions/avg.rs b/native/core/src/execution/datafusion/expressions/avg.rs similarity index 100% rename from core/src/execution/datafusion/expressions/avg.rs rename to native/core/src/execution/datafusion/expressions/avg.rs diff --git a/core/src/execution/datafusion/expressions/avg_decimal.rs b/native/core/src/execution/datafusion/expressions/avg_decimal.rs similarity index 100% rename from core/src/execution/datafusion/expressions/avg_decimal.rs rename to native/core/src/execution/datafusion/expressions/avg_decimal.rs diff --git a/core/src/execution/datafusion/expressions/bitwise_not.rs b/native/core/src/execution/datafusion/expressions/bitwise_not.rs similarity index 100% rename from core/src/execution/datafusion/expressions/bitwise_not.rs rename to native/core/src/execution/datafusion/expressions/bitwise_not.rs diff --git a/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs b/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs similarity index 100% rename from core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs rename to native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs diff --git a/core/src/execution/datafusion/expressions/cast.rs b/native/core/src/execution/datafusion/expressions/cast.rs similarity index 100% rename from core/src/execution/datafusion/expressions/cast.rs rename to native/core/src/execution/datafusion/expressions/cast.rs diff --git a/core/src/execution/datafusion/expressions/checkoverflow.rs b/native/core/src/execution/datafusion/expressions/checkoverflow.rs similarity index 100% rename from core/src/execution/datafusion/expressions/checkoverflow.rs rename to native/core/src/execution/datafusion/expressions/checkoverflow.rs diff --git a/core/src/execution/datafusion/expressions/correlation.rs b/native/core/src/execution/datafusion/expressions/correlation.rs similarity index 100% rename from core/src/execution/datafusion/expressions/correlation.rs rename to native/core/src/execution/datafusion/expressions/correlation.rs diff --git a/core/src/execution/datafusion/expressions/covariance.rs b/native/core/src/execution/datafusion/expressions/covariance.rs similarity index 100% rename from core/src/execution/datafusion/expressions/covariance.rs rename to native/core/src/execution/datafusion/expressions/covariance.rs diff --git a/core/src/execution/datafusion/expressions/create_named_struct.rs b/native/core/src/execution/datafusion/expressions/create_named_struct.rs similarity index 100% rename from core/src/execution/datafusion/expressions/create_named_struct.rs rename to native/core/src/execution/datafusion/expressions/create_named_struct.rs diff --git a/core/src/execution/datafusion/expressions/if_expr.rs b/native/core/src/execution/datafusion/expressions/if_expr.rs similarity index 100% rename from core/src/execution/datafusion/expressions/if_expr.rs rename to native/core/src/execution/datafusion/expressions/if_expr.rs diff --git a/core/src/execution/datafusion/expressions/mod.rs b/native/core/src/execution/datafusion/expressions/mod.rs similarity index 100% rename from core/src/execution/datafusion/expressions/mod.rs rename to native/core/src/execution/datafusion/expressions/mod.rs diff --git a/core/src/execution/datafusion/expressions/negative.rs b/native/core/src/execution/datafusion/expressions/negative.rs similarity index 100% rename from core/src/execution/datafusion/expressions/negative.rs rename to native/core/src/execution/datafusion/expressions/negative.rs diff --git a/core/src/execution/datafusion/expressions/normalize_nan.rs b/native/core/src/execution/datafusion/expressions/normalize_nan.rs similarity index 100% rename from core/src/execution/datafusion/expressions/normalize_nan.rs rename to native/core/src/execution/datafusion/expressions/normalize_nan.rs diff --git a/core/src/execution/datafusion/expressions/scalar_funcs.rs b/native/core/src/execution/datafusion/expressions/scalar_funcs.rs similarity index 100% rename from core/src/execution/datafusion/expressions/scalar_funcs.rs rename to native/core/src/execution/datafusion/expressions/scalar_funcs.rs diff --git a/core/src/execution/datafusion/expressions/scalar_funcs/chr.rs b/native/core/src/execution/datafusion/expressions/scalar_funcs/chr.rs similarity index 100% rename from core/src/execution/datafusion/expressions/scalar_funcs/chr.rs rename to native/core/src/execution/datafusion/expressions/scalar_funcs/chr.rs diff --git a/core/src/execution/datafusion/expressions/scalar_funcs/hash_expressions.rs b/native/core/src/execution/datafusion/expressions/scalar_funcs/hash_expressions.rs similarity index 100% rename from core/src/execution/datafusion/expressions/scalar_funcs/hash_expressions.rs rename to native/core/src/execution/datafusion/expressions/scalar_funcs/hash_expressions.rs diff --git a/core/src/execution/datafusion/expressions/scalar_funcs/hex.rs b/native/core/src/execution/datafusion/expressions/scalar_funcs/hex.rs similarity index 100% rename from core/src/execution/datafusion/expressions/scalar_funcs/hex.rs rename to native/core/src/execution/datafusion/expressions/scalar_funcs/hex.rs diff --git a/core/src/execution/datafusion/expressions/scalar_funcs/unhex.rs b/native/core/src/execution/datafusion/expressions/scalar_funcs/unhex.rs similarity index 100% rename from core/src/execution/datafusion/expressions/scalar_funcs/unhex.rs rename to native/core/src/execution/datafusion/expressions/scalar_funcs/unhex.rs diff --git a/core/src/execution/datafusion/expressions/stats.rs b/native/core/src/execution/datafusion/expressions/stats.rs similarity index 100% rename from core/src/execution/datafusion/expressions/stats.rs rename to native/core/src/execution/datafusion/expressions/stats.rs diff --git a/core/src/execution/datafusion/expressions/stddev.rs b/native/core/src/execution/datafusion/expressions/stddev.rs similarity index 100% rename from core/src/execution/datafusion/expressions/stddev.rs rename to native/core/src/execution/datafusion/expressions/stddev.rs diff --git a/core/src/execution/datafusion/expressions/strings.rs b/native/core/src/execution/datafusion/expressions/strings.rs similarity index 100% rename from core/src/execution/datafusion/expressions/strings.rs rename to native/core/src/execution/datafusion/expressions/strings.rs diff --git a/core/src/execution/datafusion/expressions/subquery.rs b/native/core/src/execution/datafusion/expressions/subquery.rs similarity index 100% rename from core/src/execution/datafusion/expressions/subquery.rs rename to native/core/src/execution/datafusion/expressions/subquery.rs diff --git a/core/src/execution/datafusion/expressions/sum_decimal.rs b/native/core/src/execution/datafusion/expressions/sum_decimal.rs similarity index 100% rename from core/src/execution/datafusion/expressions/sum_decimal.rs rename to native/core/src/execution/datafusion/expressions/sum_decimal.rs diff --git a/core/src/execution/datafusion/expressions/temporal.rs b/native/core/src/execution/datafusion/expressions/temporal.rs similarity index 100% rename from core/src/execution/datafusion/expressions/temporal.rs rename to native/core/src/execution/datafusion/expressions/temporal.rs diff --git a/core/src/execution/datafusion/expressions/unbound.rs b/native/core/src/execution/datafusion/expressions/unbound.rs similarity index 100% rename from core/src/execution/datafusion/expressions/unbound.rs rename to native/core/src/execution/datafusion/expressions/unbound.rs diff --git a/core/src/execution/datafusion/expressions/utils.rs b/native/core/src/execution/datafusion/expressions/utils.rs similarity index 100% rename from core/src/execution/datafusion/expressions/utils.rs rename to native/core/src/execution/datafusion/expressions/utils.rs diff --git a/core/src/execution/datafusion/expressions/variance.rs b/native/core/src/execution/datafusion/expressions/variance.rs similarity index 100% rename from core/src/execution/datafusion/expressions/variance.rs rename to native/core/src/execution/datafusion/expressions/variance.rs diff --git a/core/src/execution/datafusion/expressions/xxhash64.rs b/native/core/src/execution/datafusion/expressions/xxhash64.rs similarity index 100% rename from core/src/execution/datafusion/expressions/xxhash64.rs rename to native/core/src/execution/datafusion/expressions/xxhash64.rs diff --git a/core/src/execution/datafusion/mod.rs b/native/core/src/execution/datafusion/mod.rs similarity index 100% rename from core/src/execution/datafusion/mod.rs rename to native/core/src/execution/datafusion/mod.rs diff --git a/core/src/execution/datafusion/operators/expand.rs b/native/core/src/execution/datafusion/operators/expand.rs similarity index 100% rename from core/src/execution/datafusion/operators/expand.rs rename to native/core/src/execution/datafusion/operators/expand.rs diff --git a/core/src/execution/datafusion/operators/mod.rs b/native/core/src/execution/datafusion/operators/mod.rs similarity index 100% rename from core/src/execution/datafusion/operators/mod.rs rename to native/core/src/execution/datafusion/operators/mod.rs diff --git a/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs similarity index 100% rename from core/src/execution/datafusion/planner.rs rename to native/core/src/execution/datafusion/planner.rs diff --git a/core/src/execution/datafusion/shuffle_writer.rs b/native/core/src/execution/datafusion/shuffle_writer.rs similarity index 100% rename from core/src/execution/datafusion/shuffle_writer.rs rename to native/core/src/execution/datafusion/shuffle_writer.rs diff --git a/core/src/execution/datafusion/spark_hash.rs b/native/core/src/execution/datafusion/spark_hash.rs similarity index 100% rename from core/src/execution/datafusion/spark_hash.rs rename to native/core/src/execution/datafusion/spark_hash.rs diff --git a/core/src/execution/datafusion/util/mod.rs b/native/core/src/execution/datafusion/util/mod.rs similarity index 100% rename from core/src/execution/datafusion/util/mod.rs rename to native/core/src/execution/datafusion/util/mod.rs diff --git a/core/src/execution/datafusion/util/spark_bit_array.rs b/native/core/src/execution/datafusion/util/spark_bit_array.rs similarity index 100% rename from core/src/execution/datafusion/util/spark_bit_array.rs rename to native/core/src/execution/datafusion/util/spark_bit_array.rs diff --git a/core/src/execution/datafusion/util/spark_bloom_filter.rs b/native/core/src/execution/datafusion/util/spark_bloom_filter.rs similarity index 100% rename from core/src/execution/datafusion/util/spark_bloom_filter.rs rename to native/core/src/execution/datafusion/util/spark_bloom_filter.rs diff --git a/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs similarity index 100% rename from core/src/execution/jni_api.rs rename to native/core/src/execution/jni_api.rs diff --git a/core/src/execution/kernels/hash.rs b/native/core/src/execution/kernels/hash.rs similarity index 100% rename from core/src/execution/kernels/hash.rs rename to native/core/src/execution/kernels/hash.rs diff --git a/core/src/execution/kernels/mod.rs b/native/core/src/execution/kernels/mod.rs similarity index 100% rename from core/src/execution/kernels/mod.rs rename to native/core/src/execution/kernels/mod.rs diff --git a/core/src/execution/kernels/strings.rs b/native/core/src/execution/kernels/strings.rs similarity index 100% rename from core/src/execution/kernels/strings.rs rename to native/core/src/execution/kernels/strings.rs diff --git a/core/src/execution/kernels/temporal.rs b/native/core/src/execution/kernels/temporal.rs similarity index 100% rename from core/src/execution/kernels/temporal.rs rename to native/core/src/execution/kernels/temporal.rs diff --git a/core/src/execution/memory_pool.rs b/native/core/src/execution/memory_pool.rs similarity index 100% rename from core/src/execution/memory_pool.rs rename to native/core/src/execution/memory_pool.rs diff --git a/core/src/execution/metrics/mod.rs b/native/core/src/execution/metrics/mod.rs similarity index 100% rename from core/src/execution/metrics/mod.rs rename to native/core/src/execution/metrics/mod.rs diff --git a/core/src/execution/metrics/utils.rs b/native/core/src/execution/metrics/utils.rs similarity index 100% rename from core/src/execution/metrics/utils.rs rename to native/core/src/execution/metrics/utils.rs diff --git a/core/src/execution/mod.rs b/native/core/src/execution/mod.rs similarity index 100% rename from core/src/execution/mod.rs rename to native/core/src/execution/mod.rs diff --git a/core/src/execution/operators/copy.rs b/native/core/src/execution/operators/copy.rs similarity index 100% rename from core/src/execution/operators/copy.rs rename to native/core/src/execution/operators/copy.rs diff --git a/core/src/execution/operators/mod.rs b/native/core/src/execution/operators/mod.rs similarity index 100% rename from core/src/execution/operators/mod.rs rename to native/core/src/execution/operators/mod.rs diff --git a/core/src/execution/operators/scan.rs b/native/core/src/execution/operators/scan.rs similarity index 100% rename from core/src/execution/operators/scan.rs rename to native/core/src/execution/operators/scan.rs diff --git a/core/src/execution/proto/expr.proto b/native/core/src/execution/proto/expr.proto similarity index 100% rename from core/src/execution/proto/expr.proto rename to native/core/src/execution/proto/expr.proto diff --git a/core/src/execution/proto/operator.proto b/native/core/src/execution/proto/operator.proto similarity index 100% rename from core/src/execution/proto/operator.proto rename to native/core/src/execution/proto/operator.proto diff --git a/core/src/execution/proto/partitioning.proto b/native/core/src/execution/proto/partitioning.proto similarity index 100% rename from core/src/execution/proto/partitioning.proto rename to native/core/src/execution/proto/partitioning.proto diff --git a/core/src/execution/serde.rs b/native/core/src/execution/serde.rs similarity index 100% rename from core/src/execution/serde.rs rename to native/core/src/execution/serde.rs diff --git a/core/src/execution/shuffle/list.rs b/native/core/src/execution/shuffle/list.rs similarity index 100% rename from core/src/execution/shuffle/list.rs rename to native/core/src/execution/shuffle/list.rs diff --git a/core/src/execution/shuffle/map.rs b/native/core/src/execution/shuffle/map.rs similarity index 100% rename from core/src/execution/shuffle/map.rs rename to native/core/src/execution/shuffle/map.rs diff --git a/core/src/execution/shuffle/mod.rs b/native/core/src/execution/shuffle/mod.rs similarity index 100% rename from core/src/execution/shuffle/mod.rs rename to native/core/src/execution/shuffle/mod.rs diff --git a/core/src/execution/shuffle/row.rs b/native/core/src/execution/shuffle/row.rs similarity index 100% rename from core/src/execution/shuffle/row.rs rename to native/core/src/execution/shuffle/row.rs diff --git a/core/src/execution/sort.rs b/native/core/src/execution/sort.rs similarity index 100% rename from core/src/execution/sort.rs rename to native/core/src/execution/sort.rs diff --git a/core/src/execution/timezone.rs b/native/core/src/execution/timezone.rs similarity index 100% rename from core/src/execution/timezone.rs rename to native/core/src/execution/timezone.rs diff --git a/core/src/execution/utils.rs b/native/core/src/execution/utils.rs similarity index 100% rename from core/src/execution/utils.rs rename to native/core/src/execution/utils.rs diff --git a/core/src/jvm_bridge/batch_iterator.rs b/native/core/src/jvm_bridge/batch_iterator.rs similarity index 100% rename from core/src/jvm_bridge/batch_iterator.rs rename to native/core/src/jvm_bridge/batch_iterator.rs diff --git a/core/src/jvm_bridge/comet_exec.rs b/native/core/src/jvm_bridge/comet_exec.rs similarity index 100% rename from core/src/jvm_bridge/comet_exec.rs rename to native/core/src/jvm_bridge/comet_exec.rs diff --git a/core/src/jvm_bridge/comet_metric_node.rs b/native/core/src/jvm_bridge/comet_metric_node.rs similarity index 100% rename from core/src/jvm_bridge/comet_metric_node.rs rename to native/core/src/jvm_bridge/comet_metric_node.rs diff --git a/core/src/jvm_bridge/comet_task_memory_manager.rs b/native/core/src/jvm_bridge/comet_task_memory_manager.rs similarity index 100% rename from core/src/jvm_bridge/comet_task_memory_manager.rs rename to native/core/src/jvm_bridge/comet_task_memory_manager.rs diff --git a/core/src/jvm_bridge/mod.rs b/native/core/src/jvm_bridge/mod.rs similarity index 100% rename from core/src/jvm_bridge/mod.rs rename to native/core/src/jvm_bridge/mod.rs diff --git a/core/src/lib.rs b/native/core/src/lib.rs similarity index 100% rename from core/src/lib.rs rename to native/core/src/lib.rs diff --git a/core/src/parquet/compression.rs b/native/core/src/parquet/compression.rs similarity index 100% rename from core/src/parquet/compression.rs rename to native/core/src/parquet/compression.rs diff --git a/core/src/parquet/data_type.rs b/native/core/src/parquet/data_type.rs similarity index 100% rename from core/src/parquet/data_type.rs rename to native/core/src/parquet/data_type.rs diff --git a/core/src/parquet/mod.rs b/native/core/src/parquet/mod.rs similarity index 100% rename from core/src/parquet/mod.rs rename to native/core/src/parquet/mod.rs diff --git a/core/src/parquet/mutable_vector.rs b/native/core/src/parquet/mutable_vector.rs similarity index 100% rename from core/src/parquet/mutable_vector.rs rename to native/core/src/parquet/mutable_vector.rs diff --git a/core/src/parquet/read/column.rs b/native/core/src/parquet/read/column.rs similarity index 100% rename from core/src/parquet/read/column.rs rename to native/core/src/parquet/read/column.rs diff --git a/core/src/parquet/read/levels.rs b/native/core/src/parquet/read/levels.rs similarity index 100% rename from core/src/parquet/read/levels.rs rename to native/core/src/parquet/read/levels.rs diff --git a/core/src/parquet/read/mod.rs b/native/core/src/parquet/read/mod.rs similarity index 100% rename from core/src/parquet/read/mod.rs rename to native/core/src/parquet/read/mod.rs diff --git a/core/src/parquet/read/values.rs b/native/core/src/parquet/read/values.rs similarity index 100% rename from core/src/parquet/read/values.rs rename to native/core/src/parquet/read/values.rs diff --git a/core/src/parquet/util/bit_packing.rs b/native/core/src/parquet/util/bit_packing.rs similarity index 100% rename from core/src/parquet/util/bit_packing.rs rename to native/core/src/parquet/util/bit_packing.rs diff --git a/core/src/parquet/util/buffer.rs b/native/core/src/parquet/util/buffer.rs similarity index 100% rename from core/src/parquet/util/buffer.rs rename to native/core/src/parquet/util/buffer.rs diff --git a/core/src/parquet/util/jni.rs b/native/core/src/parquet/util/jni.rs similarity index 100% rename from core/src/parquet/util/jni.rs rename to native/core/src/parquet/util/jni.rs diff --git a/core/src/parquet/util/jni_buffer.rs b/native/core/src/parquet/util/jni_buffer.rs similarity index 100% rename from core/src/parquet/util/jni_buffer.rs rename to native/core/src/parquet/util/jni_buffer.rs diff --git a/core/src/parquet/util/memory.rs b/native/core/src/parquet/util/memory.rs similarity index 100% rename from core/src/parquet/util/memory.rs rename to native/core/src/parquet/util/memory.rs diff --git a/core/src/parquet/util/mod.rs b/native/core/src/parquet/util/mod.rs similarity index 100% rename from core/src/parquet/util/mod.rs rename to native/core/src/parquet/util/mod.rs diff --git a/core/src/parquet/util/test_common/file_util.rs b/native/core/src/parquet/util/test_common/file_util.rs similarity index 100% rename from core/src/parquet/util/test_common/file_util.rs rename to native/core/src/parquet/util/test_common/file_util.rs diff --git a/core/src/parquet/util/test_common/mod.rs b/native/core/src/parquet/util/test_common/mod.rs similarity index 100% rename from core/src/parquet/util/test_common/mod.rs rename to native/core/src/parquet/util/test_common/mod.rs diff --git a/core/src/parquet/util/test_common/page_util.rs b/native/core/src/parquet/util/test_common/page_util.rs similarity index 100% rename from core/src/parquet/util/test_common/page_util.rs rename to native/core/src/parquet/util/test_common/page_util.rs diff --git a/core/src/parquet/util/test_common/rand_gen.rs b/native/core/src/parquet/util/test_common/rand_gen.rs similarity index 100% rename from core/src/parquet/util/test_common/rand_gen.rs rename to native/core/src/parquet/util/test_common/rand_gen.rs diff --git a/core/testdata/backtrace.txt b/native/core/testdata/backtrace.txt similarity index 100% rename from core/testdata/backtrace.txt rename to native/core/testdata/backtrace.txt diff --git a/core/testdata/stacktrace.txt b/native/core/testdata/stacktrace.txt similarity index 100% rename from core/testdata/stacktrace.txt rename to native/core/testdata/stacktrace.txt diff --git a/pom.xml b/pom.xml index 043c11e89..fcea91cf7 100644 --- a/pom.xml +++ b/pom.xml @@ -59,7 +59,7 @@ under the License. 0.8.11 4.8.8 2.0.6 - ${project.basedir}/../core/target/debug + ${project.basedir}/../native/target/debug darwin x86_64 org.apache.comet.shaded @@ -454,7 +454,7 @@ under the License. release - ${project.basedir}/../core/target/release + ${project.basedir}/../native/target/release diff --git a/spark/pom.xml b/spark/pom.xml index 6b3b36aa5..49672e0bb 100644 --- a/spark/pom.xml +++ b/spark/pom.xml @@ -152,7 +152,7 @@ under the License. com.google.protobuf:protoc:${protobuf.version} - ../core/src/execution/proto + ../native/core/src/execution/proto From 8f4427aadf060c328efb5302aee81143b6e20f4c Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Tue, 9 Jul 2024 01:55:55 +0530 Subject: [PATCH 14/16] add cargo machete to remove udeps (#641) --- .github/actions/rust-test/action.yaml | 6 +++ native/Cargo.lock | 74 ++------------------------- native/core/Cargo.toml | 7 --- 3 files changed, 10 insertions(+), 77 deletions(-) diff --git a/.github/actions/rust-test/action.yaml b/.github/actions/rust-test/action.yaml index b543bcf00..3aeba0666 100644 --- a/.github/actions/rust-test/action.yaml +++ b/.github/actions/rust-test/action.yaml @@ -37,6 +37,12 @@ runs: cd native cargo check --benches + - name: Check unused dependencies + shell: bash + run: | + cd native + cargo install cargo-machete && cargo machete + - name: Cache Maven dependencies uses: actions/cache@v4 with: diff --git a/native/Cargo.lock b/native/Cargo.lock index ccb6433d9..6136e0339 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -843,11 +843,9 @@ dependencies = [ "arrow-buffer", "arrow-data", "arrow-schema", - "arrow-string", "assertables", "async-trait", "brotli", - "byteorder", "bytes", "chrono", "chrono-tz 0.8.6", @@ -862,7 +860,6 @@ dependencies = [ "flate2", "futures", "half", - "hashbrown", "hex", "itertools 0.11.0", "jni", @@ -873,9 +870,7 @@ dependencies = [ "mimalloc", "num", "once_cell", - "parking_lot", "parquet", - "parquet-format", "paste", "pprof", "prost 0.12.6", @@ -887,9 +882,7 @@ dependencies = [ "snap", "tempfile", "thiserror", - "thrift 0.17.0", "tokio", - "tokio-stream", "twox-hash", "unicode-segmentation", "zstd", @@ -1500,12 +1493,6 @@ dependencies = [ "web-sys", ] -[[package]] -name = "integer-encoding" -version = "1.1.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "48dc51180a9b377fd75814d0cc02199c20f8e99433d6762f650d39cdbbd3b56f" - [[package]] name = "integer-encoding" version = "3.0.4" @@ -1994,15 +1981,6 @@ version = "11.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0ab1bc2a289d34bd04a330323ac98a1b4bc82c9d9fcb1e66b63caa84da26b575" -[[package]] -name = "ordered-float" -version = "1.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3305af35278dd29f46fcdd139e0b1fbfae2153f0e5928b39b035542dd31e37b7" -dependencies = [ - "num-traits", -] - [[package]] name = "ordered-float" version = "2.10.1" @@ -2049,19 +2027,10 @@ dependencies = [ "num-bigint", "paste", "seq-macro", - "thrift 0.17.0", + "thrift", "twox-hash", ] -[[package]] -name = "parquet-format" -version = "4.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f0c06cdcd5460967c485f9c40a821746f5955ad81990533c7fae95dbd9bc0b5" -dependencies = [ - "thrift 0.13.0", -] - [[package]] name = "parse-zoneinfo" version = "0.3.1" @@ -2486,7 +2455,7 @@ version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f3a1a3341211875ef120e117ea7fd5228530ae7e7036a779fdc9117be6b3282c" dependencies = [ - "ordered-float 2.10.1", + "ordered-float", "serde", ] @@ -2745,28 +2714,6 @@ dependencies = [ "winapi", ] -[[package]] -name = "threadpool" -version = "1.8.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d050e60b33d41c19108b32cea32164033a9013fe3b46cbd4457559bfbf77afaa" -dependencies = [ - "num_cpus", -] - -[[package]] -name = "thrift" -version = "0.13.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c6d965454947cc7266d22716ebfd07b18d84ebaf35eec558586bbb2a8cb6b5b" -dependencies = [ - "byteorder", - "integer-encoding 1.1.7", - "log", - "ordered-float 1.1.1", - "threadpool", -] - [[package]] name = "thrift" version = "0.17.0" @@ -2774,10 +2721,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7e54bc85fc7faa8bc175c4bab5b92ba8d9a3ce893d0e9f42cc455c8ab16a9e09" dependencies = [ "byteorder", - "integer-encoding 3.0.4", - "log", - "ordered-float 2.10.1", - "threadpool", + "integer-encoding", + "ordered-float", ] [[package]] @@ -2838,17 +2783,6 @@ dependencies = [ "syn 2.0.68", ] -[[package]] -name = "tokio-stream" -version = "0.1.15" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "267ac89e0bec6e691e5813911606935d77c476ff49024f98abcea3e7b15e37af" -dependencies = [ - "futures-core", - "pin-project-lite", - "tokio", -] - [[package]] name = "tracing" version = "0.1.40" diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index c3e924a44..bd0a3d5e4 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -32,19 +32,16 @@ include = [ ] [dependencies] -parquet-format = "4.0.0" # This must be kept in sync with that from parquet crate arrow = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1", features = ["prettyprint", "ffi", "chrono-tz"] } arrow-array = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } arrow-buffer = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } arrow-data = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } arrow-schema = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } -arrow-string = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } parquet = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1", default-features = false, features = ["experimental"] } half = { version = "2.4.1", default-features = false } futures = "0.3.28" mimalloc = { version = "*", default-features = false, optional = true } tokio = { version = "1", features = ["rt-multi-thread"] } -tokio-stream = { version = "0.1.8" } async-trait = "0.1" log = "0.4" log4rs = "1.2.0" @@ -52,9 +49,7 @@ thiserror = "1" serde = { version = "1", features = ["derive"] } lazy_static = "1.4.0" prost = "0.12.1" -thrift = "0.17" jni = "0.21" -byteorder = "1.4.3" snap = "1.1" brotli = "3.3" flate2 = "1.0" @@ -63,8 +58,6 @@ zstd = "0.11" rand = "0.8" num = "0.4" bytes = "1.5.0" -hashbrown = { version = "0.14", features = ["raw"] } -parking_lot = "0.12" tempfile = "3.8.0" ahash = { version = "0.8", default-features = false } itertools = "0.11.0" From b924aeb5a323c9e9a5442236773086c76e132a73 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 8 Jul 2024 16:52:44 -0700 Subject: [PATCH 15/16] fix: Avoid creating huge duplicate of canonicalized plans for CometNativeExec (#639) * fix: Remove original plan parameter from CometNativeExec * Revert "fix: Remove original plan parameter from CometNativeExec" This reverts commit b272551d76d4bba6b30501b5660420368a874145. * More * Revert "Revert "fix: Remove original plan parameter from CometNativeExec"" This reverts commit 722dc07c75407edbf671d47f23769d4e543dbab5. * More * More * Fix * Fix diffs * Update --- dev/diffs/3.4.3.diff | 4 +- dev/diffs/3.5.1.diff | 4 +- dev/diffs/4.0.0-preview1.diff | 4 +- .../comet/CometSparkSessionExtensions.scala | 68 ++++++++-- .../comet/CometBroadcastExchangeExec.scala | 8 +- .../spark/sql/comet/CometCoalesceExec.scala | 2 + .../sql/comet/CometCollectLimitExec.scala | 4 + .../CometTakeOrderedAndProjectExec.scala | 3 +- .../spark/sql/comet/CometWindowExec.scala | 3 +- .../apache/spark/sql/comet/operators.scala | 46 +++++-- .../q1/simplified.txt | 22 ++-- .../q10/simplified.txt | 22 ++-- .../q11/simplified.txt | 60 ++++----- .../q12/simplified.txt | 18 +-- .../q13/simplified.txt | 30 ++--- .../q14a/simplified.txt | 56 ++++----- .../q14b/simplified.txt | 58 ++++----- .../q15/simplified.txt | 22 ++-- .../q16/simplified.txt | 8 +- .../q17/simplified.txt | 46 +++---- .../q18/simplified.txt | 40 +++--- .../q19/simplified.txt | 30 ++--- .../q2/simplified.txt | 14 +-- .../q20/simplified.txt | 18 +-- .../q21/simplified.txt | 24 ++-- .../q22/simplified.txt | 22 ++-- .../q23a/simplified.txt | 42 +++---- .../q23b/simplified.txt | 48 +++---- .../q24a/simplified.txt | 14 +-- .../q24b/simplified.txt | 14 +-- .../q25/simplified.txt | 46 +++---- .../q26/simplified.txt | 30 ++--- .../q27/simplified.txt | 30 ++--- .../q28/simplified.txt | 24 ++-- .../q29/simplified.txt | 52 ++++---- .../q3/simplified.txt | 16 +-- .../q30/simplified.txt | 28 ++--- .../q31/simplified.txt | 68 +++++----- .../q32/simplified.txt | 20 +-- .../q33/simplified.txt | 50 ++++---- .../q34/simplified.txt | 26 ++-- .../q35/simplified.txt | 22 ++-- .../q36/simplified.txt | 24 ++-- .../q37/simplified.txt | 22 ++-- .../q38/simplified.txt | 28 ++--- .../q39a/simplified.txt | 38 +++--- .../q39b/simplified.txt | 38 +++--- .../q4/simplified.txt | 84 ++++++------- .../q40/simplified.txt | 10 +- .../q41/simplified.txt | 6 +- .../q42/simplified.txt | 16 +-- .../q43/simplified.txt | 16 +-- .../q44/simplified.txt | 10 +- .../q45/simplified.txt | 28 ++--- .../q46/simplified.txt | 30 ++--- .../q47/simplified.txt | 22 ++-- .../q48/simplified.txt | 26 ++-- .../q49/simplified.txt | 42 +++---- .../q5/simplified.txt | 58 ++++----- .../q50/simplified.txt | 28 ++--- .../q51/simplified.txt | 18 +-- .../q52/simplified.txt | 16 +-- .../q53/simplified.txt | 22 ++-- .../q54/simplified.txt | 32 ++--- .../q55/simplified.txt | 16 +-- .../q56/simplified.txt | 50 ++++---- .../q57/simplified.txt | 22 ++-- .../q58/simplified.txt | 42 +++---- .../q59/simplified.txt | 12 +- .../q6/simplified.txt | 30 ++--- .../q60/simplified.txt | 50 ++++---- .../q61/simplified.txt | 54 ++++---- .../q62/simplified.txt | 28 ++--- .../q63/simplified.txt | 22 ++-- .../q64/simplified.txt | 32 ++--- .../q65/simplified.txt | 22 ++-- .../q66/simplified.txt | 42 +++---- .../q67/simplified.txt | 24 ++-- .../q68/simplified.txt | 30 ++--- .../q69/simplified.txt | 22 ++-- .../q7/simplified.txt | 30 ++--- .../q70/simplified.txt | 24 ++-- .../q71/simplified.txt | 34 ++--- .../q72/simplified.txt | 50 ++++---- .../q73/simplified.txt | 26 ++-- .../q74/simplified.txt | 60 ++++----- .../q75/simplified.txt | 58 ++++----- .../q76/simplified.txt | 30 ++--- .../q77/simplified.txt | 50 ++++---- .../q78/simplified.txt | 14 +-- .../q79/simplified.txt | 26 ++-- .../q8/simplified.txt | 26 ++-- .../q80/simplified.txt | 24 ++-- .../q81/simplified.txt | 28 ++--- .../q82/simplified.txt | 22 ++-- .../q83/simplified.txt | 44 +++---- .../q84/simplified.txt | 28 ++--- .../q85/simplified.txt | 42 +++---- .../q86/simplified.txt | 18 +-- .../q87/simplified.txt | 28 ++--- .../q88/simplified.txt | 104 +++++++-------- .../q89/simplified.txt | 24 ++-- .../q9/simplified.txt | 20 +-- .../q90/simplified.txt | 32 ++--- .../q91/simplified.txt | 42 +++---- .../q92/simplified.txt | 20 +-- .../q93/simplified.txt | 4 +- .../q94/simplified.txt | 8 +- .../q95/simplified.txt | 12 +- .../q96/simplified.txt | 20 +-- .../q97/simplified.txt | 10 +- .../q98/simplified.txt | 18 +-- .../q99/simplified.txt | 28 ++--- .../q1/simplified.txt | 18 +-- .../q10/simplified.txt | 22 ++-- .../q11/simplified.txt | 52 ++++---- .../q12/simplified.txt | 16 +-- .../q13/simplified.txt | 28 ++--- .../q14a/simplified.txt | 54 ++++---- .../q14b/simplified.txt | 56 ++++----- .../q15/simplified.txt | 20 +-- .../q16/simplified.txt | 8 +- .../q17/simplified.txt | 44 +++---- .../q18/simplified.txt | 38 +++--- .../q19/simplified.txt | 28 ++--- .../q2/simplified.txt | 12 +- .../q20/simplified.txt | 16 +-- .../q21/simplified.txt | 22 ++-- .../q22/simplified.txt | 20 +-- .../q23a/simplified.txt | 38 +++--- .../q23b/simplified.txt | 44 +++---- .../q24a/simplified.txt | 14 +-- .../q24b/simplified.txt | 14 +-- .../q25/simplified.txt | 44 +++---- .../q26/simplified.txt | 28 ++--- .../q27/simplified.txt | 28 ++--- .../q28/simplified.txt | 12 +- .../q29/simplified.txt | 50 ++++---- .../q3/simplified.txt | 14 +-- .../q30/simplified.txt | 24 ++-- .../q31/simplified.txt | 56 ++++----- .../q32/simplified.txt | 18 +-- .../q33/simplified.txt | 44 +++---- .../q34/simplified.txt | 26 ++-- .../q35/simplified.txt | 22 ++-- .../q36/simplified.txt | 22 ++-- .../q37/simplified.txt | 22 ++-- .../q38/simplified.txt | 28 ++--- .../q39a/simplified.txt | 34 ++--- .../q39b/simplified.txt | 34 ++--- .../q4/simplified.txt | 72 +++++------ .../q40/simplified.txt | 10 +- .../q41/simplified.txt | 6 +- .../q42/simplified.txt | 14 +-- .../q43/simplified.txt | 14 +-- .../q44/simplified.txt | 6 +- .../q45/simplified.txt | 28 ++--- .../q46/simplified.txt | 28 ++--- .../q47/simplified.txt | 20 +-- .../q48/simplified.txt | 24 ++-- .../q49/simplified.txt | 36 +++--- .../q5/simplified.txt | 52 ++++---- .../q50/simplified.txt | 26 ++-- .../q51/simplified.txt | 14 +-- .../q52/simplified.txt | 14 +-- .../q53/simplified.txt | 20 +-- .../q54/simplified.txt | 32 ++--- .../q55/simplified.txt | 14 +-- .../q56/simplified.txt | 44 +++---- .../q57/simplified.txt | 20 +-- .../q58/simplified.txt | 36 +++--- .../q59/simplified.txt | 10 +- .../q6/simplified.txt | 28 ++--- .../q60/simplified.txt | 44 +++---- .../q61/simplified.txt | 50 ++++---- .../q62/simplified.txt | 26 ++-- .../q63/simplified.txt | 20 +-- .../q64/simplified.txt | 32 ++--- .../q65/simplified.txt | 18 +-- .../q66/simplified.txt | 38 +++--- .../q67/simplified.txt | 22 ++-- .../q68/simplified.txt | 28 ++--- .../q69/simplified.txt | 22 ++-- .../q7/simplified.txt | 28 ++--- .../q70/simplified.txt | 22 ++-- .../q71/simplified.txt | 32 ++--- .../q72/simplified.txt | 50 ++++---- .../q73/simplified.txt | 26 ++-- .../q74/simplified.txt | 52 ++++---- .../q75/simplified.txt | 58 ++++----- .../q76/simplified.txt | 28 ++--- .../q77/simplified.txt | 38 +++--- .../q78/simplified.txt | 14 +-- .../q79/simplified.txt | 24 ++-- .../q8/simplified.txt | 26 ++-- .../q80/simplified.txt | 24 ++-- .../q81/simplified.txt | 24 ++-- .../q82/simplified.txt | 22 ++-- .../q83.ansi/simplified.txt | 38 +++--- .../q84/simplified.txt | 28 ++--- .../q85/simplified.txt | 40 +++--- .../q86/simplified.txt | 16 +-- .../q87/simplified.txt | 28 ++--- .../q88/simplified.txt | 104 +++++++-------- .../q89/simplified.txt | 22 ++-- .../q9/simplified.txt | 10 +- .../q90/simplified.txt | 32 ++--- .../q91/simplified.txt | 40 +++--- .../q92/simplified.txt | 18 +-- .../q93/simplified.txt | 4 +- .../q94/simplified.txt | 8 +- .../q95/simplified.txt | 12 +- .../q96/simplified.txt | 20 +-- .../q97/simplified.txt | 10 +- .../q98/simplified.txt | 16 +-- .../q99/simplified.txt | 26 ++-- .../approved-plans-v1_4/q1/simplified.txt | 22 ++-- .../approved-plans-v1_4/q10/simplified.txt | 22 ++-- .../approved-plans-v1_4/q11/simplified.txt | 60 ++++----- .../approved-plans-v1_4/q12/simplified.txt | 18 +-- .../approved-plans-v1_4/q13/simplified.txt | 30 ++--- .../approved-plans-v1_4/q14a/simplified.txt | 56 ++++----- .../approved-plans-v1_4/q14b/simplified.txt | 58 ++++----- .../approved-plans-v1_4/q15/simplified.txt | 22 ++-- .../approved-plans-v1_4/q16/simplified.txt | 8 +- .../approved-plans-v1_4/q17/simplified.txt | 46 +++---- .../approved-plans-v1_4/q18/simplified.txt | 40 +++--- .../approved-plans-v1_4/q19/simplified.txt | 30 ++--- .../approved-plans-v1_4/q2/simplified.txt | 14 +-- .../approved-plans-v1_4/q20/simplified.txt | 18 +-- .../approved-plans-v1_4/q21/simplified.txt | 24 ++-- .../approved-plans-v1_4/q22/simplified.txt | 22 ++-- .../approved-plans-v1_4/q23a/simplified.txt | 42 +++---- .../approved-plans-v1_4/q23b/simplified.txt | 48 +++---- .../approved-plans-v1_4/q24a/simplified.txt | 14 +-- .../approved-plans-v1_4/q24b/simplified.txt | 14 +-- .../approved-plans-v1_4/q25/simplified.txt | 46 +++---- .../approved-plans-v1_4/q26/simplified.txt | 30 ++--- .../approved-plans-v1_4/q27/simplified.txt | 30 ++--- .../approved-plans-v1_4/q28/simplified.txt | 24 ++-- .../approved-plans-v1_4/q29/simplified.txt | 52 ++++---- .../approved-plans-v1_4/q3/simplified.txt | 16 +-- .../approved-plans-v1_4/q30/simplified.txt | 28 ++--- .../approved-plans-v1_4/q31/simplified.txt | 68 +++++----- .../approved-plans-v1_4/q32/simplified.txt | 20 +-- .../approved-plans-v1_4/q33/simplified.txt | 50 ++++---- .../approved-plans-v1_4/q34/simplified.txt | 26 ++-- .../approved-plans-v1_4/q35/simplified.txt | 22 ++-- .../approved-plans-v1_4/q36/simplified.txt | 24 ++-- .../approved-plans-v1_4/q37/simplified.txt | 22 ++-- .../approved-plans-v1_4/q38/simplified.txt | 28 ++--- .../approved-plans-v1_4/q39a/simplified.txt | 38 +++--- .../approved-plans-v1_4/q39b/simplified.txt | 38 +++--- .../approved-plans-v1_4/q4/simplified.txt | 84 ++++++------- .../approved-plans-v1_4/q40/simplified.txt | 10 +- .../approved-plans-v1_4/q41/simplified.txt | 6 +- .../approved-plans-v1_4/q42/simplified.txt | 16 +-- .../approved-plans-v1_4/q43/simplified.txt | 16 +-- .../approved-plans-v1_4/q44/simplified.txt | 10 +- .../approved-plans-v1_4/q45/simplified.txt | 28 ++--- .../approved-plans-v1_4/q46/simplified.txt | 30 ++--- .../approved-plans-v1_4/q47/simplified.txt | 22 ++-- .../approved-plans-v1_4/q48/simplified.txt | 26 ++-- .../approved-plans-v1_4/q49/simplified.txt | 42 +++---- .../approved-plans-v1_4/q5/simplified.txt | 58 ++++----- .../approved-plans-v1_4/q50/simplified.txt | 28 ++--- .../approved-plans-v1_4/q51/simplified.txt | 18 +-- .../approved-plans-v1_4/q52/simplified.txt | 16 +-- .../approved-plans-v1_4/q53/simplified.txt | 22 ++-- .../approved-plans-v1_4/q54/simplified.txt | 32 ++--- .../approved-plans-v1_4/q55/simplified.txt | 16 +-- .../approved-plans-v1_4/q56/simplified.txt | 50 ++++---- .../approved-plans-v1_4/q57/simplified.txt | 22 ++-- .../approved-plans-v1_4/q58/simplified.txt | 42 +++---- .../approved-plans-v1_4/q59/simplified.txt | 12 +- .../approved-plans-v1_4/q6/simplified.txt | 30 ++--- .../approved-plans-v1_4/q60/simplified.txt | 50 ++++---- .../approved-plans-v1_4/q61/simplified.txt | 54 ++++---- .../approved-plans-v1_4/q62/simplified.txt | 28 ++--- .../approved-plans-v1_4/q63/simplified.txt | 22 ++-- .../approved-plans-v1_4/q64/simplified.txt | 32 ++--- .../approved-plans-v1_4/q65/simplified.txt | 22 ++-- .../approved-plans-v1_4/q66/simplified.txt | 42 +++---- .../approved-plans-v1_4/q67/simplified.txt | 24 ++-- .../approved-plans-v1_4/q68/simplified.txt | 30 ++--- .../approved-plans-v1_4/q69/simplified.txt | 22 ++-- .../approved-plans-v1_4/q7/simplified.txt | 30 ++--- .../approved-plans-v1_4/q70/simplified.txt | 24 ++-- .../approved-plans-v1_4/q71/simplified.txt | 34 ++--- .../approved-plans-v1_4/q72/simplified.txt | 50 ++++---- .../approved-plans-v1_4/q73/simplified.txt | 26 ++-- .../approved-plans-v1_4/q74/simplified.txt | 60 ++++----- .../approved-plans-v1_4/q75/simplified.txt | 58 ++++----- .../approved-plans-v1_4/q76/simplified.txt | 30 ++--- .../approved-plans-v1_4/q77/simplified.txt | 50 ++++---- .../approved-plans-v1_4/q78/simplified.txt | 14 +-- .../approved-plans-v1_4/q79/simplified.txt | 26 ++-- .../approved-plans-v1_4/q8/simplified.txt | 26 ++-- .../approved-plans-v1_4/q80/simplified.txt | 24 ++-- .../approved-plans-v1_4/q81/simplified.txt | 28 ++--- .../approved-plans-v1_4/q82/simplified.txt | 22 ++-- .../approved-plans-v1_4/q83/simplified.txt | 44 +++---- .../approved-plans-v1_4/q84/simplified.txt | 28 ++--- .../approved-plans-v1_4/q85/simplified.txt | 42 +++---- .../approved-plans-v1_4/q86/simplified.txt | 18 +-- .../approved-plans-v1_4/q87/simplified.txt | 28 ++--- .../approved-plans-v1_4/q88/simplified.txt | 104 +++++++-------- .../approved-plans-v1_4/q89/simplified.txt | 24 ++-- .../approved-plans-v1_4/q9/simplified.txt | 20 +-- .../approved-plans-v1_4/q90/simplified.txt | 32 ++--- .../approved-plans-v1_4/q91/simplified.txt | 42 +++---- .../approved-plans-v1_4/q92/simplified.txt | 20 +-- .../approved-plans-v1_4/q93/simplified.txt | 4 +- .../approved-plans-v1_4/q94/simplified.txt | 8 +- .../approved-plans-v1_4/q95/simplified.txt | 12 +- .../approved-plans-v1_4/q96/simplified.txt | 20 +-- .../approved-plans-v1_4/q97/simplified.txt | 10 +- .../approved-plans-v1_4/q98/simplified.txt | 18 +-- .../approved-plans-v1_4/q99/simplified.txt | 28 ++--- .../q10a/simplified.txt | 38 +++--- .../q11/simplified.txt | 60 ++++----- .../q12/simplified.txt | 18 +-- .../q14/simplified.txt | 58 ++++----- .../q14a/simplified.txt | 62 ++++----- .../q18a/simplified.txt | 118 +++++++++--------- .../q20/simplified.txt | 18 +-- .../q22/simplified.txt | 16 +-- .../q22a/simplified.txt | 22 ++-- .../q24/simplified.txt | 14 +-- .../q27a/simplified.txt | 60 ++++----- .../q34/simplified.txt | 26 ++-- .../q35/simplified.txt | 22 ++-- .../q35a/simplified.txt | 38 +++--- .../q36a/simplified.txt | 24 ++-- .../q47/simplified.txt | 22 ++-- .../q49/simplified.txt | 42 +++---- .../q51a/simplified.txt | 18 +-- .../q57/simplified.txt | 22 ++-- .../q5a/simplified.txt | 58 ++++----- .../q6/simplified.txt | 30 ++--- .../q64/simplified.txt | 32 ++--- .../q67a/simplified.txt | 24 ++-- .../q70a/simplified.txt | 24 ++-- .../q72/simplified.txt | 50 ++++---- .../q74/simplified.txt | 60 ++++----- .../q75/simplified.txt | 58 ++++----- .../q77a/simplified.txt | 50 ++++---- .../q78/simplified.txt | 14 +-- .../q80a/simplified.txt | 24 ++-- .../q86a/simplified.txt | 18 +-- .../q98/simplified.txt | 18 +-- .../q10a/simplified.txt | 38 +++--- .../q11/simplified.txt | 52 ++++---- .../q12/simplified.txt | 16 +-- .../q14/simplified.txt | 56 ++++----- .../q14a/simplified.txt | 60 ++++----- .../q18a/simplified.txt | 108 ++++++++-------- .../q20/simplified.txt | 16 +-- .../q22/simplified.txt | 16 +-- .../q22a/simplified.txt | 20 +-- .../q24/simplified.txt | 14 +-- .../q27a/simplified.txt | 54 ++++---- .../q34/simplified.txt | 26 ++-- .../q35/simplified.txt | 22 ++-- .../q35a/simplified.txt | 36 +++--- .../q36a/simplified.txt | 22 ++-- .../q47/simplified.txt | 20 +-- .../q49/simplified.txt | 36 +++--- .../q51a/simplified.txt | 14 +-- .../q57/simplified.txt | 20 +-- .../q5a/simplified.txt | 52 ++++---- .../q6/simplified.txt | 28 ++--- .../q64/simplified.txt | 32 ++--- .../q67a/simplified.txt | 22 ++-- .../q70a/simplified.txt | 22 ++-- .../q72/simplified.txt | 50 ++++---- .../q74/simplified.txt | 52 ++++---- .../q75/simplified.txt | 58 ++++----- .../q77a/simplified.txt | 38 +++--- .../q78/simplified.txt | 14 +-- .../q80a/simplified.txt | 24 ++-- .../q86a/simplified.txt | 16 +-- .../q98/simplified.txt | 16 +-- .../approved-plans-v2_7/q10a/simplified.txt | 38 +++--- .../approved-plans-v2_7/q11/simplified.txt | 60 ++++----- .../approved-plans-v2_7/q12/simplified.txt | 18 +-- .../approved-plans-v2_7/q14/simplified.txt | 58 ++++----- .../approved-plans-v2_7/q14a/simplified.txt | 62 ++++----- .../approved-plans-v2_7/q18a/simplified.txt | 118 +++++++++--------- .../approved-plans-v2_7/q20/simplified.txt | 18 +-- .../approved-plans-v2_7/q22/simplified.txt | 16 +-- .../approved-plans-v2_7/q22a/simplified.txt | 22 ++-- .../approved-plans-v2_7/q24/simplified.txt | 14 +-- .../approved-plans-v2_7/q27a/simplified.txt | 60 ++++----- .../approved-plans-v2_7/q34/simplified.txt | 26 ++-- .../approved-plans-v2_7/q35/simplified.txt | 22 ++-- .../approved-plans-v2_7/q35a/simplified.txt | 38 +++--- .../approved-plans-v2_7/q36a/simplified.txt | 24 ++-- .../approved-plans-v2_7/q47/simplified.txt | 22 ++-- .../approved-plans-v2_7/q49/simplified.txt | 42 +++---- .../approved-plans-v2_7/q51a/simplified.txt | 18 +-- .../approved-plans-v2_7/q57/simplified.txt | 22 ++-- .../approved-plans-v2_7/q5a/simplified.txt | 58 ++++----- .../approved-plans-v2_7/q6/simplified.txt | 30 ++--- .../approved-plans-v2_7/q64/simplified.txt | 32 ++--- .../approved-plans-v2_7/q67a/simplified.txt | 24 ++-- .../approved-plans-v2_7/q70a/simplified.txt | 24 ++-- .../approved-plans-v2_7/q72/simplified.txt | 50 ++++---- .../approved-plans-v2_7/q74/simplified.txt | 60 ++++----- .../approved-plans-v2_7/q75/simplified.txt | 58 ++++----- .../approved-plans-v2_7/q77a/simplified.txt | 50 ++++---- .../approved-plans-v2_7/q78/simplified.txt | 14 +-- .../approved-plans-v2_7/q80a/simplified.txt | 24 ++-- .../approved-plans-v2_7/q86a/simplified.txt | 18 +-- .../approved-plans-v2_7/q98/simplified.txt | 18 +-- 415 files changed, 6380 insertions(+), 6302 deletions(-) diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 8129c7455..3c88d6a7e 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -2491,8 +2491,8 @@ index dd55fcfe42c..293e9dc2986 100644 val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child -+ case CometFilterExec(_, _, _, child, _) => child -+ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, child, _), _) => child ++ case CometFilterExec(_, _, _, _, child, _) => child ++ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, _, child, _), _) => child } spark.internalCreateDataFrame(withoutFilters.execute(), schema) diff --git a/dev/diffs/3.5.1.diff b/dev/diffs/3.5.1.diff index ebc4cdb8c..cf6754416 100644 --- a/dev/diffs/3.5.1.diff +++ b/dev/diffs/3.5.1.diff @@ -2650,8 +2650,8 @@ index dd55fcfe42c..293e9dc2986 100644 val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child -+ case CometFilterExec(_, _, _, child, _) => child -+ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, child, _), _) => child ++ case CometFilterExec(_, _, _, _, child, _) => child ++ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, _, child, _), _) => child } spark.internalCreateDataFrame(withoutFilters.execute(), schema) diff --git a/dev/diffs/4.0.0-preview1.diff b/dev/diffs/4.0.0-preview1.diff index 4031015df..744722ecf 100644 --- a/dev/diffs/4.0.0-preview1.diff +++ b/dev/diffs/4.0.0-preview1.diff @@ -2651,8 +2651,8 @@ index 5fbf379644f..32711763ec1 100644 val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child -+ case CometFilterExec(_, _, _, child, _) => child -+ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, child, _), _) => child ++ case CometFilterExec(_, _, _, _, child, _) => child ++ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, _, child, _), _) => child } spark.internalCreateDataFrame(withoutFilters.execute(), schema) diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index fb1f62542..c30d6b0f7 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -331,8 +331,8 @@ class CometSparkSessionExtensions CometProjectExec( nativeOp, op, - op.projectList, op.output, + op.projectList, op.child, SerializedPlan(None)) case None => @@ -343,7 +343,13 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - CometFilterExec(nativeOp, op, op.condition, op.child, SerializedPlan(None)) + CometFilterExec( + nativeOp, + op, + op.output, + op.condition, + op.child, + SerializedPlan(None)) case None => op } @@ -352,7 +358,14 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - CometSortExec(nativeOp, op, op.sortOrder, op.child, SerializedPlan(None)) + CometSortExec( + nativeOp, + op, + op.output, + op.outputOrdering, + op.sortOrder, + op.child, + SerializedPlan(None)) case None => op } @@ -393,12 +406,27 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - CometExpandExec(nativeOp, op, op.projections, op.child, SerializedPlan(None)) + CometExpandExec( + nativeOp, + op, + op.output, + op.projections, + op.child, + SerializedPlan(None)) case None => op } - case op @ HashAggregateExec(_, _, _, groupingExprs, aggExprs, _, _, _, child) => + case op @ HashAggregateExec( + _, + _, + _, + groupingExprs, + aggExprs, + _, + _, + resultExpressions, + child) => val modes = aggExprs.map(_.mode).distinct if (!modes.isEmpty && modes.size != 1) { @@ -425,8 +453,10 @@ class CometSparkSessionExtensions CometHashAggregateExec( nativeOp, op, + op.output, groupingExprs, aggExprs, + resultExpressions, child.output, if (modes.nonEmpty) Some(modes.head) else None, child, @@ -446,6 +476,8 @@ class CometSparkSessionExtensions CometHashJoinExec( nativeOp, op, + op.output, + op.outputOrdering, op.leftKeys, op.rightKeys, op.joinType, @@ -478,6 +510,8 @@ class CometSparkSessionExtensions CometBroadcastHashJoinExec( nativeOp, op, + op.output, + op.outputOrdering, op.leftKeys, op.rightKeys, op.joinType, @@ -499,6 +533,8 @@ class CometSparkSessionExtensions CometSortMergeJoinExec( nativeOp, op, + op.output, + op.outputOrdering, op.leftKeys, op.rightKeys, op.joinType, @@ -535,7 +571,7 @@ class CometSparkSessionExtensions && isCometNative(child) => QueryPlanSerde.operator2Proto(c) match { case Some(nativeOp) => - val cometOp = CometCoalesceExec(c, numPartitions, child) + val cometOp = CometCoalesceExec(c, c.output, numPartitions, child) CometSinkPlaceHolder(nativeOp, c, cometOp) case None => c @@ -559,7 +595,13 @@ class CometSparkSessionExtensions QueryPlanSerde.operator2Proto(s) match { case Some(nativeOp) => val cometOp = - CometTakeOrderedAndProjectExec(s, s.limit, s.sortOrder, s.projectList, s.child) + CometTakeOrderedAndProjectExec( + s, + s.output, + s.limit, + s.sortOrder, + s.projectList, + s.child) CometSinkPlaceHolder(nativeOp, s, cometOp) case None => s @@ -580,7 +622,13 @@ class CometSparkSessionExtensions newOp match { case Some(nativeOp) => val cometOp = - CometWindowExec(w, w.windowExpression, w.partitionSpec, w.orderSpec, w.child) + CometWindowExec( + w, + w.output, + w.windowExpression, + w.partitionSpec, + w.orderSpec, + w.child) CometSinkPlaceHolder(nativeOp, w, cometOp) case None => w @@ -591,7 +639,7 @@ class CometSparkSessionExtensions u.children.forall(isCometNative) => QueryPlanSerde.operator2Proto(u) match { case Some(nativeOp) => - val cometOp = CometUnionExec(u, u.children) + val cometOp = CometUnionExec(u, u.output, u.children) CometSinkPlaceHolder(nativeOp, u, cometOp) case None => u @@ -631,7 +679,7 @@ class CometSparkSessionExtensions isSpark34Plus => // Spark 3.4+ only QueryPlanSerde.operator2Proto(b) match { case Some(nativeOp) => - val cometOp = CometBroadcastExchangeExec(b, b.child) + val cometOp = CometBroadcastExchangeExec(b, b.output, b.child) CometSinkPlaceHolder(nativeOp, b, cometOp) case None => b } 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 38247b2c4..64d14a00e 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 @@ -31,6 +31,7 @@ import org.apache.spark.comet.shims.ShimCometBroadcastExchangeExec import org.apache.spark.launcher.SparkLauncher import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, SQLExecution} @@ -60,7 +61,10 @@ import org.apache.comet.CometRuntimeException * Note that this only supports Spark 3.4 and later, because the serialization class * `ChunkedByteBuffer` is only serializable in Spark 3.4 and later. */ -case class CometBroadcastExchangeExec(originalPlan: SparkPlan, child: SparkPlan) +case class CometBroadcastExchangeExec( + originalPlan: SparkPlan, + override val output: Seq[Attribute], + override val child: SparkPlan) extends BroadcastExchangeLike with ShimCometBroadcastExchangeExec { import CometBroadcastExchangeExec._ @@ -75,7 +79,7 @@ case class CometBroadcastExchangeExec(originalPlan: SparkPlan, child: SparkPlan) "broadcastTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to broadcast")) override def doCanonicalize(): SparkPlan = { - CometBroadcastExchangeExec(originalPlan.canonicalized, child.canonicalized) + CometBroadcastExchangeExec(null, null, child.canonicalized) } override def runtimeStatistics: Statistics = { diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala index cc635d739..b216c7d81 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.comet import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, SinglePartition, UnknownPartitioning} import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.vectorized.ColumnarBatch @@ -32,6 +33,7 @@ import com.google.common.base.Objects */ case class CometCoalesceExec( override val originalPlan: SparkPlan, + override val output: Seq[Attribute], numPartitions: Int, child: SparkPlan) extends CometExec diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala index d2c9158ee..3effcca4f 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala @@ -22,6 +22,8 @@ package org.apache.spark.sql.comet import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, SinglePartition} import org.apache.spark.sql.comet.execution.shuffle.{CometShuffledBatchRDD, CometShuffleExchangeExec} import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec.{METRIC_NATIVE_TIME_DESCRIPTION, METRIC_NATIVE_TIME_NAME} import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, UnaryExecNode, UnsafeRowSerializer} @@ -45,6 +47,8 @@ case class CometCollectLimitExec( child: SparkPlan) extends CometExec with UnaryExecNode { + override def output: Seq[Attribute] = child.output + override def outputPartitioning: Partitioning = SinglePartition private lazy val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala index 6e9bfe424..3f09e2fe2 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala @@ -42,14 +42,13 @@ import org.apache.comet.shims.ShimCometTakeOrderedAndProjectExec */ case class CometTakeOrderedAndProjectExec( override val originalPlan: SparkPlan, + override val output: Seq[Attribute], limit: Int, sortOrder: Seq[SortOrder], projectList: Seq[NamedExpression], child: SparkPlan) extends CometExec with UnaryExecNode { - override def output: Seq[Attribute] = projectList.map(_.toAttribute) - private lazy val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) private lazy val readMetrics = diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala index 9685e75e1..4c10a8abb 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala @@ -43,6 +43,7 @@ import org.apache.comet.serde.QueryPlanSerde.{exprToProto, serializeDataType, wi */ case class CometWindowExec( override val originalPlan: SparkPlan, + override val output: Seq[Attribute], windowExpression: Seq[NamedExpression], partitionSpec: Seq[Expression], orderSpec: Seq[SortOrder], @@ -52,8 +53,6 @@ case class CometWindowExec( override def nodeName: String = "CometWindowExec" - override def output: Seq[Attribute] = child.output ++ windowExpression.map(_.toAttribute) - private lazy val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) private lazy val readMetrics = 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 8b1ba168f..1c9bf9947 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 @@ -39,7 +39,6 @@ import org.apache.spark.sql.comet.plans.PartitioningPreservingUnaryExecNode import org.apache.spark.sql.comet.util.Utils import org.apache.spark.sql.execution.{BinaryExecNode, ColumnarToRowExec, ExecSubqueryExpression, ExplainUtils, LeafExecNode, ScalarSubquery, SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.adaptive.{AQEShuffleReadExec, BroadcastQueryStageExec, ShuffleQueryStageExec} -import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.internal.SQLConf @@ -154,7 +153,7 @@ abstract class CometNativeExec extends CometExec { /** The Comet native operator */ def nativeOp: Operator - override protected def doPrepare(): Unit = prepareSubqueries(originalPlan) + override protected def doPrepare(): Unit = prepareSubqueries(this) override lazy val metrics: Map[String, SQLMetric] = CometMetricNode.baselineMetrics(sparkContext) @@ -217,12 +216,12 @@ abstract class CometNativeExec extends CometExec { val it = new CometExecIterator(CometExec.newIterId, inputs, serializedPlanCopy, nativeMetrics) - setSubqueries(it.id, originalPlan) + setSubqueries(it.id, this) Option(TaskContext.get()).foreach { context => context.addTaskCompletionListener[Unit] { _ => it.close() - cleanSubqueries(it.id, originalPlan) + cleanSubqueries(it.id, this) } } @@ -381,11 +380,13 @@ abstract class CometNativeExec extends CometExec { */ protected def canonicalizePlans(): CometNativeExec = { def transform(arg: Any): AnyRef = arg match { - case sparkPlan: SparkPlan if !sparkPlan.isInstanceOf[CometNativeExec] => + case sparkPlan: SparkPlan + if !sparkPlan.isInstanceOf[CometNativeExec] && + children.forall(_ != sparkPlan) => // Different to Spark, Comet native query node might have a Spark plan as Product element. // We need to canonicalize the Spark plan. But it cannot be another Comet native query node, // otherwise it will cause recursive canonicalization. - sparkPlan.canonicalized + null case other: AnyRef => other case null => null } @@ -412,8 +413,8 @@ case class SerializedPlan(plan: Option[Array[Byte]]) { case class CometProjectExec( override val nativeOp: Operator, override val originalPlan: SparkPlan, - projectList: Seq[NamedExpression], override val output: Seq[Attribute], + projectList: Seq[NamedExpression], child: SparkPlan, override val serializedPlanOpt: SerializedPlan) extends CometUnaryExec @@ -443,18 +444,20 @@ case class CometProjectExec( case class CometFilterExec( override val nativeOp: Operator, override val originalPlan: SparkPlan, + override val output: Seq[Attribute], condition: Expression, child: SparkPlan, override val serializedPlanOpt: SerializedPlan) extends CometUnaryExec { override def outputPartitioning: Partitioning = child.outputPartitioning + override def outputOrdering: Seq[SortOrder] = child.outputOrdering override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = this.copy(child = newChild) override def stringArgs: Iterator[Any] = - Iterator(originalPlan.output, condition, child) + Iterator(output, condition, child) override def equals(obj: Any): Boolean = { obj match { @@ -480,6 +483,8 @@ case class CometFilterExec( case class CometSortExec( override val nativeOp: Operator, override val originalPlan: SparkPlan, + override val output: Seq[Attribute], + override val outputOrdering: Seq[SortOrder], sortOrder: Seq[SortOrder], child: SparkPlan, override val serializedPlanOpt: SerializedPlan) @@ -491,7 +496,7 @@ case class CometSortExec( this.copy(child = newChild) override def stringArgs: Iterator[Any] = - Iterator(originalPlan.output, sortOrder, child) + Iterator(output, sortOrder, child) override def equals(obj: Any): Boolean = { obj match { @@ -520,7 +525,9 @@ case class CometLocalLimitExec( override val serializedPlanOpt: SerializedPlan) extends CometUnaryExec { + override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = child.outputPartitioning + override def outputOrdering: Seq[SortOrder] = child.outputOrdering override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = this.copy(child = newChild) @@ -550,7 +557,9 @@ case class CometGlobalLimitExec( override val serializedPlanOpt: SerializedPlan) extends CometUnaryExec { + override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = child.outputPartitioning + override def outputOrdering: Seq[SortOrder] = child.outputOrdering override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = this.copy(child = newChild) @@ -573,10 +582,13 @@ case class CometGlobalLimitExec( case class CometExpandExec( override val nativeOp: Operator, override val originalPlan: SparkPlan, + override val output: Seq[Attribute], projections: Seq[Seq[Expression]], child: SparkPlan, override val serializedPlanOpt: SerializedPlan) extends CometUnaryExec { + override def outputPartitioning: Partitioning = UnknownPartitioning(0) + override def producedAttributes: AttributeSet = outputSet override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = @@ -600,7 +612,10 @@ case class CometExpandExec( override lazy val metrics: Map[String, SQLMetric] = Map.empty } -case class CometUnionExec(override val originalPlan: SparkPlan, children: Seq[SparkPlan]) +case class CometUnionExec( + override val originalPlan: SparkPlan, + override val output: Seq[Attribute], + children: Seq[SparkPlan]) extends CometExec { override def doExecuteColumnar(): RDD[ColumnarBatch] = { sparkContext.union(children.map(_.executeColumnar())) @@ -634,8 +649,10 @@ case class CometUnionExec(override val originalPlan: SparkPlan, children: Seq[Sp case class CometHashAggregateExec( override val nativeOp: Operator, override val originalPlan: SparkPlan, + override val output: Seq[Attribute], groupingExpressions: Seq[NamedExpression], aggregateExpressions: Seq[AggregateExpression], + resultExpressions: Seq[NamedExpression], input: Seq[Attribute], mode: Option[AggregateMode], child: SparkPlan, @@ -674,13 +691,14 @@ case class CometHashAggregateExec( override def hashCode(): Int = Objects.hashCode(groupingExpressions, aggregateExpressions, input, mode, child) - override protected def outputExpressions: Seq[NamedExpression] = - originalPlan.asInstanceOf[HashAggregateExec].resultExpressions + override protected def outputExpressions: Seq[NamedExpression] = resultExpressions } case class CometHashJoinExec( override val nativeOp: Operator, override val originalPlan: SparkPlan, + override val output: Seq[Attribute], + override val outputOrdering: Seq[SortOrder], leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, @@ -733,6 +751,8 @@ case class CometHashJoinExec( case class CometBroadcastHashJoinExec( override val nativeOp: Operator, override val originalPlan: SparkPlan, + override val output: Seq[Attribute], + override val outputOrdering: Seq[SortOrder], leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, @@ -867,6 +887,8 @@ case class CometBroadcastHashJoinExec( case class CometSortMergeJoinExec( override val nativeOp: Operator, override val originalPlan: SparkPlan, + override val output: Seq[Attribute], + override val outputOrdering: Seq[SortOrder], leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/simplified.txt index dc0fb18f3..d6f75837b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/simplified.txt @@ -13,10 +13,10 @@ TakeOrderedAndProject [c_customer_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] + CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk,sr_customer_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,11 +24,11 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #4 @@ -45,10 +45,10 @@ TakeOrderedAndProject [c_customer_id] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] + CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -58,12 +58,12 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/simplified.txt index 19243e359..efd4b187d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -60,12 +60,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] + CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/simplified.txt index dd1a52206..127a5a291 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/simplified.txt @@ -13,25 +13,25 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -42,25 +42,25 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -72,15 +72,15 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -93,15 +93,15 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/simplified.txt index 905a35c8b..4c2d3f3d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/simplified.txt index 4de403664..98d995db0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/simplified.txt @@ -5,18 +5,18 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + CometHashAggregate [sum,count,sum,count,sum,count,sum,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] + CometBroadcastHashJoin [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status,hd_demo_sk,hd_dep_count] CometProject [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk,d_date_sk] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,s_store_sk] + CometFilter [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,22 +24,22 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 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] - CometBroadcastExchange #7 + CometBroadcastExchange [hd_demo_sk,hd_dep_count] #7 CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/simplified.txt index e3dfa631b..2257d398f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/simplified.txt @@ -19,20 +19,20 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 @@ -48,7 +48,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -56,7 +56,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -65,7 +65,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 @@ -79,10 +79,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 @@ -90,26 +90,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #9 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #9 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #10 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #11 + CometBroadcastExchange [d_date_sk] #11 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #11 InputAdapter @@ -118,10 +118,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 @@ -132,7 +132,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #4 @@ -154,7 +154,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [cs_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -179,7 +179,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ws_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ws_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/simplified.txt index 799f74a36..aa49638c7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/simplified.txt @@ -10,20 +10,20 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 @@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -47,13 +47,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #2 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 @@ -77,10 +77,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -88,26 +88,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #8 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk] #10 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #10 InputAdapter @@ -116,10 +116,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 @@ -153,7 +153,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #16 @@ -161,13 +161,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #6 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/simplified.txt index a03346372..b876fe4de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/simplified.txt @@ -6,14 +6,14 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_zip,cs_sales_price] + CometHashAggregate [ca_zip,sum,cs_sales_price] CometProject [cs_sales_price,ca_zip] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,d_date_sk] CometProject [cs_sales_price,cs_sold_date_sk,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_state,ca_zip] CometProject [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -21,15 +21,15 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #4 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4 + CometFilter [ca_address_sk,ca_state,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/simplified.txt index a55c182be..8935abb54 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/simplified.txt @@ -26,7 +26,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] + CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -54,7 +54,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -62,7 +62,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -70,5 +70,5 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [cc_call_center_sk] - CometFilter [cc_county,cc_call_center_sk] + CometFilter [cc_call_center_sk,cc_county] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/simplified.txt index 77aba376e..ae6bab279 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/simplified.txt @@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] + CometHashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,ss_quantity,sr_return_quantity,cs_quantity] CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 @@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/simplified.txt index 3d101857b..2519d58a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/simplified.txt @@ -6,21 +6,21 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] CometExpand [i_item_id,ca_country,ca_state,ca_county] [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,26 +28,26 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk] #5 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #6 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/simplified.txt index 18a69bcb4..163a31f47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/simplified.txt @@ -6,33 +6,33 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] + CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_store_sk,s_store_sk,ca_zip,s_zip] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip,s_store_sk,s_zip] CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip] CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_sk] CometProject [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_zip] #5 CometFilter [ca_address_sk,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange #6 - CometFilter [s_zip,s_store_sk] + CometBroadcastExchange [s_store_sk,s_zip] #6 + CometFilter [s_store_sk,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt index c7999d981..5d0658192 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt @@ -13,16 +13,16 @@ WholeStageCodegen (7) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_week_seq,d_day_name,sales_price] + CometHashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum,d_day_name,sales_price] CometProject [sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [sold_date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [sold_date_sk,sales_price,d_date_sk,d_week_seq,d_day_name] + CometUnion [sold_date_sk,sales_price] CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange #3 - CometFilter [d_date_sk,d_week_seq] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 + CometFilter [d_date_sk,d_week_seq,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #4 @@ -30,7 +30,7 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_week_seq] + CometFilter [d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] InputAdapter BroadcastExchange #5 @@ -46,5 +46,5 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_week_seq] + CometFilter [d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/simplified.txt index d805e3868..08088a386 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/simplified.txt index 52bd7a85e..bd81a3a7b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/simplified.txt @@ -7,29 +7,29 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,i_item_id,sum,sum,d_date,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id,d_date_sk,d_date] CometProject [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_sk,i_item_id] CometProject [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - CometFilter [inv_warehouse_sk,inv_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk,i_item_id] #4 CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange #5 - CometFilter [d_date,d_date_sk] + CometBroadcastExchange [d_date_sk,d_date] #5 + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/simplified.txt index bda583c17..7d36dc400 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/simplified.txt @@ -6,15 +6,15 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count,inv_quantity_on_hand] CometExpand [i_product_name,i_brand,i_class,i_category] [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category,spark_grouping_id] CometProject [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,15 +22,15 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange #5 + CometBroadcastExchange [w_warehouse_sk] #5 CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/simplified.txt index f818fd25f..e065a0a87 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/simplified.txt @@ -28,7 +28,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -41,12 +41,12 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date] + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 @@ -54,14 +54,14 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (6) @@ -81,12 +81,12 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 @@ -94,12 +94,12 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange #14 + CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter @@ -107,13 +107,13 @@ WholeStageCodegen (18) WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #10 + CometBroadcastExchange [c_customer_sk] #10 CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt index 54ee3dbde..2862f64cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [cs_item_sk,item_sk] ColumnarToRow InputAdapter - CometFilter [cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,7 +29,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -42,12 +42,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date] + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 @@ -55,14 +55,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (6) @@ -82,12 +82,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 @@ -95,12 +95,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange #14 + CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter @@ -108,13 +108,13 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #10 + CometBroadcastExchange [c_customer_sk] #10 CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter @@ -129,7 +129,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter WholeStageCodegen (10) @@ -163,7 +163,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [ws_item_sk,item_sk] ColumnarToRow InputAdapter - CometFilter [ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt index 8ebd45fd1..30091beb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt @@ -39,7 +39,7 @@ WholeStageCodegen (11) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 @@ -74,7 +74,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -85,7 +85,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #5 @@ -93,26 +93,26 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_birth_country] + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_country,ca_zip] + CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt index 8ebd45fd1..30091beb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt @@ -39,7 +39,7 @@ WholeStageCodegen (11) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 @@ -74,7 +74,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -85,7 +85,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #5 @@ -93,26 +93,26 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_birth_country] + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_country,ca_zip] + CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt index 74a5e8777..1f4faaac5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt @@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] + CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,ss_net_profit,sr_net_loss,cs_net_profit] CometProject [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_sk,s_store_id,s_store_name] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 @@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #8 + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/simplified.txt index cba306f68..a39fbbb79 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/simplified.txt @@ -6,16 +6,16 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - CometBroadcastHashJoin [cs_promo_sk,p_promo_sk] + CometBroadcastHashJoin [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id,p_promo_sk] CometProject [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk,cd_demo_sk] + CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [p_promo_sk] #6 CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/simplified.txt index 7fcbe967a..9c4dd659a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/simplified.txt @@ -6,17 +6,17 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + CometHashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] CometExpand [i_item_id,s_state] [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state,spark_grouping_id] CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,19 +24,19 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [s_state,s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/simplified.txt index 47a4fe2c7..80c886708 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/simplified.txt @@ -15,9 +15,9 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #3 @@ -33,9 +33,9 @@ WholeStageCodegen (18) WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #6 @@ -51,9 +51,9 @@ WholeStageCodegen (18) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #9 @@ -69,9 +69,9 @@ WholeStageCodegen (18) WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #12 @@ -87,9 +87,9 @@ WholeStageCodegen (18) WholeStageCodegen (12) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #15 @@ -105,7 +105,7 @@ WholeStageCodegen (18) WholeStageCodegen (15) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/simplified.txt index 2bcef6168..88b4ecd1d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/simplified.txt @@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] + CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,ss_quantity,sr_return_quantity,cs_quantity] CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_store_id,s_store_name] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 @@ -40,10 +40,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -51,23 +51,23 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #8 + CometBroadcastExchange [d_date_sk] #8 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk] #9 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #10 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10 + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange #11 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/simplified.txt index cc9c4edf3..bccda58a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] + CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/simplified.txt index f24e34c36..a80e0ca71 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/simplified.txt @@ -13,12 +13,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] + CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_addr_sk,wr_returning_customer_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter @@ -50,12 +50,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] + CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_addr_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -65,7 +65,7 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] InputAdapter BroadcastExchange #9 @@ -73,5 +73,5 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt index a94a8a94d..a48c7ad26 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt @@ -17,24 +17,24 @@ WholeStageCodegen (13) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_county] #5 CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter @@ -46,22 +46,22 @@ WholeStageCodegen (13) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #9 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter @@ -73,22 +73,22 @@ WholeStageCodegen (13) WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #12 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #13 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter @@ -100,12 +100,12 @@ WholeStageCodegen (13) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year,d_qoy] #4 @@ -119,12 +119,12 @@ WholeStageCodegen (13) WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #9 @@ -138,12 +138,12 @@ WholeStageCodegen (13) WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk,d_year,d_qoy] #13 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/simplified.txt index b8df1e929..c3bfdfd77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/simplified.txt @@ -11,8 +11,8 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk,cs_ext_discount_amt] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -20,11 +20,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 @@ -36,15 +36,15 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cs_item_sk,cs_ext_discount_amt] + CometHashAggregate [cs_item_sk,sum,count,cs_ext_discount_amt] CometProject [cs_item_sk,cs_ext_discount_amt] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/simplified.txt index af2b7cb5d..7a3c2c153 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/simplified.txt @@ -14,14 +14,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,ss_ext_sales_price] + CometHashAggregate [i_manufact_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_manufact_id,i_manufact_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_manufact_id] #6 + CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_manufact_id] #7 CometProject [i_manufact_id] - CometFilter [i_category] + CometFilter [i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] WholeStageCodegen (4) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -54,14 +54,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,cs_ext_sales_price] + CometHashAggregate [i_manufact_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_manufact_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -74,14 +74,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,ws_ext_sales_price] + CometHashAggregate [i_manufact_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt index eefd38343..e1f7f9bd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/simplified.txt index efe0b0b4e..9deb51342 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/simplified.txt index 574a20fc0..4ad9b501b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/simplified.txt @@ -14,15 +14,15 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [i_category,i_class,spark_grouping_id,sum,sum,ss_net_profit,ss_ext_sales_price] CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,16 +30,16 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt index c3fcd79f0..efd86d5ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt @@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] InputAdapter CometHashAggregate [i_item_id,i_item_desc,i_current_price] CometProject [i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometBroadcastExchange #2 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,cs_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - CometBroadcastHashJoin [i_item_sk,inv_item_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange #3 + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [cs_item_sk] - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/simplified.txt index cfac83844..9d667265c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/simplified.txt @@ -15,10 +15,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,14 +26,14 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #5 - CometFilter [c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 @@ -46,10 +46,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [cs_bill_customer_sk,d_date] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] + CometFilter [cs_bill_customer_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 @@ -65,10 +65,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ws_bill_customer_sk,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt index 7d28d5c10..0c709e4f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,17 +27,17 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_moy] #6 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 @@ -50,14 +50,14 @@ WholeStageCodegen (5) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -65,11 +65,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk,d_moy] #10 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt index 7d28d5c10..0c709e4f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,17 +27,17 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_moy] #6 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 @@ -50,14 +50,14 @@ WholeStageCodegen (5) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -65,11 +65,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk,d_moy] #10 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/simplified.txt index 048da153b..1a19c58ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/simplified.txt @@ -16,25 +16,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -45,25 +45,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -75,15 +75,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [cs_bill_customer_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #12 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -96,15 +96,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [cs_bill_customer_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #15 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 @@ -118,15 +118,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #18 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #18 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -139,15 +139,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #21 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #21 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/simplified.txt index 10e0735b4..11c0201a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/simplified.txt @@ -21,14 +21,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [cs_warehouse_sk,cs_item_sk] + CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -39,14 +39,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [w_warehouse_sk] + CometFilter [w_warehouse_sk,w_state] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] InputAdapter BroadcastExchange #6 @@ -54,7 +54,7 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] ColumnarToRow InputAdapter CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] InputAdapter ReusedExchange [d_date_sk,d_date] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt index e31217066..457d441ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject [i_product_name] ColumnarToRow InputAdapter CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact] + CometFilter [i_manufact_id,i_manufact,i_product_name] CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] InputAdapter BroadcastExchange #2 @@ -23,7 +23,7 @@ TakeOrderedAndProject [i_product_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact] + CometHashAggregate [i_manufact,count] CometProject [i_manufact] - CometFilter [i_category,i_color,i_units,i_size,i_manufact] + CometFilter [i_category,i_manufact,i_size,i_color,i_units] CometScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/simplified.txt index 3e69a3341..e3b850ef5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price] + CometHashAggregate [d_year,i_category_id,i_category,sum,ss_ext_sales_price] CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_category_id,i_category] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 CometProject [i_item_sk,i_category_id,i_category] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/simplified.txt index d29a65bd4..5a060b85c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_name,s_store_id,d_day_name,ss_sales_price] + CometHashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [d_day_name,ss_store_sk,ss_sales_price,s_store_sk,s_store_id,s_store_name] CometProject [d_day_name,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_day_name] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange #2 - CometFilter [ss_store_sk] + CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 CometProject [s_store_sk,s_store_id,s_store_name] - CometFilter [s_gmt_offset,s_store_sk] + CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt index 0a66507e1..232d65d72 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt @@ -30,9 +30,9 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_store_sk,ss_net_profit] + CometHashAggregate [ss_store_sk,sum,count,ss_net_profit] CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_store_sk,ss_addr_sk] + CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count] InputAdapter @@ -40,9 +40,9 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,ss_net_profit] + CometHashAggregate [ss_item_sk,sum,count,ss_net_profit] CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_store_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] InputAdapter WholeStageCodegen (8) @@ -69,7 +69,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (9) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_product_name] InputAdapter ReusedExchange [i_item_sk,i_product_name] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt index c376c4fcd..f60fdb18a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt @@ -11,14 +11,14 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ca_city,ca_zip,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_sales_price,ca_city,ca_zip] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip,d_date_sk] CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_city,ca_zip] CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - CometFilter [ws_bill_customer_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,20 +26,20 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #4 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4 + CometFilter [ca_address_sk,ca_city,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #7 @@ -47,5 +47,5 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [i_item_id] - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/simplified.txt index 93e3eb05b..57defd961 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/simplified.txt @@ -10,16 +10,16 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,ss_coupon_amt,ss_net_profit] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_address_sk,ca_city] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] + CometFilter [s_store_sk,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk,ca_city] #6 CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter @@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/simplified.txt index d2e615f39..b6e5c469b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/simplified.txt index 6ebf6af07..5f628dbae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/simplified.txt @@ -5,16 +5,16 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_quantity] + CometHashAggregate [sum,ss_quantity] CometProject [ss_quantity] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] CometProject [ss_quantity,ss_sold_date_sk] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + CometBroadcastHashJoin [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] CometProject [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,s_store_sk] + CometFilter [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,19 +22,19 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #4 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] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_state] #5 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/simplified.txt index 8d7b158d0..0e6b65b06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] + CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] 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 + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #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] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_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 @@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] 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] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] @@ -71,18 +71,18 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] + CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] 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 #9 + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 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] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_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] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] ReusedExchange [d_date_sk] #6 WholeStageCodegen (15) @@ -105,17 +105,17 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] + CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] 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 #12 + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 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] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_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] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt index c4ea8fe24..537cba446 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt @@ -15,14 +15,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,s_store_sk] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,18 +30,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (4) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] @@ -50,23 +50,23 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #7 - CometFilter [cp_catalog_page_sk] + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (6) 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] @@ -75,25 +75,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] 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] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_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 + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #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] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #10 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_site_id] #10 + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/simplified.txt index 473b9cdd4..58adabcec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sr_returned_date_sk,ss_sold_date_sk] + CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sum,sum,sum,sum,sum,sr_returned_date_sk,ss_sold_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk,s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometProject [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] - CometFilter [ss_ticket_number,ss_item_sk,ss_customer_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [sr_ticket_number,sr_item_sk,sr_customer_sk] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] #2 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,15 +26,15 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #4 + CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometFilter [d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/simplified.txt index ce6005da6..988297f02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/simplified.txt @@ -30,10 +30,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,d_date,ws_sales_price] + CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -41,11 +41,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_date] #6 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter WholeStageCodegen (10) @@ -67,10 +67,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,d_date,ss_sales_price] + CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/simplified.txt index 557dd3b4a..a0932c7cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [d_year,ext_price,brand_id,brand] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] + CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/simplified.txt index 83e53bb66..b2199fb34 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/simplified.txt @@ -15,18 +15,18 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,d_qoy,ss_sales_price] + CometHashAggregate [i_manufact_id,d_qoy,sum,ss_sales_price] CometProject [i_manufact_id,ss_sales_price,d_qoy] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [i_item_sk,i_manufact_id] - CometFilter [i_category,i_class,i_brand,i_item_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -34,12 +34,12 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [d_date_sk,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_qoy] #5 CometProject [d_date_sk,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/simplified.txt index c6886735f..2193d875c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/simplified.txt @@ -26,14 +26,14 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [c_customer_sk,c_current_addr_sk] CometProject [c_customer_sk,c_current_addr_sk] - CometBroadcastHashJoin [customer_sk,c_customer_sk] + CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk] CometProject [customer_sk] - CometBroadcastHashJoin [sold_date_sk,d_date_sk] + CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] CometProject [sold_date_sk,customer_sk] - CometBroadcastHashJoin [item_sk,i_item_sk] - CometUnion + CometBroadcastHashJoin [sold_date_sk,customer_sk,item_sk,i_item_sk] + CometUnion [sold_date_sk,customer_sk,item_sk] CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -41,21 +41,21 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #5 + CometBroadcastExchange [i_item_sk] #5 CometProject [i_item_sk] - CometFilter [i_category,i_class,i_item_sk] + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #7 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -71,7 +71,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] Subquery #3 WholeStageCodegen (2) HashAggregate [(d_month_seq + 1)] @@ -82,7 +82,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [(d_month_seq + 1)] CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] Subquery #4 WholeStageCodegen (2) @@ -94,7 +94,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [(d_month_seq + 3)] CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/simplified.txt index 2750a6ba2..fc38884c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [ext_price,brand_id,brand] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_brand,i_brand_id,ss_ext_sales_price] + CometHashAggregate [i_brand,i_brand_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/simplified.txt index 7fdead831..0b48046fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/simplified.txt @@ -14,14 +14,14 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_ext_sales_price] + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_item_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_item_id,i_item_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_item_id] #7 CometProject [i_item_id] - CometFilter [i_color] + CometFilter [i_item_id,i_color] CometScan parquet spark_catalog.default.item [i_item_id,i_color] WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -54,14 +54,14 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_ext_sales_price] + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -74,14 +74,14 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ws_ext_sales_price] + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/simplified.txt index c630cad48..fb2b09b9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [cs_item_sk,cs_call_center_sk] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/simplified.txt index 2ed2bde44..fc7c21e94 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/simplified.txt @@ -11,12 +11,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_ext_sales_price] + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,27 +24,27 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date] #3 CometProject [d_date] - CometFilter [d_week_seq] + CometFilter [d_date,d_week_seq] Subquery #2 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_date] + CometFilter [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk,i_item_id] #4 CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 InputAdapter @@ -57,12 +57,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_ext_sales_price] + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #4 @@ -77,12 +77,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ws_ext_sales_price] + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/simplified.txt index e00d52dbb..6c1e6cd92 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/simplified.txt @@ -12,20 +12,20 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price] + CometHashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_week_seq,d_day_name] + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [d_date_sk,d_week_seq] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 + CometFilter [d_date_sk,d_week_seq,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/simplified.txt index 89a080d85..824670101 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/simplified.txt @@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #2 - CometFilter [c_current_addr_sk,c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #3 - CometFilter [ss_customer_sk,ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] Subquery #2 WholeStageCodegen (2) HashAggregate [d_month_seq] @@ -41,12 +41,12 @@ TakeOrderedAndProject [cnt,state] InputAdapter CometHashAggregate [d_month_seq] CometProject [d_month_seq] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter @@ -56,7 +56,7 @@ TakeOrderedAndProject [cnt,state] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow InputAdapter - CometFilter [i_current_price,i_category,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #8 @@ -68,6 +68,6 @@ TakeOrderedAndProject [cnt,state] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_current_price] - CometFilter [i_category] + CometHashAggregate [i_category,sum,count,i_current_price] + CometFilter [i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt index b76e7c9b3..7cfcb75da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt @@ -14,14 +14,14 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_ext_sales_price] + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [i_item_id,total_sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_item_id,i_item_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_item_id] #7 CometProject [i_item_id] - CometFilter [i_category] + CometFilter [i_item_id,i_category] CometScan parquet spark_catalog.default.item [i_item_id,i_category] WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -54,14 +54,14 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_ext_sales_price] + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -74,14 +74,14 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ws_ext_sales_price] + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/simplified.txt index 4ca1dd667..1b2af33b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/simplified.txt @@ -7,20 +7,20 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_sales_price] + CometHashAggregate [sum,ss_ext_sales_price] CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,p_promo_sk] CometProject [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,30 +28,30 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometProject [s_store_sk] - CometFilter [s_gmt_offset,s_store_sk] + CometFilter [s_store_sk,s_gmt_offset] CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - CometBroadcastExchange #4 + CometBroadcastExchange [p_promo_sk] #4 CometProject [p_promo_sk] - CometFilter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk] #7 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #8 + CometBroadcastExchange [i_item_sk] #8 CometProject [i_item_sk] - CometFilter [i_category,i_item_sk] + CometFilter [i_item_sk,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_category] InputAdapter BroadcastExchange #9 @@ -62,18 +62,18 @@ WholeStageCodegen (4) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_sales_price] + CometHashAggregate [sum,ss_ext_sales_price] CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_customer_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [s_store_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/simplified.txt index c6b7e1834..0b4fc61a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/simplified.txt @@ -6,27 +6,27 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,sm_type,web_name,ws_ship_date_sk,ws_sold_date_sk] + CometHashAggregate [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum,ws_ship_date_sk,ws_sold_date_sk] CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression] - CometBroadcastHashJoin [ws_ship_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name,d_date_sk] CometProject [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] - CometBroadcastHashJoin [ws_web_site_sk,web_site_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name] CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - CometFilter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #3 - CometFilter [sm_ship_mode_sk] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange #4 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_name] #4 + CometFilter [web_site_sk,web_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/simplified.txt index 35e09ec7d..b2033c7a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/simplified.txt @@ -15,18 +15,18 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manager_id,d_moy,ss_sales_price] + CometHashAggregate [i_manager_id,d_moy,sum,ss_sales_price] CometProject [i_manager_id,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [i_item_sk,i_manager_id] - CometFilter [i_category,i_class,i_brand,i_item_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -34,12 +34,12 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt index d972e0082..3b5e4f19e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt @@ -54,19 +54,19 @@ WholeStageCodegen (52) 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] - 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] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #4 + CometFilter [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] 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] + CometFilter [d_date_sk,d_year] 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] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (8) @@ -89,7 +89,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -100,7 +100,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] 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 @@ -116,14 +116,14 @@ WholeStageCodegen (52) WholeStageCodegen (11) 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] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter BroadcastExchange #11 WholeStageCodegen (12) ColumnarToRow InputAdapter - CometFilter [d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #11 @@ -157,7 +157,7 @@ WholeStageCodegen (52) WholeStageCodegen (19) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] InputAdapter ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 @@ -176,7 +176,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] - CometFilter [i_current_price,i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter WholeStageCodegen (50) @@ -227,19 +227,19 @@ WholeStageCodegen (52) 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] - 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] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #20 + CometFilter [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] 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] + CometFilter [d_date_sk,d_year] 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] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (33) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/simplified.txt index 8de564ed1..6cb247973 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/simplified.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes BroadcastHashJoin [s_store_sk,ss_store_sk] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #1 @@ -20,10 +20,10 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] + CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -31,18 +31,18 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] InputAdapter BroadcastExchange #6 @@ -59,10 +59,10 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] + CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/simplified.txt index d746739b3..72133811b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/simplified.txt @@ -14,37 +14,37 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] + CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] CometProject [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk] CometProject [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk] CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy] CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - CometFilter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 + CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastExchange #5 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [t_time_sk] #6 CometProject [t_time_sk] - CometFilter [t_time,t_time_sk] + CometFilter [t_time_sk,t_time] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] - CometBroadcastExchange #7 + CometBroadcastExchange [sm_ship_mode_sk] #7 CometProject [sm_ship_mode_sk] - CometFilter [sm_carrier,sm_ship_mode_sk] + CometFilter [sm_ship_mode_sk,sm_carrier] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] WholeStageCodegen (4) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] @@ -53,16 +53,16 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] + CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] CometProject [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk] CometProject [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk] CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy] CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - CometFilter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/simplified.txt index ef965d7d0..4dc4bcaf5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/simplified.txt @@ -17,15 +17,15 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty,ss_sales_price,ss_quantity] CometExpand [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] CometProject [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -33,15 +33,15 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ ColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/simplified.txt index 43f44c9f7..c5f56a69c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/simplified.txt @@ -10,16 +10,16 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_address_sk,ca_city] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] + CometFilter [s_store_sk,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk,ca_city] #6 CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter @@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/simplified.txt index de2d5eeda..c9cc4959b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,11 +27,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -39,7 +39,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -49,7 +49,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/simplified.txt index c583ba8e8..f327d5b06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/simplified.txt @@ -6,16 +6,16 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] + CometBroadcastHashJoin [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,p_promo_sk] CometProject [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_item_sk,ss_promo_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [p_promo_sk] #6 CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/simplified.txt index 7cddbc640..030dfc4d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/simplified.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #6 @@ -58,15 +58,15 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_state,ss_net_profit] + CometHashAggregate [s_state,sum,ss_net_profit] CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt index fd6777886..c604a8fa7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt @@ -9,19 +9,19 @@ WholeStageCodegen (3) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] + CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum,ext_price] CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] - CometBroadcastHashJoin [time_sk,t_time_sk] + CometBroadcastHashJoin [i_brand_id,i_brand,ext_price,time_sk,t_time_sk,t_hour,t_minute] CometProject [i_brand_id,i_brand,ext_price,time_sk] - CometBroadcastHashJoin [i_item_sk,sold_item_sk] - CometBroadcastExchange #3 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometUnion + CometUnion [ext_price,sold_item_sk,time_sk] CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_sold_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -29,25 +29,25 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_item_sk,cs_sold_time_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 CometProject [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_sold_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #6 + CometBroadcastExchange [t_time_sk,t_hour,t_minute] #6 CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_meal_time,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/simplified.txt index bea1fd4a1..6cba2d0e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/simplified.txt @@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #4 - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange #7 + CometBroadcastExchange [cd_demo_sk] #7 CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange #8 + CometBroadcastExchange [hd_demo_sk] #8 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #10 - CometFilter [d_week_seq,d_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #11 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #12 @@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt index 060c3e153..c91c4cf47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/simplified.txt index 2e6286d93..7c2a42ca3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/simplified.txt @@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/simplified.txt index 34866bc37..27a4dcb67 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/simplified.txt @@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #7 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] @@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] @@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_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 WholeStageCodegen (38) @@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange #18 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (23) @@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/simplified.txt index 473eef9e4..e463296d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/simplified.txt @@ -6,33 +6,33 @@ TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometUnion + CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum,ext_sales_price] + CometUnion [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] CometProject [ss_store_sk,ss_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_category] #2 + CometFilter [i_item_sk,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange #3 - CometFilter [d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometProject [ws_ship_customer_sk,ws_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_ship_customer_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category] + CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedExchange [i_item_sk,i_category] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #3 CometProject [cs_ship_addr_sk,cs_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_ship_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category] + CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedExchange [i_item_sk,i_category] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/simplified.txt index 590c59fdc..f9088f784 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/simplified.txt @@ -17,12 +17,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] + CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit] CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,13 +30,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter @@ -48,12 +48,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,sr_return_amt,sr_net_loss] + CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss] CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -70,9 +70,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit] CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -82,9 +82,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometHashAggregate [cr_return_amount,cr_net_loss] + CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -97,16 +97,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit] CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #12 + CometBroadcastExchange [wp_web_page_sk] #12 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter @@ -118,12 +118,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] + CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss] CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/simplified.txt index 280687e30..13a8aecff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -45,7 +45,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -83,7 +83,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -109,7 +109,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/simplified.txt index b68a9474b..57598ec35 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/simplified.txt @@ -8,14 +8,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum,ss_coupon_amt,ss_net_profit] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk,s_city] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,24 +23,24 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk,s_city] #4 CometProject [s_store_sk,s_city] - CometFilter [s_number_employees,s_store_sk] + CometFilter [s_store_sk,s_number_employees,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/simplified.txt index adf555417..637f3b7ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/simplified.txt @@ -10,10 +10,10 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] ColumnarToRow InputAdapter CometProject [ss_net_profit,s_store_name,s_zip] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip] CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -21,14 +21,14 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 - CometFilter [s_store_sk,s_zip] + CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4 + 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 #5 @@ -55,12 +55,12 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_zip] + CometHashAggregate [ca_zip,count] CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] + CometFilter [ca_address_sk,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange #9 + CometBroadcastExchange [c_current_addr_sk] #9 CometProject [c_current_addr_sk] - CometFilter [c_preferred_cust_flag,c_current_addr_sk] + CometFilter [c_current_addr_sk,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt index 7e257bdc6..cdc8dc486 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -40,7 +40,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -51,7 +51,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -60,7 +60,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #7 @@ -68,7 +68,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [i_item_sk] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #8 @@ -76,7 +76,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [p_promo_sk] - CometFilter [p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (20) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] @@ -102,7 +102,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -114,7 +114,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -123,7 +123,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cp_catalog_page_sk] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter ReusedExchange [i_item_sk] #7 @@ -153,7 +153,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (21) ColumnarToRow InputAdapter - CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -165,7 +165,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -174,7 +174,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (26) ColumnarToRow InputAdapter - CometFilter [web_site_sk] + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] InputAdapter ReusedExchange [i_item_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/simplified.txt index ac7b2c7e0..11f3a0ff1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/simplified.txt @@ -13,12 +13,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] + CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_addr_sk,cr_returning_customer_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter @@ -50,12 +50,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] + CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_addr_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -65,12 +65,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] InputAdapter BroadcastExchange #9 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt index a4d96e6f1..71a1c8e17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt @@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] InputAdapter CometHashAggregate [i_item_id,i_item_desc,i_current_price] CometProject [i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometBroadcastExchange #2 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,ss_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - CometBroadcastHashJoin [i_item_sk,inv_item_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange #3 + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [ss_item_sk] - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/simplified.txt index ef7d35e21..390c20b7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/simplified.txt @@ -10,12 +10,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,sr_return_quantity] + CometHashAggregate [i_item_id,sum,sr_return_quantity] CometProject [sr_return_quantity,i_item_id] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [sr_return_quantity,sr_returned_date_sk,i_item_id,d_date_sk] CometProject [sr_return_quantity,sr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [sr_item_sk,i_item_sk] - CometFilter [sr_item_sk] + CometBroadcastHashJoin [sr_item_sk,sr_return_quantity,sr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [sr_item_sk,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,24 +23,24 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date] #3 CometProject [d_date] - CometBroadcastHashJoin [d_week_seq,d_week_seq] + CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_week_seq] #4 CometProject [d_week_seq] - CometFilter [d_date] + CometFilter [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [i_item_sk,i_item_id] #5 CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 InputAdapter @@ -52,12 +52,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cr_return_quantity] + CometHashAggregate [i_item_id,sum,cr_return_quantity] CometProject [cr_return_quantity,i_item_id] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_quantity,cr_returned_date_sk,i_item_id,d_date_sk] CometProject [cr_return_quantity,cr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [cr_item_sk,i_item_sk] - CometFilter [cr_item_sk] + CometBroadcastHashJoin [cr_item_sk,cr_return_quantity,cr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [cr_item_sk,cr_return_quantity,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #5 @@ -71,12 +71,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,wr_return_quantity] + CometHashAggregate [i_item_id,sum,wr_return_quantity] CometProject [wr_return_quantity,i_item_id] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [wr_return_quantity,wr_returned_date_sk,i_item_id,d_date_sk] CometProject [wr_return_quantity,wr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [wr_item_sk,i_item_sk] - CometFilter [wr_item_sk] + CometBroadcastHashJoin [wr_item_sk,wr_return_quantity,wr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [wr_item_sk,wr_return_quantity,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt index fe7f7a207..646285a08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt @@ -3,32 +3,32 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] Project [c_customer_id,c_last_name,c_first_name] ColumnarToRow InputAdapter - CometBroadcastHashJoin [cd_demo_sk,sr_cdemo_sk] - CometBroadcastExchange #1 + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] + CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk,ib_income_band_sk] CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk] - CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk,hd_demo_sk,hd_income_band_sk] CometProject [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] CometProject [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] + CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastExchange #2 + CometBroadcastExchange [ca_address_sk] #2 CometProject [ca_address_sk] - CometFilter [ca_city,ca_address_sk] + CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 CometFilter [hd_demo_sk,hd_income_band_sk] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [ib_income_band_sk] #5 CometProject [ib_income_band_sk] - CometFilter [ib_lower_bound,ib_upper_bound,ib_income_band_sk] + CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] CometScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] CometProject [sr_cdemo_sk] - CometFilter [sr_cdemo_sk] + CometFilter [sr_cdemo_sk,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_cdemo_sk,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/simplified.txt index ecae29c78..2313467ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/simplified.txt @@ -6,23 +6,23 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] + CometHashAggregate [r_reason_desc,sum,count,sum,count,sum,count,ws_quantity,wr_refunded_cash,wr_fee] CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - CometBroadcastHashJoin [wr_reason_sk,r_reason_sk] + CometBroadcastHashJoin [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash,r_reason_sk,r_reason_desc] CometProject [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash,d_date_sk] CometProject [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] + CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash,ca_address_sk,ca_state] CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] + CometBroadcastHashJoin [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,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [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_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [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,wp_web_page_sk] 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] + CometBroadcastHashJoin [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,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] + CometBroadcastExchange [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] #2 + CometFilter [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] 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 @@ -30,28 +30,28 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] 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] + CometFilter [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] 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] - CometBroadcastExchange #4 + CometBroadcastExchange [wp_web_page_sk] #4 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #5 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] - CometBroadcastExchange #6 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 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] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk,ca_state] #7 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #8 + CometBroadcastExchange [d_date_sk] #8 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [r_reason_sk] + CometBroadcastExchange [r_reason_sk,r_reason_desc] #9 + CometFilter [r_reason_sk,r_reason_desc] CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/simplified.txt index 4218938c1..a6d1c25fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/simplified.txt @@ -14,13 +14,13 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] + CometHashAggregate [i_category,i_class,spark_grouping_id,sum,ws_net_paid] CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] CometProject [ws_net_paid,i_category,i_class] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,12 +28,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt index cfac83844..9d667265c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt @@ -15,10 +15,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,14 +26,14 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #5 - CometFilter [c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 @@ -46,10 +46,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [cs_bill_customer_sk,d_date] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] + CometFilter [cs_bill_customer_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 @@ -65,10 +65,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ws_bill_customer_sk,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/simplified.txt index b846d25d8..d3af10d19 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/simplified.txt @@ -12,27 +12,27 @@ WholeStageCodegen (16) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #5 @@ -43,20 +43,20 @@ WholeStageCodegen (16) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #7 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -68,20 +68,20 @@ WholeStageCodegen (16) WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #10 + CometBroadcastExchange [t_time_sk] #10 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -93,20 +93,20 @@ WholeStageCodegen (16) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #13 + CometBroadcastExchange [t_time_sk] #13 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -118,20 +118,20 @@ WholeStageCodegen (16) WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #16 + CometBroadcastExchange [t_time_sk] #16 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -143,20 +143,20 @@ WholeStageCodegen (16) WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #19 + CometBroadcastExchange [t_time_sk] #19 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -168,20 +168,20 @@ WholeStageCodegen (16) WholeStageCodegen (12) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #22 + CometBroadcastExchange [t_time_sk] #22 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -193,19 +193,19 @@ WholeStageCodegen (16) WholeStageCodegen (14) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #25 + CometBroadcastExchange [t_time_sk] #25 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/simplified.txt index 507ac8a91..aa858c158 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/simplified.txt @@ -15,17 +15,17 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] + CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum,ss_sales_price] CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_category,i_class,i_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -33,12 +33,12 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/simplified.txt index c54606f6e..5cb600551 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/simplified.txt @@ -9,9 +9,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #1 ReusedSubquery [mergedValue] #1 @@ -24,9 +24,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #2 ReusedSubquery [mergedValue] #2 @@ -39,9 +39,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #3 ReusedSubquery [mergedValue] #3 @@ -54,9 +54,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #4 ReusedSubquery [mergedValue] #4 @@ -69,9 +69,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/simplified.txt index 50c8494fb..44159cc73 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/simplified.txt @@ -7,27 +7,27 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] + CometFilter [t_time_sk,t_hour] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - CometBroadcastExchange #4 + CometBroadcastExchange [wp_web_page_sk] #4 CometProject [wp_web_page_sk] - CometFilter [wp_char_count,wp_web_page_sk] + CometFilter [wp_web_page_sk,wp_char_count] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] InputAdapter BroadcastExchange #5 @@ -38,19 +38,19 @@ WholeStageCodegen (4) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #7 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] + CometFilter [t_time_sk,t_hour] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] ReusedExchange [wp_web_page_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt index b415eb5c4..232c174cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt @@ -9,23 +9,23 @@ WholeStageCodegen (3) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] + CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,cr_net_loss] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status,hd_demo_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,ca_address_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk,d_date_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] - CometBroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] - CometFilter [cc_call_center_sk] + CometBroadcastHashJoin [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] + CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometBroadcastExchange #3 - CometFilter [cr_call_center_sk,cr_returning_customer_sk] + CometBroadcastExchange [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] #3 + CometFilter [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -33,23 +33,23 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 - CometFilter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] #6 + CometFilter [c_customer_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] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk] #7 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #8 - CometFilter [cd_marital_status,cd_education_status,cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 + 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] - CometBroadcastExchange #9 + CometBroadcastExchange [hd_demo_sk] #9 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/simplified.txt index e2f498028..0681a64bd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/simplified.txt @@ -11,8 +11,8 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk,ws_ext_discount_amt] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -20,11 +20,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 @@ -36,15 +36,15 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,ws_ext_discount_amt] + CometHashAggregate [ws_item_sk,sum,count,ws_ext_discount_amt] CometProject [ws_item_sk,ws_ext_discount_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/simplified.txt index 3ec7ac7b6..a8eb231b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/simplified.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_ticket_number,sr_reason_sk] + CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] InputAdapter BroadcastExchange #4 @@ -36,5 +36,5 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] ColumnarToRow InputAdapter CometProject [r_reason_sk] - CometFilter [r_reason_desc,r_reason_sk] + CometFilter [r_reason_sk,r_reason_desc] CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/simplified.txt index 34ddde768..d54c9e0c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/simplified.txt @@ -26,7 +26,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -54,7 +54,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -62,7 +62,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -70,5 +70,5 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [web_site_sk] - CometFilter [web_company_name,web_site_sk] + CometFilter [web_site_sk,web_company_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/simplified.txt index 5b699890c..178c25f66 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/simplified.txt @@ -25,7 +25,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] InputAdapter WholeStageCodegen (7) @@ -40,7 +40,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ws_warehouse_sk,ws_order_number] - CometFilter [ws_order_number,ws_warehouse_sk] + CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -60,7 +60,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [wr_order_number] - CometFilter [wr_order_number] + CometFilter [wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] InputAdapter WholeStageCodegen (15) @@ -82,7 +82,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -90,7 +90,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -98,5 +98,5 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [web_site_sk] - CometFilter [web_company_name,web_site_sk] + CometFilter [web_site_sk,web_company_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/simplified.txt index 614915226..e9d33a7f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/simplified.txt @@ -5,25 +5,25 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/simplified.txt index 24e6dceef..41f3e579f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/simplified.txt @@ -17,7 +17,7 @@ WholeStageCodegen (6) InputAdapter CometHashAggregate [ss_customer_sk,ss_item_sk] CometProject [ss_item_sk,ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -25,11 +25,11 @@ WholeStageCodegen (6) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter WholeStageCodegen (4) @@ -42,7 +42,7 @@ WholeStageCodegen (6) InputAdapter CometHashAggregate [cs_bill_customer_sk,cs_item_sk] CometProject [cs_bill_customer_sk,cs_item_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/simplified.txt index c03b8be9a..3895cdc78 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/simplified.txt @@ -18,12 +18,12 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -31,12 +31,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/simplified.txt index adfe90bab..fc63929bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/simplified.txt @@ -6,27 +6,27 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days , WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,sm_type,cc_name,cs_ship_date_sk,cs_sold_date_sk] + CometHashAggregate [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum,cs_ship_date_sk,cs_sold_date_sk] CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression] - CometBroadcastHashJoin [cs_ship_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name,d_date_sk] CometProject [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name] CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - CometFilter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #3 - CometFilter [sm_ship_mode_sk] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange #4 - CometFilter [cc_call_center_sk] + CometBroadcastExchange [cc_call_center_sk,cc_name] #4 + CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt index 26b5570b7..43bad5d70 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt @@ -15,8 +15,8 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk,sr_customer_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,11 +24,11 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #4 @@ -47,8 +47,8 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -58,12 +58,12 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt index 19243e359..efd4b187d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -60,12 +60,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] + CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt index fe6b3a625..240d88bb7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt @@ -15,23 +15,23 @@ TakeOrderedAndProject [customer_preferred_cust_flag] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -44,23 +44,23 @@ TakeOrderedAndProject [customer_preferred_cust_flag] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -74,13 +74,13 @@ TakeOrderedAndProject [customer_preferred_cust_flag] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -95,13 +95,13 @@ TakeOrderedAndProject [customer_preferred_cust_flag] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt index 4bdb7ae0f..1640fa3b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt @@ -16,10 +16,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt index ef54c7add..cc52bb323 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt @@ -7,16 +7,16 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] + CometBroadcastHashJoin [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status,hd_demo_sk,hd_dep_count] CometProject [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk,d_date_sk] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,s_store_sk] + CometFilter [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,22 +24,22 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 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] - CometBroadcastExchange #7 + CometBroadcastExchange [hd_demo_sk,hd_dep_count] #7 CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt index 72aa4a054..d2ea2d830 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt @@ -20,19 +20,19 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su HashAggregate [quantity,list_price] [sum,count,sum,count] ColumnarToRow InputAdapter - CometUnion + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 @@ -48,7 +48,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -56,7 +56,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -65,7 +65,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 @@ -79,10 +79,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 @@ -90,26 +90,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #9 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #9 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #10 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #11 + CometBroadcastExchange [d_date_sk] #11 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #11 InputAdapter @@ -118,10 +118,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 @@ -132,7 +132,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #4 @@ -154,7 +154,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [cs_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -179,7 +179,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ws_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ws_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt index 73badab8d..cd4817202 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt @@ -11,19 +11,19 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ HashAggregate [quantity,list_price] [sum,count,sum,count] ColumnarToRow InputAdapter - CometUnion + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 @@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -47,7 +47,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] ReusedSubquery [d_week_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] Subquery #2 @@ -55,7 +55,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] InputAdapter BroadcastExchange #3 @@ -64,7 +64,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 @@ -78,10 +78,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -89,26 +89,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #8 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk] #10 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #10 InputAdapter @@ -117,10 +117,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 @@ -154,7 +154,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #16 @@ -162,7 +162,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] ReusedSubquery [d_week_seq] #6 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] Subquery #6 @@ -170,7 +170,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] InputAdapter ReusedExchange [ss_item_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt index b0fd9ebf1..a34cd9c1b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt @@ -8,12 +8,12 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] ColumnarToRow InputAdapter CometProject [cs_sales_price,ca_zip] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,d_date_sk] CometProject [cs_sales_price,cs_sold_date_sk,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_state,ca_zip] CometProject [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -21,15 +21,15 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #4 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4 + CometFilter [ca_address_sk,ca_state,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt index a55c182be..8935abb54 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt @@ -26,7 +26,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] + CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -54,7 +54,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -62,7 +62,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -70,5 +70,5 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [cc_call_center_sk] - CometFilter [cc_county,cc_call_center_sk] + CometFilter [cc_call_center_sk,cc_county] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt index 5c9726a22..4a634b069 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt @@ -8,20 +8,20 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 @@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt index 561ad3d4f..c49bfc6c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt @@ -9,18 +9,18 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag InputAdapter CometExpand [i_item_id,ca_country,ca_state,ca_county] [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,26 +28,26 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk] #5 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #6 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt index 852b61d1c..60caecf91 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt @@ -8,31 +8,31 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_store_sk,s_store_sk,ca_zip,s_zip] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip,s_store_sk,s_zip] CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip] CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_sk] CometProject [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_zip] #5 CometFilter [ca_address_sk,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange #6 - CometFilter [s_zip,s_store_sk] + CometBroadcastExchange [s_store_sk,s_zip] #6 + CometFilter [s_store_sk,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt index 334c6f0ea..535c61622 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt @@ -15,14 +15,14 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [sold_date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [sold_date_sk,sales_price,d_date_sk,d_week_seq,d_day_name] + CometUnion [sold_date_sk,sales_price] CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange #3 - CometFilter [d_date_sk,d_week_seq] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 + CometFilter [d_date_sk,d_week_seq,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #4 @@ -30,7 +30,7 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_week_seq] + CometFilter [d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] InputAdapter BroadcastExchange #5 @@ -46,5 +46,5 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_week_seq] + CometFilter [d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt index dd1edb95f..5bfd9a907 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt @@ -16,10 +16,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt index c6058d18f..2c8892b68 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt @@ -9,27 +9,27 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id,d_date_sk,d_date] CometProject [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_sk,i_item_id] CometProject [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - CometFilter [inv_warehouse_sk,inv_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk,i_item_id] #4 CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange #5 - CometFilter [d_date,d_date_sk] + CometBroadcastExchange [d_date_sk,d_date] #5 + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt index d89167ec8..eb25d6d3e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt @@ -9,12 +9,12 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] InputAdapter CometExpand [i_product_name,i_brand,i_class,i_category] [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category,spark_grouping_id] CometProject [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,15 +22,15 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange #5 + CometBroadcastExchange [w_warehouse_sk] #5 CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt index 1ed007bc7..412f27ffb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt @@ -28,7 +28,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -41,12 +41,12 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date] + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 @@ -54,14 +54,14 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (6) @@ -83,10 +83,10 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 @@ -94,12 +94,12 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange #14 + CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter @@ -109,11 +109,11 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #10 + CometBroadcastExchange [c_customer_sk] #10 CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt index 09a2bf7ce..d784ed47e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [cs_item_sk,item_sk] ColumnarToRow InputAdapter - CometFilter [cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,7 +29,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -42,12 +42,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date] + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 @@ -55,14 +55,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (6) @@ -84,10 +84,10 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 @@ -95,12 +95,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange #14 + CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter @@ -110,11 +110,11 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #10 + CometBroadcastExchange [c_customer_sk] #10 CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter @@ -129,7 +129,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter WholeStageCodegen (10) @@ -163,7 +163,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [ws_item_sk,item_sk] ColumnarToRow InputAdapter - CometFilter [ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt index 8ebd45fd1..30091beb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt @@ -39,7 +39,7 @@ WholeStageCodegen (11) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 @@ -74,7 +74,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -85,7 +85,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #5 @@ -93,26 +93,26 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_birth_country] + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_country,ca_zip] + CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt index 8ebd45fd1..30091beb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt @@ -39,7 +39,7 @@ WholeStageCodegen (11) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 @@ -74,7 +74,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -85,7 +85,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #5 @@ -93,26 +93,26 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_birth_country] + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_country,ca_zip] + CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt index 41efffcb4..6335e50ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt @@ -8,20 +8,20 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_sk,s_store_id,s_store_name] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 @@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #8 + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt index 67f335c66..b98e61704 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt @@ -8,14 +8,14 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - CometBroadcastHashJoin [cs_promo_sk,p_promo_sk] + CometBroadcastHashJoin [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id,p_promo_sk] CometProject [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk,cd_demo_sk] + CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [p_promo_sk] #6 CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt index 74c8a1ca2..4d4c824bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt @@ -9,14 +9,14 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] InputAdapter CometExpand [i_item_id,s_state] [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state,spark_grouping_id] CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,19 +24,19 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [s_state,s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt index a7a3f9537..2136886fe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt @@ -17,7 +17,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #3 @@ -35,7 +35,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #6 @@ -53,7 +53,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #9 @@ -71,7 +71,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #12 @@ -89,7 +89,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #15 @@ -107,5 +107,5 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt index dbdfc8572..ee1f435d5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt @@ -8,20 +8,20 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_store_id,s_store_name] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 @@ -40,10 +40,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -51,23 +51,23 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #8 + CometBroadcastExchange [d_date_sk] #8 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk] #9 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #10 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10 + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange #11 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt index 3dc9ede25..36bb17a16 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt @@ -8,16 +8,16 @@ TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] ColumnarToRow InputAdapter CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt index d5b8077ba..c4e59b49a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt @@ -15,10 +15,10 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_addr_sk,wr_returning_customer_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter @@ -52,10 +52,10 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_addr_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -65,7 +65,7 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] InputAdapter BroadcastExchange #9 @@ -73,5 +73,5 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt index c004db9b7..9040179da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt @@ -19,22 +19,22 @@ WholeStageCodegen (13) ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_county] #5 CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter @@ -48,20 +48,20 @@ WholeStageCodegen (13) ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #9 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter @@ -75,20 +75,20 @@ WholeStageCodegen (13) ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #12 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #13 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter @@ -102,10 +102,10 @@ WholeStageCodegen (13) ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year,d_qoy] #4 @@ -121,10 +121,10 @@ WholeStageCodegen (13) ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #9 @@ -140,10 +140,10 @@ WholeStageCodegen (13) ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk,d_year,d_qoy] #13 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt index 8cc26d8f4..b6ba1fbab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt @@ -11,8 +11,8 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk,cs_ext_discount_amt] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -20,11 +20,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 @@ -38,13 +38,13 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_ext_discount_amt] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt index 84143e015..bf0ef7ae4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt @@ -16,12 +16,12 @@ TakeOrderedAndProject [total_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_manufact_id,i_manufact_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_manufact_id] #6 + CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_manufact_id] #7 CometProject [i_manufact_id] - CometFilter [i_category] + CometFilter [i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] WholeStageCodegen (4) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -56,12 +56,12 @@ TakeOrderedAndProject [total_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [cs_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_manufact_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -76,12 +76,12 @@ TakeOrderedAndProject [total_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt index eefd38343..e1f7f9bd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt index efe0b0b4e..9deb51342 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt index 5bdf60081..58d617430 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt @@ -17,12 +17,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i InputAdapter CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,16 +30,16 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt index c3fcd79f0..efd86d5ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt @@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] InputAdapter CometHashAggregate [i_item_id,i_item_desc,i_current_price] CometProject [i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometBroadcastExchange #2 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,cs_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - CometBroadcastHashJoin [i_item_sk,inv_item_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange #3 + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [cs_item_sk] - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt index cfac83844..9d667265c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt @@ -15,10 +15,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,14 +26,14 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #5 - CometFilter [c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 @@ -46,10 +46,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [cs_bill_customer_sk,d_date] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] + CometFilter [cs_bill_customer_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 @@ -65,10 +65,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ws_bill_customer_sk,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt index ff321c523..d16db5c8b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt @@ -14,12 +14,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,17 +27,17 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_moy] #6 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 @@ -52,12 +52,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -65,11 +65,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk,d_moy] #10 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt index ff321c523..d16db5c8b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt @@ -14,12 +14,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,17 +27,17 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_moy] #6 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 @@ -52,12 +52,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -65,11 +65,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk,d_moy] #10 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt index 0918da575..a412a54d9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt @@ -18,23 +18,23 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -47,23 +47,23 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -77,13 +77,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [cs_bill_customer_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #12 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -98,13 +98,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [cs_bill_customer_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #15 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 @@ -120,13 +120,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #18 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #18 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -141,13 +141,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #21 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #21 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt index 10e0735b4..11c0201a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt @@ -21,14 +21,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [cs_warehouse_sk,cs_item_sk] + CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -39,14 +39,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [w_warehouse_sk] + CometFilter [w_warehouse_sk,w_state] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] InputAdapter BroadcastExchange #6 @@ -54,7 +54,7 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] ColumnarToRow InputAdapter CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] InputAdapter ReusedExchange [d_date_sk,d_date] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt index e31217066..457d441ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject [i_product_name] ColumnarToRow InputAdapter CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact] + CometFilter [i_manufact_id,i_manufact,i_product_name] CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] InputAdapter BroadcastExchange #2 @@ -23,7 +23,7 @@ TakeOrderedAndProject [i_product_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact] + CometHashAggregate [i_manufact,count] CometProject [i_manufact] - CometFilter [i_category,i_color,i_units,i_size,i_manufact] + CometFilter [i_category,i_manufact,i_size,i_color,i_units] CometScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt index e3967a972..d6e3733ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt @@ -8,16 +8,16 @@ TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] ColumnarToRow InputAdapter CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_category_id,i_category] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 CometProject [i_item_sk,i_category_id,i_category] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt index 63945612a..f21d846c0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt @@ -8,16 +8,16 @@ TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed ColumnarToRow InputAdapter CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [d_day_name,ss_store_sk,ss_sales_price,s_store_sk,s_store_id,s_store_name] CometProject [d_day_name,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_day_name] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange #2 - CometFilter [ss_store_sk] + CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 CometProject [s_store_sk,s_store_id,s_store_name] - CometFilter [s_gmt_offset,s_store_sk] + CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt index 35a3e9efa..12c8e0af9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt @@ -35,7 +35,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_store_sk,ss_addr_sk] + CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count] InputAdapter @@ -45,7 +45,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_store_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] InputAdapter WholeStageCodegen (10) @@ -75,7 +75,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_product_name] InputAdapter ReusedExchange [i_item_sk,i_product_name] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt index c376c4fcd..f60fdb18a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt @@ -11,14 +11,14 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ca_city,ca_zip,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_sales_price,ca_city,ca_zip] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip,d_date_sk] CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_city,ca_zip] CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - CometFilter [ws_bill_customer_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,20 +26,20 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #4 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4 + CometFilter [ca_address_sk,ca_city,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #7 @@ -47,5 +47,5 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [i_item_id] - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt index 718557c10..9ce3a98eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt @@ -12,14 +12,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu ColumnarToRow InputAdapter CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_address_sk,ca_city] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] + CometFilter [s_store_sk,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk,ca_city] #6 CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter @@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt index 66eb5a4cb..1db8e7528 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt @@ -25,27 +25,27 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra ColumnarToRow InputAdapter CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt index 30f9822ab..23e083535 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt @@ -7,14 +7,14 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [ss_quantity] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] CometProject [ss_quantity,ss_sold_date_sk] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + CometBroadcastHashJoin [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] CometProject [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,s_store_sk] + CometFilter [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,19 +22,19 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #4 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] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_state] #5 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt index d1480ef10..bd3b21cdb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt @@ -29,12 +29,12 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] 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 + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #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] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_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 @@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] 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] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] @@ -73,16 +73,16 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] 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 #9 + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 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] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_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] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] ReusedExchange [d_date_sk] #6 WholeStageCodegen (15) @@ -107,15 +107,15 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] 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 #12 + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 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] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_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] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt index 3dc3cca6c..ceb3e9213 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt @@ -17,12 +17,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,s_store_sk] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,18 +30,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (4) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] @@ -52,21 +52,21 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #7 - CometFilter [cp_catalog_page_sk] + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (6) 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] @@ -77,23 +77,23 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] 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] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_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 + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #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] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #10 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_site_id] #10 + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt index cd8a8ff7f..aa3f7f2cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt @@ -8,17 +8,17 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s ColumnarToRow InputAdapter CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk,s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometProject [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] - CometFilter [ss_ticket_number,ss_item_sk,ss_customer_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [sr_ticket_number,sr_item_sk,sr_customer_sk] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] #2 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,15 +26,15 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #4 + CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometFilter [d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt index 9ab7f18e7..f19312ca8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt @@ -32,8 +32,8 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -41,11 +41,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_date] #6 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter WholeStageCodegen (10) @@ -69,8 +69,8 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt index d0b9da882..d7661b7be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt @@ -8,16 +8,16 @@ TakeOrderedAndProject [d_year,ext_price,brand_id,brand] ColumnarToRow InputAdapter CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt index a04777412..b90cb42d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt @@ -17,16 +17,16 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [i_manufact_id,ss_sales_price,d_qoy] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [i_item_sk,i_manufact_id] - CometFilter [i_category,i_class,i_brand,i_item_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -34,12 +34,12 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [d_date_sk,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_qoy] #5 CometProject [d_date_sk,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt index 32e9a92a7..a8df48770 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt @@ -26,14 +26,14 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [c_customer_sk,c_current_addr_sk] CometProject [c_customer_sk,c_current_addr_sk] - CometBroadcastHashJoin [customer_sk,c_customer_sk] + CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk] CometProject [customer_sk] - CometBroadcastHashJoin [sold_date_sk,d_date_sk] + CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] CometProject [sold_date_sk,customer_sk] - CometBroadcastHashJoin [item_sk,i_item_sk] - CometUnion + CometBroadcastHashJoin [sold_date_sk,customer_sk,item_sk,i_item_sk] + CometUnion [sold_date_sk,customer_sk,item_sk] CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -41,21 +41,21 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #5 + CometBroadcastExchange [i_item_sk] #5 CometProject [i_item_sk] - CometFilter [i_category,i_class,i_item_sk] + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #7 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -71,7 +71,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [(d_month_seq + 1)] #3 ReusedSubquery [(d_month_seq + 3)] #4 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] @@ -85,7 +85,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [(d_month_seq + 1)] CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] Subquery #4 WholeStageCodegen (2) @@ -97,7 +97,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [(d_month_seq + 3)] CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] InputAdapter BroadcastExchange #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt index a404b9b37..2461ab830 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt @@ -8,16 +8,16 @@ TakeOrderedAndProject [ext_price,brand_id,brand] ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt index a5cb59e57..c4ce35e15 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt @@ -16,12 +16,12 @@ TakeOrderedAndProject [total_sales,i_item_id] ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_item_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_item_id,i_item_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_item_id] #7 CometProject [i_item_id] - CometFilter [i_color] + CometFilter [i_item_id,i_color] CometScan parquet spark_catalog.default.item [i_item_id,i_color] WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -56,12 +56,12 @@ TakeOrderedAndProject [total_sales,i_item_id] ColumnarToRow InputAdapter CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -76,12 +76,12 @@ TakeOrderedAndProject [total_sales,i_item_id] ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt index ee91816c9..b64c2efdc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt @@ -25,27 +25,27 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ ColumnarToRow InputAdapter CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [cs_item_sk,cs_call_center_sk] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt index 959d98ba9..0f5a783b9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt @@ -13,10 +13,10 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,12 +24,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date] #3 CometProject [d_date] - CometFilter [d_week_seq] + CometFilter [d_date,d_week_seq] ReusedSubquery [d_week_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] Subquery #2 @@ -37,15 +37,15 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_date] + CometFilter [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk,i_item_id] #4 CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 InputAdapter @@ -60,10 +60,10 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #4 @@ -80,10 +80,10 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt index 55cb85ddb..58bae46d1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt @@ -14,18 +14,18 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_week_seq,d_day_name] + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [d_date_sk,d_week_seq] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 + CometFilter [d_date_sk,d_week_seq,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt index a7c65f50a..febbfd32f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt @@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #2 - CometFilter [c_current_addr_sk,c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #3 - CometFilter [ss_customer_sk,ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] Subquery #2 @@ -43,11 +43,11 @@ TakeOrderedAndProject [cnt,state] InputAdapter CometHashAggregate [d_month_seq] CometProject [d_month_seq] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 @@ -58,7 +58,7 @@ TakeOrderedAndProject [cnt,state] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow InputAdapter - CometFilter [i_current_price,i_category,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #8 @@ -71,5 +71,5 @@ TakeOrderedAndProject [cnt,state] HashAggregate [i_category,i_current_price] [sum,count,sum,count] ColumnarToRow InputAdapter - CometFilter [i_category] + CometFilter [i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt index cf226bb14..51025cf34 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt @@ -16,12 +16,12 @@ TakeOrderedAndProject [i_item_id,total_sales] ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [i_item_id,total_sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_item_id,i_item_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_item_id] #7 CometProject [i_item_id] - CometFilter [i_category] + CometFilter [i_item_id,i_category] CometScan parquet spark_catalog.default.item [i_item_id,i_category] WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -56,12 +56,12 @@ TakeOrderedAndProject [i_item_id,total_sales] ColumnarToRow InputAdapter CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -76,12 +76,12 @@ TakeOrderedAndProject [i_item_id,total_sales] ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt index db13f94bd..8d39d9e4e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt @@ -9,18 +9,18 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,p_promo_sk] CometProject [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,30 +28,30 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometProject [s_store_sk] - CometFilter [s_gmt_offset,s_store_sk] + CometFilter [s_store_sk,s_gmt_offset] CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - CometBroadcastExchange #4 + CometBroadcastExchange [p_promo_sk] #4 CometProject [p_promo_sk] - CometFilter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk] #7 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #8 + CometBroadcastExchange [i_item_sk] #8 CometProject [i_item_sk] - CometFilter [i_category,i_item_sk] + CometFilter [i_item_sk,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_category] InputAdapter BroadcastExchange #9 @@ -64,16 +64,16 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_customer_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [s_store_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt index b25008e64..b9e35e282 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt @@ -8,25 +8,25 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ColumnarToRow InputAdapter CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression] - CometBroadcastHashJoin [ws_ship_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name,d_date_sk] CometProject [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] - CometBroadcastHashJoin [ws_web_site_sk,web_site_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name] CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - CometFilter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #3 - CometFilter [sm_ship_mode_sk] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange #4 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_name] #4 + CometFilter [web_site_sk,web_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt index a8eb9d7c6..e84d3c2a0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt @@ -17,16 +17,16 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] ColumnarToRow InputAdapter CometProject [i_manager_id,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [i_item_sk,i_manager_id] - CometFilter [i_category,i_class,i_brand,i_item_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -34,12 +34,12 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt index d972e0082..3b5e4f19e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt @@ -54,19 +54,19 @@ WholeStageCodegen (52) 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] - 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] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #4 + CometFilter [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] 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] + CometFilter [d_date_sk,d_year] 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] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (8) @@ -89,7 +89,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -100,7 +100,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] 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 @@ -116,14 +116,14 @@ WholeStageCodegen (52) WholeStageCodegen (11) 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] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter BroadcastExchange #11 WholeStageCodegen (12) ColumnarToRow InputAdapter - CometFilter [d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #11 @@ -157,7 +157,7 @@ WholeStageCodegen (52) WholeStageCodegen (19) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] InputAdapter ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 @@ -176,7 +176,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] - CometFilter [i_current_price,i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter WholeStageCodegen (50) @@ -227,19 +227,19 @@ WholeStageCodegen (52) 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] - 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] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #20 + CometFilter [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] 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] + CometFilter [d_date_sk,d_year] 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] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (33) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt index 1e34ce22f..84d23af42 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes BroadcastHashJoin [s_store_sk,ss_store_sk] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #1 @@ -22,8 +22,8 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -31,18 +31,18 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] InputAdapter BroadcastExchange #6 @@ -61,8 +61,8 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt index 7d6c85af3..880c38bbf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt @@ -16,35 +16,35 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat ColumnarToRow InputAdapter CometProject [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk] CometProject [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk] CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy] CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - CometFilter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 + CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastExchange #5 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [t_time_sk] #6 CometProject [t_time_sk] - CometFilter [t_time,t_time_sk] + CometFilter [t_time_sk,t_time] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] - CometBroadcastExchange #7 + CometBroadcastExchange [sm_ship_mode_sk] #7 CometProject [sm_ship_mode_sk] - CometFilter [sm_carrier,sm_ship_mode_sk] + CometFilter [sm_ship_mode_sk,sm_carrier] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] WholeStageCodegen (4) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] @@ -55,14 +55,14 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat ColumnarToRow InputAdapter CometProject [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk] CometProject [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk] CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy] CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - CometFilter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt index 0386feff9..46d9cf4ef 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt @@ -20,12 +20,12 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ InputAdapter CometExpand [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] CometProject [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -33,15 +33,15 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ ColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt index 1ecde47e8..78e1b1cce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt @@ -12,14 +12,14 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ ColumnarToRow InputAdapter CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_address_sk,ca_city] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] + CometFilter [s_store_sk,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk,ca_city] #6 CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter @@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt index de2d5eeda..c9cc4959b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,11 +27,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -39,7 +39,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -49,7 +49,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/simplified.txt index 0429448f5..a27daca36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/simplified.txt @@ -8,14 +8,14 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] + CometBroadcastHashJoin [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,p_promo_sk] CometProject [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_item_sk,ss_promo_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [p_promo_sk] #6 CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt index c962534aa..4a6dffc08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #6 @@ -60,13 +60,13 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt index c732d903f..a4be53684 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt @@ -11,17 +11,17 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] - CometBroadcastHashJoin [time_sk,t_time_sk] + CometBroadcastHashJoin [i_brand_id,i_brand,ext_price,time_sk,t_time_sk,t_hour,t_minute] CometProject [i_brand_id,i_brand,ext_price,time_sk] - CometBroadcastHashJoin [i_item_sk,sold_item_sk] - CometBroadcastExchange #3 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometUnion + CometUnion [ext_price,sold_item_sk,time_sk] CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_sold_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -29,25 +29,25 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_item_sk,cs_sold_time_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 CometProject [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_sold_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #6 + CometBroadcastExchange [t_time_sk,t_hour,t_minute] #6 CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_meal_time,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt index bea1fd4a1..6cba2d0e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt @@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #4 - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange #7 + CometBroadcastExchange [cd_demo_sk] #7 CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange #8 + CometBroadcastExchange [hd_demo_sk] #8 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #10 - CometFilter [d_week_seq,d_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #11 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #12 @@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt index 060c3e153..c91c4cf47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt index ea253444c..a7aa864ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt @@ -14,23 +14,23 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -43,23 +43,23 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -73,13 +73,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -94,13 +94,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt index 34866bc37..27a4dcb67 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt @@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #7 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] @@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] @@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_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 WholeStageCodegen (38) @@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange #18 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (23) @@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt index 6f8c28004..cfcf6a99a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt @@ -7,32 +7,32 @@ TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_ HashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] [count,sum,count,sum] ColumnarToRow InputAdapter - CometUnion + CometUnion [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] CometProject [ss_store_sk,ss_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_category] #2 + CometFilter [i_item_sk,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange #3 - CometFilter [d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometProject [ws_ship_customer_sk,ws_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_ship_customer_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category] + CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedExchange [i_item_sk,i_category] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #3 CometProject [cs_ship_addr_sk,cs_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_ship_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category] + CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedExchange [i_item_sk,i_category] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt index 2989455d9..91786374e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt @@ -19,10 +19,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,13 +30,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter @@ -50,10 +50,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -72,7 +72,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -84,7 +84,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -99,14 +99,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #12 + CometBroadcastExchange [wp_web_page_sk] #12 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter @@ -120,10 +120,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt index 280687e30..13a8aecff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -45,7 +45,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -83,7 +83,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -109,7 +109,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt index 5fcbcfed3..f2883e450 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt @@ -10,12 +10,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, ColumnarToRow InputAdapter CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk,s_city] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,24 +23,24 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk,s_city] #4 CometProject [s_store_sk,s_city] - CometFilter [s_number_employees,s_store_sk] + CometFilter [s_store_sk,s_number_employees,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt index adf555417..637f3b7ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt @@ -10,10 +10,10 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] ColumnarToRow InputAdapter CometProject [ss_net_profit,s_store_name,s_zip] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip] CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -21,14 +21,14 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 - CometFilter [s_store_sk,s_zip] + CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4 + 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 #5 @@ -55,12 +55,12 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_zip] + CometHashAggregate [ca_zip,count] CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] + CometFilter [ca_address_sk,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange #9 + CometBroadcastExchange [c_current_addr_sk] #9 CometProject [c_current_addr_sk] - CometFilter [c_preferred_cust_flag,c_current_addr_sk] + CometFilter [c_current_addr_sk,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt index 7e257bdc6..cdc8dc486 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -40,7 +40,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -51,7 +51,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -60,7 +60,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #7 @@ -68,7 +68,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [i_item_sk] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #8 @@ -76,7 +76,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [p_promo_sk] - CometFilter [p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (20) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] @@ -102,7 +102,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -114,7 +114,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -123,7 +123,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cp_catalog_page_sk] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter ReusedExchange [i_item_sk] #7 @@ -153,7 +153,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (21) ColumnarToRow InputAdapter - CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -165,7 +165,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -174,7 +174,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (26) ColumnarToRow InputAdapter - CometFilter [web_site_sk] + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] InputAdapter ReusedExchange [i_item_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt index 3daee59f8..991c00de9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt @@ -15,10 +15,10 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st ColumnarToRow InputAdapter CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_addr_sk,cr_returning_customer_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter @@ -52,10 +52,10 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st ColumnarToRow InputAdapter CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_addr_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -65,12 +65,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] InputAdapter BroadcastExchange #9 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt index a4d96e6f1..71a1c8e17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt @@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] InputAdapter CometHashAggregate [i_item_id,i_item_desc,i_current_price] CometProject [i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometBroadcastExchange #2 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,ss_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - CometBroadcastHashJoin [i_item_sk,inv_item_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange #3 + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [ss_item_sk] - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt index 7b7e907ee..8cd3f3f45 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt @@ -12,10 +12,10 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter CometProject [sr_return_quantity,i_item_id] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [sr_return_quantity,sr_returned_date_sk,i_item_id,d_date_sk] CometProject [sr_return_quantity,sr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [sr_item_sk,i_item_sk] - CometFilter [sr_item_sk] + CometBroadcastHashJoin [sr_item_sk,sr_return_quantity,sr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [sr_item_sk,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,24 +23,24 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date] #3 CometProject [d_date] - CometBroadcastHashJoin [d_week_seq,d_week_seq] + CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_week_seq] #4 CometProject [d_week_seq] - CometFilter [d_date] + CometFilter [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [i_item_sk,i_item_id] #5 CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 InputAdapter @@ -54,10 +54,10 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter CometProject [cr_return_quantity,i_item_id] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_quantity,cr_returned_date_sk,i_item_id,d_date_sk] CometProject [cr_return_quantity,cr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [cr_item_sk,i_item_sk] - CometFilter [cr_item_sk] + CometBroadcastHashJoin [cr_item_sk,cr_return_quantity,cr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [cr_item_sk,cr_return_quantity,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #5 @@ -73,10 +73,10 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter CometProject [wr_return_quantity,i_item_id] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [wr_return_quantity,wr_returned_date_sk,i_item_id,d_date_sk] CometProject [wr_return_quantity,wr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [wr_item_sk,i_item_sk] - CometFilter [wr_item_sk] + CometBroadcastHashJoin [wr_item_sk,wr_return_quantity,wr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [wr_item_sk,wr_return_quantity,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt index fe7f7a207..646285a08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt @@ -3,32 +3,32 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] Project [c_customer_id,c_last_name,c_first_name] ColumnarToRow InputAdapter - CometBroadcastHashJoin [cd_demo_sk,sr_cdemo_sk] - CometBroadcastExchange #1 + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] + CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk,ib_income_band_sk] CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk] - CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk,hd_demo_sk,hd_income_band_sk] CometProject [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] CometProject [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] + CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastExchange #2 + CometBroadcastExchange [ca_address_sk] #2 CometProject [ca_address_sk] - CometFilter [ca_city,ca_address_sk] + CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 CometFilter [hd_demo_sk,hd_income_band_sk] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [ib_income_band_sk] #5 CometProject [ib_income_band_sk] - CometFilter [ib_lower_bound,ib_upper_bound,ib_income_band_sk] + CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] CometScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] CometProject [sr_cdemo_sk] - CometFilter [sr_cdemo_sk] + CometFilter [sr_cdemo_sk,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_cdemo_sk,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt index ad94ef82c..0d350bd59 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt @@ -8,21 +8,21 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu ColumnarToRow InputAdapter CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - CometBroadcastHashJoin [wr_reason_sk,r_reason_sk] + CometBroadcastHashJoin [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash,r_reason_sk,r_reason_desc] CometProject [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash,d_date_sk] CometProject [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] + CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash,ca_address_sk,ca_state] CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] + CometBroadcastHashJoin [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,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [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_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [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,wp_web_page_sk] 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] + CometBroadcastHashJoin [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,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] + CometBroadcastExchange [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] #2 + CometFilter [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] 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 @@ -30,28 +30,28 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] 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] + CometFilter [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] 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] - CometBroadcastExchange #4 + CometBroadcastExchange [wp_web_page_sk] #4 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #5 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] - CometBroadcastExchange #6 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 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] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk,ca_state] #7 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #8 + CometBroadcastExchange [d_date_sk] #8 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [r_reason_sk] + CometBroadcastExchange [r_reason_sk,r_reason_desc] #9 + CometFilter [r_reason_sk,r_reason_desc] CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt index ea5dce5b6..e4d028f2f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt @@ -17,10 +17,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl InputAdapter CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] CometProject [ws_net_paid,i_category,i_class] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,12 +28,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt index cfac83844..9d667265c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt @@ -15,10 +15,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,14 +26,14 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #5 - CometFilter [c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 @@ -46,10 +46,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [cs_bill_customer_sk,d_date] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] + CometFilter [cs_bill_customer_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 @@ -65,10 +65,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ws_bill_customer_sk,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt index b846d25d8..d3af10d19 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt @@ -12,27 +12,27 @@ WholeStageCodegen (16) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #5 @@ -43,20 +43,20 @@ WholeStageCodegen (16) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #7 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -68,20 +68,20 @@ WholeStageCodegen (16) WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #10 + CometBroadcastExchange [t_time_sk] #10 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -93,20 +93,20 @@ WholeStageCodegen (16) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #13 + CometBroadcastExchange [t_time_sk] #13 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -118,20 +118,20 @@ WholeStageCodegen (16) WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #16 + CometBroadcastExchange [t_time_sk] #16 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -143,20 +143,20 @@ WholeStageCodegen (16) WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #19 + CometBroadcastExchange [t_time_sk] #19 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -168,20 +168,20 @@ WholeStageCodegen (16) WholeStageCodegen (12) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #22 + CometBroadcastExchange [t_time_sk] #22 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -193,19 +193,19 @@ WholeStageCodegen (16) WholeStageCodegen (14) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #25 + CometBroadcastExchange [t_time_sk] #25 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt index 52e066fe7..2bfff90d3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt @@ -17,15 +17,15 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla ColumnarToRow InputAdapter CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_category,i_class,i_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -33,12 +33,12 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt index fdd3bd293..581739cc6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt @@ -11,7 +11,7 @@ WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #1 ReusedSubquery [mergedValue] #1 @@ -26,7 +26,7 @@ WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #2 ReusedSubquery [mergedValue] #2 @@ -41,7 +41,7 @@ WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #3 ReusedSubquery [mergedValue] #3 @@ -56,7 +56,7 @@ WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #4 ReusedSubquery [mergedValue] #4 @@ -71,7 +71,7 @@ WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt index 50c8494fb..44159cc73 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt @@ -7,27 +7,27 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] + CometFilter [t_time_sk,t_hour] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - CometBroadcastExchange #4 + CometBroadcastExchange [wp_web_page_sk] #4 CometProject [wp_web_page_sk] - CometFilter [wp_char_count,wp_web_page_sk] + CometFilter [wp_web_page_sk,wp_char_count] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] InputAdapter BroadcastExchange #5 @@ -38,19 +38,19 @@ WholeStageCodegen (4) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #7 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] + CometFilter [t_time_sk,t_hour] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] ReusedExchange [wp_web_page_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt index 7e9b7b8ba..84f1190d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt @@ -11,21 +11,21 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status,hd_demo_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,ca_address_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk,d_date_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] - CometBroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] - CometFilter [cc_call_center_sk] + CometBroadcastHashJoin [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] + CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometBroadcastExchange #3 - CometFilter [cr_call_center_sk,cr_returning_customer_sk] + CometBroadcastExchange [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] #3 + CometFilter [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -33,23 +33,23 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 - CometFilter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] #6 + CometFilter [c_customer_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] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk] #7 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #8 - CometFilter [cd_marital_status,cd_education_status,cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 + 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] - CometBroadcastExchange #9 + CometBroadcastExchange [hd_demo_sk] #9 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt index bcc1ef1b5..aa24a049b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt @@ -11,8 +11,8 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk,ws_ext_discount_amt] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -20,11 +20,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 @@ -38,13 +38,13 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_ext_discount_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt index 3ec7ac7b6..a8eb231b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_ticket_number,sr_reason_sk] + CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] InputAdapter BroadcastExchange #4 @@ -36,5 +36,5 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] ColumnarToRow InputAdapter CometProject [r_reason_sk] - CometFilter [r_reason_desc,r_reason_sk] + CometFilter [r_reason_sk,r_reason_desc] CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt index 34ddde768..d54c9e0c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt @@ -26,7 +26,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -54,7 +54,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -62,7 +62,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -70,5 +70,5 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [web_site_sk] - CometFilter [web_company_name,web_site_sk] + CometFilter [web_site_sk,web_company_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt index 5b699890c..178c25f66 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt @@ -25,7 +25,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] InputAdapter WholeStageCodegen (7) @@ -40,7 +40,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ws_warehouse_sk,ws_order_number] - CometFilter [ws_order_number,ws_warehouse_sk] + CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -60,7 +60,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [wr_order_number] - CometFilter [wr_order_number] + CometFilter [wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] InputAdapter WholeStageCodegen (15) @@ -82,7 +82,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -90,7 +90,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -98,5 +98,5 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [web_site_sk] - CometFilter [web_company_name,web_site_sk] + CometFilter [web_site_sk,web_company_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt index 614915226..e9d33a7f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt @@ -5,25 +5,25 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt index 24e6dceef..41f3e579f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt @@ -17,7 +17,7 @@ WholeStageCodegen (6) InputAdapter CometHashAggregate [ss_customer_sk,ss_item_sk] CometProject [ss_item_sk,ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -25,11 +25,11 @@ WholeStageCodegen (6) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter WholeStageCodegen (4) @@ -42,7 +42,7 @@ WholeStageCodegen (6) InputAdapter CometHashAggregate [cs_bill_customer_sk,cs_item_sk] CometProject [cs_bill_customer_sk,cs_item_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt index 84d277886..a92d99be4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt @@ -20,10 +20,10 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -31,12 +31,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt index da4ef22e0..25ff3a2fe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt @@ -8,25 +8,25 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days , ColumnarToRow InputAdapter CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression] - CometBroadcastHashJoin [cs_ship_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name,d_date_sk] CometProject [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name] CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - CometFilter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #3 - CometFilter [sm_ship_mode_sk] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange #4 - CometFilter [cc_call_center_sk] + CometBroadcastExchange [cc_call_center_sk,cc_name] #4 + CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt index 000160518..238f9b702 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt @@ -13,10 +13,10 @@ TakeOrderedAndProject [c_customer_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] + CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk,sr_customer_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,11 +24,11 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #4 @@ -45,10 +45,10 @@ TakeOrderedAndProject [c_customer_id] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] + CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -58,12 +58,12 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt index 19243e359..efd4b187d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -60,12 +60,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] + CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt index dd1a52206..127a5a291 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt @@ -13,25 +13,25 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -42,25 +42,25 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -72,15 +72,15 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -93,15 +93,15 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt index 905a35c8b..4c2d3f3d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt index 4de403664..98d995db0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt @@ -5,18 +5,18 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + CometHashAggregate [sum,count,sum,count,sum,count,sum,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] + CometBroadcastHashJoin [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status,hd_demo_sk,hd_dep_count] CometProject [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk,d_date_sk] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,s_store_sk] + CometFilter [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,22 +24,22 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 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] - CometBroadcastExchange #7 + CometBroadcastExchange [hd_demo_sk,hd_dep_count] #7 CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt index e3dfa631b..2257d398f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt @@ -19,20 +19,20 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 @@ -48,7 +48,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -56,7 +56,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -65,7 +65,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 @@ -79,10 +79,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 @@ -90,26 +90,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #9 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #9 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #10 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #11 + CometBroadcastExchange [d_date_sk] #11 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #11 InputAdapter @@ -118,10 +118,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 @@ -132,7 +132,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #4 @@ -154,7 +154,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [cs_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -179,7 +179,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ws_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ws_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt index 799f74a36..aa49638c7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt @@ -10,20 +10,20 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 @@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -47,13 +47,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #2 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 @@ -77,10 +77,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -88,26 +88,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #8 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk] #10 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #10 InputAdapter @@ -116,10 +116,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 @@ -153,7 +153,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #16 @@ -161,13 +161,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #6 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt index a03346372..b876fe4de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt @@ -6,14 +6,14 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_zip,cs_sales_price] + CometHashAggregate [ca_zip,sum,cs_sales_price] CometProject [cs_sales_price,ca_zip] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,d_date_sk] CometProject [cs_sales_price,cs_sold_date_sk,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_state,ca_zip] CometProject [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -21,15 +21,15 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #4 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4 + CometFilter [ca_address_sk,ca_state,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt index a55c182be..8935abb54 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt @@ -26,7 +26,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] + CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -54,7 +54,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -62,7 +62,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -70,5 +70,5 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [cc_call_center_sk] - CometFilter [cc_county,cc_call_center_sk] + CometFilter [cc_call_center_sk,cc_county] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt index 77aba376e..ae6bab279 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt @@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] + CometHashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,ss_quantity,sr_return_quantity,cs_quantity] CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 @@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt index 3d101857b..2519d58a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt @@ -6,21 +6,21 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] CometExpand [i_item_id,ca_country,ca_state,ca_county] [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,26 +28,26 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk] #5 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #6 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt index 18a69bcb4..163a31f47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt @@ -6,33 +6,33 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] + CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_store_sk,s_store_sk,ca_zip,s_zip] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip,s_store_sk,s_zip] CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip] CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_sk] CometProject [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_zip] #5 CometFilter [ca_address_sk,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange #6 - CometFilter [s_zip,s_store_sk] + CometBroadcastExchange [s_store_sk,s_zip] #6 + CometFilter [s_store_sk,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt index c7999d981..5d0658192 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt @@ -13,16 +13,16 @@ WholeStageCodegen (7) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_week_seq,d_day_name,sales_price] + CometHashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum,d_day_name,sales_price] CometProject [sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [sold_date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [sold_date_sk,sales_price,d_date_sk,d_week_seq,d_day_name] + CometUnion [sold_date_sk,sales_price] CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange #3 - CometFilter [d_date_sk,d_week_seq] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 + CometFilter [d_date_sk,d_week_seq,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #4 @@ -30,7 +30,7 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_week_seq] + CometFilter [d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] InputAdapter BroadcastExchange #5 @@ -46,5 +46,5 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_week_seq] + CometFilter [d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt index d805e3868..08088a386 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt index 52bd7a85e..bd81a3a7b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt @@ -7,29 +7,29 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,i_item_id,sum,sum,d_date,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id,d_date_sk,d_date] CometProject [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_sk,i_item_id] CometProject [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - CometFilter [inv_warehouse_sk,inv_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk,i_item_id] #4 CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange #5 - CometFilter [d_date,d_date_sk] + CometBroadcastExchange [d_date_sk,d_date] #5 + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt index bda583c17..7d36dc400 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt @@ -6,15 +6,15 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count,inv_quantity_on_hand] CometExpand [i_product_name,i_brand,i_class,i_category] [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category,spark_grouping_id] CometProject [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,15 +22,15 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange #5 + CometBroadcastExchange [w_warehouse_sk] #5 CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt index f818fd25f..e065a0a87 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt @@ -28,7 +28,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -41,12 +41,12 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date] + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 @@ -54,14 +54,14 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (6) @@ -81,12 +81,12 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 @@ -94,12 +94,12 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange #14 + CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter @@ -107,13 +107,13 @@ WholeStageCodegen (18) WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #10 + CometBroadcastExchange [c_customer_sk] #10 CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt index 54ee3dbde..2862f64cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [cs_item_sk,item_sk] ColumnarToRow InputAdapter - CometFilter [cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,7 +29,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -42,12 +42,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date] + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 @@ -55,14 +55,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (6) @@ -82,12 +82,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 @@ -95,12 +95,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange #14 + CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter @@ -108,13 +108,13 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #10 + CometBroadcastExchange [c_customer_sk] #10 CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter @@ -129,7 +129,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter WholeStageCodegen (10) @@ -163,7 +163,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [ws_item_sk,item_sk] ColumnarToRow InputAdapter - CometFilter [ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt index 8ebd45fd1..30091beb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt @@ -39,7 +39,7 @@ WholeStageCodegen (11) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 @@ -74,7 +74,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -85,7 +85,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #5 @@ -93,26 +93,26 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_birth_country] + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_country,ca_zip] + CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt index 8ebd45fd1..30091beb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt @@ -39,7 +39,7 @@ WholeStageCodegen (11) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 @@ -74,7 +74,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -85,7 +85,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #5 @@ -93,26 +93,26 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_birth_country] + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_country,ca_zip] + CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt index 74a5e8777..1f4faaac5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt @@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] + CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,ss_net_profit,sr_net_loss,cs_net_profit] CometProject [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_sk,s_store_id,s_store_name] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 @@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #8 + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt index cba306f68..a39fbbb79 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt @@ -6,16 +6,16 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - CometBroadcastHashJoin [cs_promo_sk,p_promo_sk] + CometBroadcastHashJoin [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id,p_promo_sk] CometProject [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk,cd_demo_sk] + CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [p_promo_sk] #6 CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt index 7fcbe967a..9c4dd659a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt @@ -6,17 +6,17 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + CometHashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] CometExpand [i_item_id,s_state] [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state,spark_grouping_id] CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,19 +24,19 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [s_state,s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt index 47a4fe2c7..80c886708 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt @@ -15,9 +15,9 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #3 @@ -33,9 +33,9 @@ WholeStageCodegen (18) WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #6 @@ -51,9 +51,9 @@ WholeStageCodegen (18) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #9 @@ -69,9 +69,9 @@ WholeStageCodegen (18) WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #12 @@ -87,9 +87,9 @@ WholeStageCodegen (18) WholeStageCodegen (12) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #15 @@ -105,7 +105,7 @@ WholeStageCodegen (18) WholeStageCodegen (15) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt index 2bcef6168..88b4ecd1d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt @@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] + CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,ss_quantity,sr_return_quantity,cs_quantity] CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_store_id,s_store_name] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 @@ -40,10 +40,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -51,23 +51,23 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #8 + CometBroadcastExchange [d_date_sk] #8 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk] #9 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #10 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10 + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange #11 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt index cc9c4edf3..bccda58a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] + CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt index 50472502e..60cad82dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt @@ -13,12 +13,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] + CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_addr_sk,wr_returning_customer_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter @@ -50,12 +50,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] + CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_addr_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -65,7 +65,7 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] InputAdapter BroadcastExchange #9 @@ -73,5 +73,5 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt index a94a8a94d..a48c7ad26 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt @@ -17,24 +17,24 @@ WholeStageCodegen (13) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_county] #5 CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter @@ -46,22 +46,22 @@ WholeStageCodegen (13) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #9 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter @@ -73,22 +73,22 @@ WholeStageCodegen (13) WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #12 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #13 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter @@ -100,12 +100,12 @@ WholeStageCodegen (13) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year,d_qoy] #4 @@ -119,12 +119,12 @@ WholeStageCodegen (13) WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #9 @@ -138,12 +138,12 @@ WholeStageCodegen (13) WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk,d_year,d_qoy] #13 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt index b8df1e929..c3bfdfd77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt @@ -11,8 +11,8 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk,cs_ext_discount_amt] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -20,11 +20,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 @@ -36,15 +36,15 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cs_item_sk,cs_ext_discount_amt] + CometHashAggregate [cs_item_sk,sum,count,cs_ext_discount_amt] CometProject [cs_item_sk,cs_ext_discount_amt] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt index af2b7cb5d..7a3c2c153 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt @@ -14,14 +14,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,ss_ext_sales_price] + CometHashAggregate [i_manufact_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_manufact_id,i_manufact_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_manufact_id] #6 + CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_manufact_id] #7 CometProject [i_manufact_id] - CometFilter [i_category] + CometFilter [i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] WholeStageCodegen (4) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -54,14 +54,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,cs_ext_sales_price] + CometHashAggregate [i_manufact_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_manufact_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -74,14 +74,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,ws_ext_sales_price] + CometHashAggregate [i_manufact_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt index eefd38343..e1f7f9bd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt index efe0b0b4e..9deb51342 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt index 574a20fc0..4ad9b501b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt @@ -14,15 +14,15 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [i_category,i_class,spark_grouping_id,sum,sum,ss_net_profit,ss_ext_sales_price] CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,16 +30,16 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt index c3fcd79f0..efd86d5ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt @@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] InputAdapter CometHashAggregate [i_item_id,i_item_desc,i_current_price] CometProject [i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometBroadcastExchange #2 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,cs_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - CometBroadcastHashJoin [i_item_sk,inv_item_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange #3 + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [cs_item_sk] - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt index cfac83844..9d667265c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt @@ -15,10 +15,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,14 +26,14 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #5 - CometFilter [c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 @@ -46,10 +46,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [cs_bill_customer_sk,d_date] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] + CometFilter [cs_bill_customer_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 @@ -65,10 +65,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ws_bill_customer_sk,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt index 7d28d5c10..0c709e4f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,17 +27,17 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_moy] #6 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 @@ -50,14 +50,14 @@ WholeStageCodegen (5) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -65,11 +65,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk,d_moy] #10 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt index 7d28d5c10..0c709e4f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,17 +27,17 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_moy] #6 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 @@ -50,14 +50,14 @@ WholeStageCodegen (5) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -65,11 +65,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk,d_moy] #10 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt index 048da153b..1a19c58ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt @@ -16,25 +16,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -45,25 +45,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -75,15 +75,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [cs_bill_customer_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #12 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -96,15 +96,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [cs_bill_customer_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #15 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 @@ -118,15 +118,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #18 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #18 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -139,15 +139,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #21 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #21 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt index 10e0735b4..11c0201a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt @@ -21,14 +21,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [cs_warehouse_sk,cs_item_sk] + CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -39,14 +39,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [w_warehouse_sk] + CometFilter [w_warehouse_sk,w_state] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] InputAdapter BroadcastExchange #6 @@ -54,7 +54,7 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] ColumnarToRow InputAdapter CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] InputAdapter ReusedExchange [d_date_sk,d_date] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt index e31217066..457d441ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject [i_product_name] ColumnarToRow InputAdapter CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact] + CometFilter [i_manufact_id,i_manufact,i_product_name] CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] InputAdapter BroadcastExchange #2 @@ -23,7 +23,7 @@ TakeOrderedAndProject [i_product_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact] + CometHashAggregate [i_manufact,count] CometProject [i_manufact] - CometFilter [i_category,i_color,i_units,i_size,i_manufact] + CometFilter [i_category,i_manufact,i_size,i_color,i_units] CometScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt index 3e69a3341..e3b850ef5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price] + CometHashAggregate [d_year,i_category_id,i_category,sum,ss_ext_sales_price] CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_category_id,i_category] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 CometProject [i_item_sk,i_category_id,i_category] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt index d29a65bd4..5a060b85c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_name,s_store_id,d_day_name,ss_sales_price] + CometHashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [d_day_name,ss_store_sk,ss_sales_price,s_store_sk,s_store_id,s_store_name] CometProject [d_day_name,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_day_name] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange #2 - CometFilter [ss_store_sk] + CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 CometProject [s_store_sk,s_store_id,s_store_name] - CometFilter [s_gmt_offset,s_store_sk] + CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt index 701a90912..f713176f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt @@ -27,9 +27,9 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_store_sk,ss_net_profit] + CometHashAggregate [ss_store_sk,sum,count,ss_net_profit] CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_store_sk,ss_addr_sk] + CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count] InputAdapter @@ -37,9 +37,9 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,ss_net_profit] + CometHashAggregate [ss_item_sk,sum,count,ss_net_profit] CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_store_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] InputAdapter WholeStageCodegen (8) @@ -57,7 +57,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (9) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_product_name] InputAdapter ReusedExchange [i_item_sk,i_product_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt index c376c4fcd..f60fdb18a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt @@ -11,14 +11,14 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ca_city,ca_zip,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_sales_price,ca_city,ca_zip] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip,d_date_sk] CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_city,ca_zip] CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - CometFilter [ws_bill_customer_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,20 +26,20 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #4 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4 + CometFilter [ca_address_sk,ca_city,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #7 @@ -47,5 +47,5 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [i_item_id] - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt index 93e3eb05b..57defd961 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt @@ -10,16 +10,16 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,ss_coupon_amt,ss_net_profit] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_address_sk,ca_city] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] + CometFilter [s_store_sk,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk,ca_city] #6 CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter @@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt index d2e615f39..b6e5c469b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt index 6ebf6af07..5f628dbae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt @@ -5,16 +5,16 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_quantity] + CometHashAggregate [sum,ss_quantity] CometProject [ss_quantity] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] CometProject [ss_quantity,ss_sold_date_sk] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + CometBroadcastHashJoin [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] CometProject [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,s_store_sk] + CometFilter [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,19 +22,19 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #4 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] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_state] #5 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt index 8d7b158d0..0e6b65b06 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 @@ -27,14 +27,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] + CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] 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 + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #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] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_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 @@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] 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] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] @@ -71,18 +71,18 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] + CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] 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 #9 + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 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] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_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] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] ReusedExchange [d_date_sk] #6 WholeStageCodegen (15) @@ -105,17 +105,17 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] + CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] 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 #12 + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 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] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_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] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] ReusedExchange [d_date_sk] #6 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 c4ea8fe24..537cba446 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 @@ -15,14 +15,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,s_store_sk] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,18 +30,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (4) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] @@ -50,23 +50,23 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #7 - CometFilter [cp_catalog_page_sk] + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (6) 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] @@ -75,25 +75,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] 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] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_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 + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #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] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #10 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_site_id] #10 + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt index 473b9cdd4..58adabcec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sr_returned_date_sk,ss_sold_date_sk] + CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sum,sum,sum,sum,sum,sr_returned_date_sk,ss_sold_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk,s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometProject [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] - CometFilter [ss_ticket_number,ss_item_sk,ss_customer_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [sr_ticket_number,sr_item_sk,sr_customer_sk] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] #2 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,15 +26,15 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #4 + CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometFilter [d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt index ce6005da6..988297f02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt @@ -30,10 +30,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,d_date,ws_sales_price] + CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -41,11 +41,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_date] #6 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter WholeStageCodegen (10) @@ -67,10 +67,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,d_date,ss_sales_price] + CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt index 557dd3b4a..a0932c7cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [d_year,ext_price,brand_id,brand] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] + CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt index 83e53bb66..b2199fb34 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt @@ -15,18 +15,18 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,d_qoy,ss_sales_price] + CometHashAggregate [i_manufact_id,d_qoy,sum,ss_sales_price] CometProject [i_manufact_id,ss_sales_price,d_qoy] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [i_item_sk,i_manufact_id] - CometFilter [i_category,i_class,i_brand,i_item_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -34,12 +34,12 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [d_date_sk,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_qoy] #5 CometProject [d_date_sk,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt index c6886735f..2193d875c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt @@ -26,14 +26,14 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [c_customer_sk,c_current_addr_sk] CometProject [c_customer_sk,c_current_addr_sk] - CometBroadcastHashJoin [customer_sk,c_customer_sk] + CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk] CometProject [customer_sk] - CometBroadcastHashJoin [sold_date_sk,d_date_sk] + CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] CometProject [sold_date_sk,customer_sk] - CometBroadcastHashJoin [item_sk,i_item_sk] - CometUnion + CometBroadcastHashJoin [sold_date_sk,customer_sk,item_sk,i_item_sk] + CometUnion [sold_date_sk,customer_sk,item_sk] CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -41,21 +41,21 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #5 + CometBroadcastExchange [i_item_sk] #5 CometProject [i_item_sk] - CometFilter [i_category,i_class,i_item_sk] + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #7 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -71,7 +71,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] Subquery #3 WholeStageCodegen (2) HashAggregate [(d_month_seq + 1)] @@ -82,7 +82,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [(d_month_seq + 1)] CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] Subquery #4 WholeStageCodegen (2) @@ -94,7 +94,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [(d_month_seq + 3)] CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt index 2750a6ba2..fc38884c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [ext_price,brand_id,brand] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_brand,i_brand_id,ss_ext_sales_price] + CometHashAggregate [i_brand,i_brand_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt index 7fdead831..0b48046fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt @@ -14,14 +14,14 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_ext_sales_price] + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_item_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_item_id,i_item_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_item_id] #7 CometProject [i_item_id] - CometFilter [i_color] + CometFilter [i_item_id,i_color] CometScan parquet spark_catalog.default.item [i_item_id,i_color] WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -54,14 +54,14 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_ext_sales_price] + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -74,14 +74,14 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ws_ext_sales_price] + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt index c630cad48..fb2b09b9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [cs_item_sk,cs_call_center_sk] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt index 2ed2bde44..fc7c21e94 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt @@ -11,12 +11,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_ext_sales_price] + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,27 +24,27 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date] #3 CometProject [d_date] - CometFilter [d_week_seq] + CometFilter [d_date,d_week_seq] Subquery #2 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_date] + CometFilter [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk,i_item_id] #4 CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 InputAdapter @@ -57,12 +57,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_ext_sales_price] + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #4 @@ -77,12 +77,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ws_ext_sales_price] + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt index e00d52dbb..6c1e6cd92 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt @@ -12,20 +12,20 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price] + CometHashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_week_seq,d_day_name] + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [d_date_sk,d_week_seq] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 + CometFilter [d_date_sk,d_week_seq,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt index 89a080d85..824670101 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt @@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #2 - CometFilter [c_current_addr_sk,c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #3 - CometFilter [ss_customer_sk,ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] Subquery #2 WholeStageCodegen (2) HashAggregate [d_month_seq] @@ -41,12 +41,12 @@ TakeOrderedAndProject [cnt,state] InputAdapter CometHashAggregate [d_month_seq] CometProject [d_month_seq] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter @@ -56,7 +56,7 @@ TakeOrderedAndProject [cnt,state] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow InputAdapter - CometFilter [i_current_price,i_category,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #8 @@ -68,6 +68,6 @@ TakeOrderedAndProject [cnt,state] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_current_price] - CometFilter [i_category] + CometHashAggregate [i_category,sum,count,i_current_price] + CometFilter [i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt index b76e7c9b3..7cfcb75da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt @@ -14,14 +14,14 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_ext_sales_price] + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [i_item_id,total_sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_item_id,i_item_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_item_id] #7 CometProject [i_item_id] - CometFilter [i_category] + CometFilter [i_item_id,i_category] CometScan parquet spark_catalog.default.item [i_item_id,i_category] WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -54,14 +54,14 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_ext_sales_price] + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -74,14 +74,14 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ws_ext_sales_price] + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt index 4ca1dd667..1b2af33b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt @@ -7,20 +7,20 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_sales_price] + CometHashAggregate [sum,ss_ext_sales_price] CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,p_promo_sk] CometProject [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,30 +28,30 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometProject [s_store_sk] - CometFilter [s_gmt_offset,s_store_sk] + CometFilter [s_store_sk,s_gmt_offset] CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - CometBroadcastExchange #4 + CometBroadcastExchange [p_promo_sk] #4 CometProject [p_promo_sk] - CometFilter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk] #7 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #8 + CometBroadcastExchange [i_item_sk] #8 CometProject [i_item_sk] - CometFilter [i_category,i_item_sk] + CometFilter [i_item_sk,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_category] InputAdapter BroadcastExchange #9 @@ -62,18 +62,18 @@ WholeStageCodegen (4) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_sales_price] + CometHashAggregate [sum,ss_ext_sales_price] CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_customer_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [s_store_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt index c6b7e1834..0b4fc61a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt @@ -6,27 +6,27 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,sm_type,web_name,ws_ship_date_sk,ws_sold_date_sk] + CometHashAggregate [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum,ws_ship_date_sk,ws_sold_date_sk] CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression] - CometBroadcastHashJoin [ws_ship_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name,d_date_sk] CometProject [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] - CometBroadcastHashJoin [ws_web_site_sk,web_site_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name] CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - CometFilter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #3 - CometFilter [sm_ship_mode_sk] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange #4 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_name] #4 + CometFilter [web_site_sk,web_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt index 35e09ec7d..b2033c7a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt @@ -15,18 +15,18 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manager_id,d_moy,ss_sales_price] + CometHashAggregate [i_manager_id,d_moy,sum,ss_sales_price] CometProject [i_manager_id,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [i_item_sk,i_manager_id] - CometFilter [i_category,i_class,i_brand,i_item_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -34,12 +34,12 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] 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 d972e0082..3b5e4f19e 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 @@ -54,19 +54,19 @@ WholeStageCodegen (52) 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] - 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] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #4 + CometFilter [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] 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] + CometFilter [d_date_sk,d_year] 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] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (8) @@ -89,7 +89,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -100,7 +100,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] 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 @@ -116,14 +116,14 @@ WholeStageCodegen (52) WholeStageCodegen (11) 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] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter BroadcastExchange #11 WholeStageCodegen (12) ColumnarToRow InputAdapter - CometFilter [d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #11 @@ -157,7 +157,7 @@ WholeStageCodegen (52) WholeStageCodegen (19) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] InputAdapter ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 @@ -176,7 +176,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] - CometFilter [i_current_price,i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter WholeStageCodegen (50) @@ -227,19 +227,19 @@ WholeStageCodegen (52) 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] - 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] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #20 + CometFilter [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] 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] + CometFilter [d_date_sk,d_year] 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] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (33) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt index 8de564ed1..6cb247973 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes BroadcastHashJoin [s_store_sk,ss_store_sk] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #1 @@ -20,10 +20,10 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] + CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -31,18 +31,18 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] InputAdapter BroadcastExchange #6 @@ -59,10 +59,10 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] + CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt index d746739b3..72133811b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt @@ -14,37 +14,37 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] + CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] CometProject [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk] CometProject [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk] CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy] CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - CometFilter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 + CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastExchange #5 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [t_time_sk] #6 CometProject [t_time_sk] - CometFilter [t_time,t_time_sk] + CometFilter [t_time_sk,t_time] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] - CometBroadcastExchange #7 + CometBroadcastExchange [sm_ship_mode_sk] #7 CometProject [sm_ship_mode_sk] - CometFilter [sm_carrier,sm_ship_mode_sk] + CometFilter [sm_ship_mode_sk,sm_carrier] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] WholeStageCodegen (4) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] @@ -53,16 +53,16 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] + CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] CometProject [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk] CometProject [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk] CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy] CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - CometFilter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt index b529cb5ff..c37569802 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt @@ -14,15 +14,15 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty,ss_sales_price,ss_quantity] CometExpand [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] CometProject [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,15 +30,15 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ ColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt index 43f44c9f7..c5f56a69c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt @@ -10,16 +10,16 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_address_sk,ca_city] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] + CometFilter [s_store_sk,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk,ca_city] #6 CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter @@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt index de2d5eeda..c9cc4959b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,11 +27,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -39,7 +39,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -49,7 +49,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt index c583ba8e8..f327d5b06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt @@ -6,16 +6,16 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] + CometBroadcastHashJoin [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,p_promo_sk] CometProject [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_item_sk,ss_promo_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [p_promo_sk] #6 CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt index f4a03a2d2..f95e8d040 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #6 @@ -57,15 +57,15 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_state,ss_net_profit] + CometHashAggregate [s_state,sum,ss_net_profit] CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt index fd6777886..c604a8fa7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt @@ -9,19 +9,19 @@ WholeStageCodegen (3) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] + CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum,ext_price] CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] - CometBroadcastHashJoin [time_sk,t_time_sk] + CometBroadcastHashJoin [i_brand_id,i_brand,ext_price,time_sk,t_time_sk,t_hour,t_minute] CometProject [i_brand_id,i_brand,ext_price,time_sk] - CometBroadcastHashJoin [i_item_sk,sold_item_sk] - CometBroadcastExchange #3 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometUnion + CometUnion [ext_price,sold_item_sk,time_sk] CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_sold_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -29,25 +29,25 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_item_sk,cs_sold_time_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 CometProject [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_sold_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #6 + CometBroadcastExchange [t_time_sk,t_hour,t_minute] #6 CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_meal_time,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt index bea1fd4a1..6cba2d0e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt @@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #4 - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange #7 + CometBroadcastExchange [cd_demo_sk] #7 CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange #8 + CometBroadcastExchange [hd_demo_sk] #8 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #10 - CometFilter [d_week_seq,d_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #11 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #12 @@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt index 060c3e153..c91c4cf47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt index 2e6286d93..7c2a42ca3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt @@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt index 34866bc37..27a4dcb67 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt @@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #7 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] @@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] @@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_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 WholeStageCodegen (38) @@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange #18 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (23) @@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt index 473eef9e4..e463296d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt @@ -6,33 +6,33 @@ TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometUnion + CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum,ext_sales_price] + CometUnion [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] CometProject [ss_store_sk,ss_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_category] #2 + CometFilter [i_item_sk,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange #3 - CometFilter [d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometProject [ws_ship_customer_sk,ws_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_ship_customer_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category] + CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedExchange [i_item_sk,i_category] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #3 CometProject [cs_ship_addr_sk,cs_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_ship_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category] + CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedExchange [i_item_sk,i_category] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt index 590c59fdc..f9088f784 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt @@ -17,12 +17,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] + CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit] CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,13 +30,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter @@ -48,12 +48,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,sr_return_amt,sr_net_loss] + CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss] CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -70,9 +70,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit] CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -82,9 +82,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometHashAggregate [cr_return_amount,cr_net_loss] + CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -97,16 +97,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit] CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #12 + CometBroadcastExchange [wp_web_page_sk] #12 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter @@ -118,12 +118,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] + CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss] CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt index 280687e30..13a8aecff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -45,7 +45,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -83,7 +83,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -109,7 +109,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt index b68a9474b..57598ec35 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt @@ -8,14 +8,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum,ss_coupon_amt,ss_net_profit] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk,s_city] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,24 +23,24 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk,s_city] #4 CometProject [s_store_sk,s_city] - CometFilter [s_number_employees,s_store_sk] + CometFilter [s_store_sk,s_number_employees,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt index adf555417..637f3b7ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt @@ -10,10 +10,10 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] ColumnarToRow InputAdapter CometProject [ss_net_profit,s_store_name,s_zip] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip] CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -21,14 +21,14 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 - CometFilter [s_store_sk,s_zip] + CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4 + 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 #5 @@ -55,12 +55,12 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_zip] + CometHashAggregate [ca_zip,count] CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] + CometFilter [ca_address_sk,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange #9 + CometBroadcastExchange [c_current_addr_sk] #9 CometProject [c_current_addr_sk] - CometFilter [c_preferred_cust_flag,c_current_addr_sk] + CometFilter [c_current_addr_sk,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt index 7e257bdc6..cdc8dc486 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -40,7 +40,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -51,7 +51,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -60,7 +60,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #7 @@ -68,7 +68,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [i_item_sk] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #8 @@ -76,7 +76,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [p_promo_sk] - CometFilter [p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (20) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] @@ -102,7 +102,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -114,7 +114,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -123,7 +123,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cp_catalog_page_sk] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter ReusedExchange [i_item_sk] #7 @@ -153,7 +153,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (21) ColumnarToRow InputAdapter - CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -165,7 +165,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -174,7 +174,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (26) ColumnarToRow InputAdapter - CometFilter [web_site_sk] + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] InputAdapter ReusedExchange [i_item_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt index f80554fb5..e576a079b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt @@ -13,12 +13,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] + CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_addr_sk,cr_returning_customer_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter @@ -50,12 +50,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] + CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_addr_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -65,12 +65,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] InputAdapter BroadcastExchange #9 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt index a4d96e6f1..71a1c8e17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt @@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] InputAdapter CometHashAggregate [i_item_id,i_item_desc,i_current_price] CometProject [i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometBroadcastExchange #2 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,ss_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - CometBroadcastHashJoin [i_item_sk,inv_item_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange #3 + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [ss_item_sk] - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt index ef7d35e21..390c20b7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt @@ -10,12 +10,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,sr_return_quantity] + CometHashAggregate [i_item_id,sum,sr_return_quantity] CometProject [sr_return_quantity,i_item_id] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [sr_return_quantity,sr_returned_date_sk,i_item_id,d_date_sk] CometProject [sr_return_quantity,sr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [sr_item_sk,i_item_sk] - CometFilter [sr_item_sk] + CometBroadcastHashJoin [sr_item_sk,sr_return_quantity,sr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [sr_item_sk,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,24 +23,24 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date] #3 CometProject [d_date] - CometBroadcastHashJoin [d_week_seq,d_week_seq] + CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_week_seq] #4 CometProject [d_week_seq] - CometFilter [d_date] + CometFilter [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [i_item_sk,i_item_id] #5 CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 InputAdapter @@ -52,12 +52,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cr_return_quantity] + CometHashAggregate [i_item_id,sum,cr_return_quantity] CometProject [cr_return_quantity,i_item_id] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_quantity,cr_returned_date_sk,i_item_id,d_date_sk] CometProject [cr_return_quantity,cr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [cr_item_sk,i_item_sk] - CometFilter [cr_item_sk] + CometBroadcastHashJoin [cr_item_sk,cr_return_quantity,cr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [cr_item_sk,cr_return_quantity,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #5 @@ -71,12 +71,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,wr_return_quantity] + CometHashAggregate [i_item_id,sum,wr_return_quantity] CometProject [wr_return_quantity,i_item_id] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [wr_return_quantity,wr_returned_date_sk,i_item_id,d_date_sk] CometProject [wr_return_quantity,wr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [wr_item_sk,i_item_sk] - CometFilter [wr_item_sk] + CometBroadcastHashJoin [wr_item_sk,wr_return_quantity,wr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [wr_item_sk,wr_return_quantity,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt index fe7f7a207..646285a08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt @@ -3,32 +3,32 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] Project [c_customer_id,c_last_name,c_first_name] ColumnarToRow InputAdapter - CometBroadcastHashJoin [cd_demo_sk,sr_cdemo_sk] - CometBroadcastExchange #1 + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] + CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk,ib_income_band_sk] CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk] - CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk,hd_demo_sk,hd_income_band_sk] CometProject [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] CometProject [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] + CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastExchange #2 + CometBroadcastExchange [ca_address_sk] #2 CometProject [ca_address_sk] - CometFilter [ca_city,ca_address_sk] + CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 CometFilter [hd_demo_sk,hd_income_band_sk] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [ib_income_band_sk] #5 CometProject [ib_income_band_sk] - CometFilter [ib_lower_bound,ib_upper_bound,ib_income_band_sk] + CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] CometScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] CometProject [sr_cdemo_sk] - CometFilter [sr_cdemo_sk] + CometFilter [sr_cdemo_sk,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_cdemo_sk,sr_returned_date_sk] 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 ecae29c78..2313467ae 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 @@ -6,23 +6,23 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] + CometHashAggregate [r_reason_desc,sum,count,sum,count,sum,count,ws_quantity,wr_refunded_cash,wr_fee] CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - CometBroadcastHashJoin [wr_reason_sk,r_reason_sk] + CometBroadcastHashJoin [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash,r_reason_sk,r_reason_desc] CometProject [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash,d_date_sk] CometProject [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] + CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash,ca_address_sk,ca_state] CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] + CometBroadcastHashJoin [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,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [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_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [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,wp_web_page_sk] 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] + CometBroadcastHashJoin [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,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] + CometBroadcastExchange [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] #2 + CometFilter [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] 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 @@ -30,28 +30,28 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] 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] + CometFilter [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] 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] - CometBroadcastExchange #4 + CometBroadcastExchange [wp_web_page_sk] #4 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #5 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] - CometBroadcastExchange #6 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 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] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk,ca_state] #7 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #8 + CometBroadcastExchange [d_date_sk] #8 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [r_reason_sk] + CometBroadcastExchange [r_reason_sk,r_reason_desc] #9 + CometFilter [r_reason_sk,r_reason_desc] CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt index 4218938c1..a6d1c25fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt @@ -14,13 +14,13 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] + CometHashAggregate [i_category,i_class,spark_grouping_id,sum,ws_net_paid] CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] CometProject [ws_net_paid,i_category,i_class] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,12 +28,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt index cfac83844..9d667265c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt @@ -15,10 +15,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,14 +26,14 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #5 - CometFilter [c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 @@ -46,10 +46,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [cs_bill_customer_sk,d_date] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] + CometFilter [cs_bill_customer_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 @@ -65,10 +65,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ws_bill_customer_sk,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt index b846d25d8..d3af10d19 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt @@ -12,27 +12,27 @@ WholeStageCodegen (16) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #5 @@ -43,20 +43,20 @@ WholeStageCodegen (16) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #7 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -68,20 +68,20 @@ WholeStageCodegen (16) WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #10 + CometBroadcastExchange [t_time_sk] #10 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -93,20 +93,20 @@ WholeStageCodegen (16) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #13 + CometBroadcastExchange [t_time_sk] #13 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -118,20 +118,20 @@ WholeStageCodegen (16) WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #16 + CometBroadcastExchange [t_time_sk] #16 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -143,20 +143,20 @@ WholeStageCodegen (16) WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #19 + CometBroadcastExchange [t_time_sk] #19 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -168,20 +168,20 @@ WholeStageCodegen (16) WholeStageCodegen (12) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #22 + CometBroadcastExchange [t_time_sk] #22 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -193,19 +193,19 @@ WholeStageCodegen (16) WholeStageCodegen (14) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #25 + CometBroadcastExchange [t_time_sk] #25 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt index 507ac8a91..aa858c158 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt @@ -15,17 +15,17 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] + CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum,ss_sales_price] CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_category,i_class,i_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -33,12 +33,12 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt index c54606f6e..5cb600551 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt @@ -9,9 +9,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #1 ReusedSubquery [mergedValue] #1 @@ -24,9 +24,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #2 ReusedSubquery [mergedValue] #2 @@ -39,9 +39,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #3 ReusedSubquery [mergedValue] #3 @@ -54,9 +54,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #4 ReusedSubquery [mergedValue] #4 @@ -69,9 +69,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt index 50c8494fb..44159cc73 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt @@ -7,27 +7,27 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] + CometFilter [t_time_sk,t_hour] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - CometBroadcastExchange #4 + CometBroadcastExchange [wp_web_page_sk] #4 CometProject [wp_web_page_sk] - CometFilter [wp_char_count,wp_web_page_sk] + CometFilter [wp_web_page_sk,wp_char_count] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] InputAdapter BroadcastExchange #5 @@ -38,19 +38,19 @@ WholeStageCodegen (4) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #7 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] + CometFilter [t_time_sk,t_hour] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] ReusedExchange [wp_web_page_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt index b415eb5c4..232c174cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt @@ -9,23 +9,23 @@ WholeStageCodegen (3) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] + CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,cr_net_loss] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status,hd_demo_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,ca_address_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk,d_date_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] - CometBroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] - CometFilter [cc_call_center_sk] + CometBroadcastHashJoin [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] + CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometBroadcastExchange #3 - CometFilter [cr_call_center_sk,cr_returning_customer_sk] + CometBroadcastExchange [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] #3 + CometFilter [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -33,23 +33,23 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 - CometFilter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] #6 + CometFilter [c_customer_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] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk] #7 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #8 - CometFilter [cd_marital_status,cd_education_status,cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 + 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] - CometBroadcastExchange #9 + CometBroadcastExchange [hd_demo_sk] #9 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt index e2f498028..0681a64bd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt @@ -11,8 +11,8 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk,ws_ext_discount_amt] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -20,11 +20,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 @@ -36,15 +36,15 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,ws_ext_discount_amt] + CometHashAggregate [ws_item_sk,sum,count,ws_ext_discount_amt] CometProject [ws_item_sk,ws_ext_discount_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt index 3ec7ac7b6..a8eb231b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_ticket_number,sr_reason_sk] + CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] InputAdapter BroadcastExchange #4 @@ -36,5 +36,5 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] ColumnarToRow InputAdapter CometProject [r_reason_sk] - CometFilter [r_reason_desc,r_reason_sk] + CometFilter [r_reason_sk,r_reason_desc] CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt index 34ddde768..d54c9e0c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt @@ -26,7 +26,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -54,7 +54,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -62,7 +62,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -70,5 +70,5 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [web_site_sk] - CometFilter [web_company_name,web_site_sk] + CometFilter [web_site_sk,web_company_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt index 5b699890c..178c25f66 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt @@ -25,7 +25,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] InputAdapter WholeStageCodegen (7) @@ -40,7 +40,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ws_warehouse_sk,ws_order_number] - CometFilter [ws_order_number,ws_warehouse_sk] + CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -60,7 +60,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [wr_order_number] - CometFilter [wr_order_number] + CometFilter [wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] InputAdapter WholeStageCodegen (15) @@ -82,7 +82,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -90,7 +90,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -98,5 +98,5 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [web_site_sk] - CometFilter [web_company_name,web_site_sk] + CometFilter [web_site_sk,web_company_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt index 614915226..e9d33a7f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt @@ -5,25 +5,25 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt index 24e6dceef..41f3e579f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt @@ -17,7 +17,7 @@ WholeStageCodegen (6) InputAdapter CometHashAggregate [ss_customer_sk,ss_item_sk] CometProject [ss_item_sk,ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -25,11 +25,11 @@ WholeStageCodegen (6) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter WholeStageCodegen (4) @@ -42,7 +42,7 @@ WholeStageCodegen (6) InputAdapter CometHashAggregate [cs_bill_customer_sk,cs_item_sk] CometProject [cs_bill_customer_sk,cs_item_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt index c03b8be9a..3895cdc78 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt @@ -18,12 +18,12 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -31,12 +31,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt index adfe90bab..fc63929bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt @@ -6,27 +6,27 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days , WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,sm_type,cc_name,cs_ship_date_sk,cs_sold_date_sk] + CometHashAggregate [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum,cs_ship_date_sk,cs_sold_date_sk] CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression] - CometBroadcastHashJoin [cs_ship_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name,d_date_sk] CometProject [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name] CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - CometFilter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #3 - CometFilter [sm_ship_mode_sk] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange #4 - CometFilter [cc_call_center_sk] + CometBroadcastExchange [cc_call_center_sk,cc_name] #4 + CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt index 520edc88d..22c208f2a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] CometProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometProject [c_current_cdemo_sk] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk] CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,customer_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customer_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,28 +26,28 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometUnion + CometBroadcastExchange [customer_sk] #5 + CometUnion [customer_sk] CometProject [ws_bill_customer_sk] [customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 CometProject [cs_ship_customer_sk] [customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk] #6 CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] + CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - CometBroadcastExchange #7 - CometFilter [cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt index ecc421bd5..25010c5eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt @@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt index 545f0ecec..a8b8a9cce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt index 799f74a36..aa49638c7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt @@ -10,20 +10,20 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 @@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -47,13 +47,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #2 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 @@ -77,10 +77,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -88,26 +88,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #8 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk] #10 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #10 InputAdapter @@ -116,10 +116,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 @@ -153,7 +153,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #16 @@ -161,13 +161,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #6 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt index 45061c290..7ef901a39 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt @@ -25,15 +25,15 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #12 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #16 @@ -41,14 +41,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #17 + CometBroadcastExchange [d_date_sk] #17 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #4 ReusedExchange [d_date_sk] #17 @@ -64,7 +64,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -72,7 +72,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -81,7 +81,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #6 @@ -95,10 +95,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 @@ -106,26 +106,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #10 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #11 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #12 + CometBroadcastExchange [d_date_sk] #12 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #12 InputAdapter @@ -134,10 +134,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 @@ -148,7 +148,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #5 @@ -169,7 +169,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [cs_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -193,7 +193,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ws_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ws_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt index d4deedf0a..f854b2b31 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt @@ -7,20 +7,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,28 +28,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk] #5 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #6 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] WholeStageCodegen (4) HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] @@ -58,27 +58,27 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #10 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state,ca_country] #10 + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -89,28 +89,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #12 + CometBroadcastExchange [ca_address_sk,ca_country] #12 CometProject [ca_address_sk,ca_country] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -121,28 +121,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (7) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #14 + CometBroadcastExchange [ca_address_sk] #14 CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -153,20 +153,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 @@ -174,6 +174,6 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ReusedExchange [cd_demo_sk] #5 ReusedExchange [ca_address_sk] #14 ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #16 + CometBroadcastExchange [i_item_sk] #16 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt index 513e6f979..e460ce5c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt index 415b430f8..0e864ab70 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt @@ -11,10 +11,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,14 +22,14 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt index 6ab6c4fd7..30307e528 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt @@ -9,14 +9,14 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,16 +24,16 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange #5 + CometBroadcastExchange [w_warehouse_sk] #5 CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] WholeStageCodegen (5) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt index 7024f439f..5b658f1d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt @@ -43,7 +43,7 @@ WholeStageCodegen (12) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 @@ -78,7 +78,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -89,7 +89,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #6 @@ -97,26 +97,26 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #7 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #8 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_birth_country] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #9 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_country,ca_zip] + CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt index 371254b08..de75d46ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt @@ -7,16 +7,16 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] + CometHashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,21 +24,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [s_state,s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] WholeStageCodegen (4) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] @@ -47,23 +47,23 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk] #3 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #8 + CometBroadcastExchange [s_store_sk] #8 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [i_item_sk,i_item_id] #6 WholeStageCodegen (6) @@ -73,21 +73,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [agg1,agg2,agg3,agg4] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk] #3 ReusedExchange [d_date_sk] #4 ReusedExchange [s_store_sk] #8 - CometBroadcastExchange #10 + CometBroadcastExchange [i_item_sk] #10 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt index 970562e5c..814e543ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt index a6b4add1f..0b55e23ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt index e3a91e471..8d73022ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,ca_state,cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometProject [c_current_cdemo_sk,ca_state] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,customsk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customsk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,27 +26,27 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #5 - CometUnion + CometBroadcastExchange [customsk] #5 + CometUnion [customsk] CometProject [ws_bill_customer_sk] [customsk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 CometProject [cs_ship_customer_sk] [customsk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #6 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state] #6 + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #7 - CometFilter [cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt index 9c35ee397..256fddfc8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt @@ -22,14 +22,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [i_category,i_class,sum,sum,ss_net_profit,ss_ext_sales_price] CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,18 +37,18 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #7 + CometBroadcastExchange [s_store_sk] #7 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt index a85302cc5..eeeb5ba36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt index 8d7b158d0..0e6b65b06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] + CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] 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 + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #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] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_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 @@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] 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] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] @@ -71,18 +71,18 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] + CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] 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 #9 + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 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] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_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] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] ReusedExchange [d_date_sk] #6 WholeStageCodegen (15) @@ -105,17 +105,17 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] + CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] 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 #12 + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 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] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_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] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt index 78dd29c2a..e9c4d46ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt @@ -42,10 +42,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,d_date,ws_sales_price] + CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #6 @@ -53,11 +53,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter BroadcastExchange #8 @@ -96,10 +96,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (11) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,d_date,ss_sales_price] + CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt index c2c8e089f..968f7e1af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [cs_item_sk,cs_call_center_sk] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt index dd66c9582..1428c7ba5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt @@ -22,14 +22,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,s_store_sk] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,18 +37,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #6 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #6 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (4) 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] @@ -57,23 +57,23 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #8 - CometFilter [cp_catalog_page_sk] + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (6) 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] @@ -82,27 +82,27 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] 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] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_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 #10 + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10 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] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #11 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_site_id] #11 + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] WholeStageCodegen (17) 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt index 9607ab887..cfb74b3b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt @@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,ca_state,state] ColumnarToRow InputAdapter CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #2 - CometFilter [c_current_addr_sk,c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #3 - CometFilter [ss_customer_sk,ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,ca_state,state] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] Subquery #2 WholeStageCodegen (2) HashAggregate [d_month_seq] @@ -41,12 +41,12 @@ TakeOrderedAndProject [cnt,ca_state,state] InputAdapter CometHashAggregate [d_month_seq] CometProject [d_month_seq] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter @@ -56,7 +56,7 @@ TakeOrderedAndProject [cnt,ca_state,state] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow InputAdapter - CometFilter [i_current_price,i_category,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #8 @@ -68,6 +68,6 @@ TakeOrderedAndProject [cnt,ca_state,state] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_current_price] - CometFilter [i_category] + CometHashAggregate [i_category,sum,count,i_current_price] + CometFilter [i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt index 2a0bc5bce..512a74f9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt @@ -54,19 +54,19 @@ WholeStageCodegen (52) 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] - 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] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #4 + CometFilter [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] 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] + CometFilter [d_date_sk,d_year] 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] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (8) @@ -89,7 +89,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -100,7 +100,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] 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 @@ -116,14 +116,14 @@ WholeStageCodegen (52) WholeStageCodegen (11) 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] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter BroadcastExchange #11 WholeStageCodegen (12) ColumnarToRow InputAdapter - CometFilter [d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #11 @@ -157,7 +157,7 @@ WholeStageCodegen (52) WholeStageCodegen (19) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] InputAdapter ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 @@ -176,7 +176,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] - CometFilter [i_current_price,i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter WholeStageCodegen (50) @@ -227,19 +227,19 @@ WholeStageCodegen (52) 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] - 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] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #20 + CometFilter [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] 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] + CometFilter [d_date_sk,d_year] 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] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (33) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt index aeb5b67da..5838786d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt @@ -20,14 +20,14 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty,ss_sales_price,ss_quantity] CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -35,17 +35,17 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ ColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] WholeStageCodegen (5) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt index 74305dd6e..e76a524db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt @@ -26,8 +26,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -35,11 +35,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #6 @@ -47,7 +47,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #7 @@ -65,16 +65,16 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_state,ss_net_profit] + CometHashAggregate [s_state,sum,ss_net_profit] CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #9 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #9 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [d_date_sk] #5 WholeStageCodegen (13) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt index bea1fd4a1..6cba2d0e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt @@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #4 - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange #7 + CometBroadcastExchange [cd_demo_sk] #7 CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange #8 + CometBroadcastExchange [hd_demo_sk] #8 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #10 - CometFilter [d_week_seq,d_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #11 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #12 @@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt index 283fe5bde..e5cf23f50 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt @@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt index fb78d64b1..2aecc9e98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt @@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #7 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] @@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] @@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_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 WholeStageCodegen (38) @@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange #18 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (23) @@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt index 752b8c854..34972d290 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt @@ -24,12 +24,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] + CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit] CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,13 +37,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter @@ -55,12 +55,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,sr_return_amt,sr_net_loss] + CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss] CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -77,9 +77,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit] CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -89,9 +89,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometHashAggregate [cr_return_amount,cr_net_loss] + CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -104,16 +104,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit] CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #13 + CometBroadcastExchange [wp_web_page_sk] #13 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter @@ -125,12 +125,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] + CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss] CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt index 49bd173f6..b8e6e0605 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -45,7 +45,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -83,7 +83,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -109,7 +109,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt index 34e47dcba..0793067df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt @@ -39,7 +39,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -47,7 +47,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -58,7 +58,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -67,7 +67,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #8 @@ -75,7 +75,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [i_item_sk] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #9 @@ -83,7 +83,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [p_promo_sk] - CometFilter [p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (20) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] @@ -109,7 +109,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -130,7 +130,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cp_catalog_page_sk] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter ReusedExchange [i_item_sk] #8 @@ -160,7 +160,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (21) ColumnarToRow InputAdapter - CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -172,7 +172,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -181,7 +181,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (26) ColumnarToRow InputAdapter - CometFilter [web_site_sk] + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] InputAdapter ReusedExchange [i_item_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt index 0b7ad4726..5adee59be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt @@ -22,12 +22,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,ws_net_paid] + CometHashAggregate [i_category,i_class,sum,ws_net_paid] CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -35,14 +35,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt index 6484c2dcb..64b162608 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt @@ -17,12 +17,12 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,12 +30,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt index 520edc88d..22c208f2a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] CometProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometProject [c_current_cdemo_sk] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk] CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,customer_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customer_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,28 +26,28 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometUnion + CometBroadcastExchange [customer_sk] #5 + CometUnion [customer_sk] CometProject [ws_bill_customer_sk] [customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 CometProject [cs_ship_customer_sk] [customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk] #6 CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] + CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - CometBroadcastExchange #7 - CometFilter [cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt index 4b9e60e18..d7c6ef3b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt @@ -14,23 +14,23 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -43,23 +43,23 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -73,13 +73,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -94,13 +94,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt index 3d0576653..07c91b94d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt @@ -16,10 +16,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt index 73badab8d..cd4817202 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt @@ -11,19 +11,19 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ HashAggregate [quantity,list_price] [sum,count,sum,count] ColumnarToRow InputAdapter - CometUnion + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 @@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -47,7 +47,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] ReusedSubquery [d_week_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] Subquery #2 @@ -55,7 +55,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] InputAdapter BroadcastExchange #3 @@ -64,7 +64,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 @@ -78,10 +78,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -89,26 +89,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #8 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk] #10 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #10 InputAdapter @@ -117,10 +117,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 @@ -154,7 +154,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #16 @@ -162,7 +162,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] ReusedSubquery [d_week_seq] #6 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] Subquery #6 @@ -170,7 +170,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] InputAdapter ReusedExchange [ss_item_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt index 6e0ac9858..3dbd7bc33 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt @@ -26,14 +26,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num HashAggregate [quantity,list_price] [sum,count,sum,count] ColumnarToRow InputAdapter - CometUnion + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #12 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #16 @@ -41,14 +41,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #17 + CometBroadcastExchange [d_date_sk] #17 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #4 ReusedExchange [d_date_sk] #17 @@ -64,7 +64,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -72,7 +72,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -81,7 +81,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #6 @@ -95,10 +95,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 @@ -106,26 +106,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #10 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #11 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #12 + CometBroadcastExchange [d_date_sk] #12 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #12 InputAdapter @@ -134,10 +134,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 @@ -148,7 +148,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #5 @@ -169,7 +169,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [cs_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -193,7 +193,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ws_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ws_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt index 9f43cdc72..509d4be06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt @@ -9,18 +9,18 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,28 +28,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk] #5 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #6 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] WholeStageCodegen (4) HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] @@ -60,25 +60,25 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #10 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state,ca_country] #10 + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -91,26 +91,26 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #12 + CometBroadcastExchange [ca_address_sk,ca_country] #12 CometProject [ca_address_sk,ca_country] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -123,26 +123,26 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #14 + CometBroadcastExchange [ca_address_sk] #14 CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -155,18 +155,18 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 @@ -174,6 +174,6 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ReusedExchange [cd_demo_sk] #5 ReusedExchange [ca_address_sk] #14 ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #16 + CometBroadcastExchange [i_item_sk] #16 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt index da567f687..cd6bcfd61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt @@ -16,10 +16,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt index 415b430f8..0e864ab70 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt @@ -11,10 +11,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,14 +22,14 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt index db81d1ba5..fc4778a22 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt @@ -11,12 +11,12 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,16 +24,16 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange #5 + CometBroadcastExchange [w_warehouse_sk] #5 CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] WholeStageCodegen (5) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt index 7024f439f..5b658f1d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt @@ -43,7 +43,7 @@ WholeStageCodegen (12) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 @@ -78,7 +78,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -89,7 +89,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #6 @@ -97,26 +97,26 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #7 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #8 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_birth_country] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #9 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_country,ca_zip] + CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt index b09f5af82..50e246e15 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt @@ -9,14 +9,14 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,21 +24,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [s_state,s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] WholeStageCodegen (4) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] @@ -49,21 +49,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk] #3 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #8 + CometBroadcastExchange [s_store_sk] #8 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [i_item_sk,i_item_id] #6 WholeStageCodegen (6) @@ -75,19 +75,19 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk] #3 ReusedExchange [d_date_sk] #4 ReusedExchange [s_store_sk] #8 - CometBroadcastExchange #10 + CometBroadcastExchange [i_item_sk] #10 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt index 970562e5c..814e543ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt index a6b4add1f..0b55e23ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/simplified.txt index a95475f7b..79ead0464 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/simplified.txt @@ -8,17 +8,17 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,ca_state,cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometProject [c_current_cdemo_sk,ca_state] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,customsk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customsk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,27 +26,27 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #5 - CometUnion + CometBroadcastExchange [customsk] #5 + CometUnion [customsk] CometProject [ws_bill_customer_sk] [customsk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 CometProject [cs_ship_customer_sk] [customsk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #6 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state] #6 + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #7 - CometFilter [cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt index 485fdc3e2..59c22c335 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt @@ -24,12 +24,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,18 +37,18 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #7 + CometBroadcastExchange [s_store_sk] #7 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt index a480db5ea..61f4976b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt @@ -25,27 +25,27 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, ColumnarToRow InputAdapter CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt index d1480ef10..bd3b21cdb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt @@ -29,12 +29,12 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] 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 + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #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] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_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 @@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] 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] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] @@ -73,16 +73,16 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] 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 #9 + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 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] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_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] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] ReusedExchange [d_date_sk] #6 WholeStageCodegen (15) @@ -107,15 +107,15 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] 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 #12 + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 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] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_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] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt index 892a3fc78..47ad5e34f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt @@ -44,8 +44,8 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #6 @@ -53,11 +53,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter BroadcastExchange #8 @@ -98,8 +98,8 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt index b038aedf0..daeafbf92 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt @@ -25,27 +25,27 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m ColumnarToRow InputAdapter CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [cs_item_sk,cs_call_center_sk] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt index 34ad132c0..c3b00538b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt @@ -24,12 +24,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,s_store_sk] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,18 +37,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #6 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #6 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (4) 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] @@ -59,21 +59,21 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #8 - CometFilter [cp_catalog_page_sk] + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (6) 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] @@ -84,25 +84,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] 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] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_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 #10 + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10 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] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #11 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_site_id] #11 + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] WholeStageCodegen (17) 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt index a7c65f50a..febbfd32f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt @@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #2 - CometFilter [c_current_addr_sk,c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #3 - CometFilter [ss_customer_sk,ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] Subquery #2 @@ -43,11 +43,11 @@ TakeOrderedAndProject [cnt,state] InputAdapter CometHashAggregate [d_month_seq] CometProject [d_month_seq] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 @@ -58,7 +58,7 @@ TakeOrderedAndProject [cnt,state] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow InputAdapter - CometFilter [i_current_price,i_category,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #8 @@ -71,5 +71,5 @@ TakeOrderedAndProject [cnt,state] HashAggregate [i_category,i_current_price] [sum,count,sum,count] ColumnarToRow InputAdapter - CometFilter [i_category] + CometFilter [i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt index 2a0bc5bce..512a74f9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt @@ -54,19 +54,19 @@ WholeStageCodegen (52) 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] - 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] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #4 + CometFilter [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] 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] + CometFilter [d_date_sk,d_year] 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] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (8) @@ -89,7 +89,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -100,7 +100,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] 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 @@ -116,14 +116,14 @@ WholeStageCodegen (52) WholeStageCodegen (11) 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] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter BroadcastExchange #11 WholeStageCodegen (12) ColumnarToRow InputAdapter - CometFilter [d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #11 @@ -157,7 +157,7 @@ WholeStageCodegen (52) WholeStageCodegen (19) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] InputAdapter ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 @@ -176,7 +176,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] - CometFilter [i_current_price,i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter WholeStageCodegen (50) @@ -227,19 +227,19 @@ WholeStageCodegen (52) 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] - 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] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #20 + CometFilter [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] 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] + CometFilter [d_date_sk,d_year] 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] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (33) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt index f68ac8dfc..cc9627e89 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt @@ -22,12 +22,12 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ ColumnarToRow InputAdapter CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -35,17 +35,17 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ ColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] WholeStageCodegen (5) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt index a7c639a49..67f4841b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt @@ -26,8 +26,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -35,11 +35,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #6 @@ -47,7 +47,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #7 @@ -67,14 +67,14 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #9 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #9 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [d_date_sk] #5 WholeStageCodegen (13) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt index bea1fd4a1..6cba2d0e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt @@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #4 - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange #7 + CometBroadcastExchange [cd_demo_sk] #7 CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange #8 + CometBroadcastExchange [hd_demo_sk] #8 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #10 - CometFilter [d_week_seq,d_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #11 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #12 @@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt index a2cb15b4e..4360a9555 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt @@ -14,23 +14,23 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -43,23 +43,23 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -73,13 +73,13 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -94,13 +94,13 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt index fb78d64b1..2aecc9e98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt @@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #7 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] @@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] @@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_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 WholeStageCodegen (38) @@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange #18 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (23) @@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt index 141e72805..db4e10247 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt @@ -26,10 +26,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,13 +37,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter @@ -57,10 +57,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -79,7 +79,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -91,7 +91,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -106,14 +106,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #13 + CometBroadcastExchange [wp_web_page_sk] #13 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter @@ -127,10 +127,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt index 49bd173f6..b8e6e0605 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -45,7 +45,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -83,7 +83,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -109,7 +109,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt index 34e47dcba..0793067df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt @@ -39,7 +39,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -47,7 +47,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -58,7 +58,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -67,7 +67,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #8 @@ -75,7 +75,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [i_item_sk] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #9 @@ -83,7 +83,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [p_promo_sk] - CometFilter [p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (20) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] @@ -109,7 +109,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -130,7 +130,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cp_catalog_page_sk] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter ReusedExchange [i_item_sk] #8 @@ -160,7 +160,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (21) ColumnarToRow InputAdapter - CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -172,7 +172,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -181,7 +181,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (26) ColumnarToRow InputAdapter - CometFilter [web_site_sk] + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] InputAdapter ReusedExchange [i_item_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt index 7edc6b80b..8996119f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt @@ -24,10 +24,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl ColumnarToRow InputAdapter CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -35,14 +35,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt index 2dd86f85b..3fc9eb261 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt @@ -19,10 +19,10 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,12 +30,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt index 520edc88d..22c208f2a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] CometProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometProject [c_current_cdemo_sk] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk] CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,customer_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customer_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,28 +26,28 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometUnion + CometBroadcastExchange [customer_sk] #5 + CometUnion [customer_sk] CometProject [ws_bill_customer_sk] [customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 CometProject [cs_ship_customer_sk] [customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk] #6 CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] + CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - CometBroadcastExchange #7 - CometFilter [cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt index ecc421bd5..25010c5eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt @@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt index 545f0ecec..a8b8a9cce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt index 799f74a36..aa49638c7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt @@ -10,20 +10,20 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 @@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -47,13 +47,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #2 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 @@ -77,10 +77,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -88,26 +88,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #8 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk] #10 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #10 InputAdapter @@ -116,10 +116,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 @@ -153,7 +153,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #16 @@ -161,13 +161,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #6 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index 45061c290..7ef901a39 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -25,15 +25,15 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #12 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #16 @@ -41,14 +41,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #17 + CometBroadcastExchange [d_date_sk] #17 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #4 ReusedExchange [d_date_sk] #17 @@ -64,7 +64,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -72,7 +72,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -81,7 +81,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #6 @@ -95,10 +95,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 @@ -106,26 +106,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #10 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #11 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #12 + CometBroadcastExchange [d_date_sk] #12 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #12 InputAdapter @@ -134,10 +134,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 @@ -148,7 +148,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #5 @@ -169,7 +169,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [cs_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -193,7 +193,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ws_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ws_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt index d4deedf0a..f854b2b31 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt @@ -7,20 +7,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,28 +28,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk] #5 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #6 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] WholeStageCodegen (4) HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] @@ -58,27 +58,27 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #10 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state,ca_country] #10 + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -89,28 +89,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #12 + CometBroadcastExchange [ca_address_sk,ca_country] #12 CometProject [ca_address_sk,ca_country] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -121,28 +121,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (7) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #14 + CometBroadcastExchange [ca_address_sk] #14 CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -153,20 +153,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 @@ -174,6 +174,6 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ReusedExchange [cd_demo_sk] #5 ReusedExchange [ca_address_sk] #14 ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #16 + CometBroadcastExchange [i_item_sk] #16 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt index 513e6f979..e460ce5c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt index 415b430f8..0e864ab70 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt @@ -11,10 +11,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,14 +22,14 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt index 6ab6c4fd7..30307e528 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt @@ -9,14 +9,14 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,16 +24,16 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange #5 + CometBroadcastExchange [w_warehouse_sk] #5 CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] WholeStageCodegen (5) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt index 7024f439f..5b658f1d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt @@ -43,7 +43,7 @@ WholeStageCodegen (12) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 @@ -78,7 +78,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -89,7 +89,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #6 @@ -97,26 +97,26 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #7 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #8 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_birth_country] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #9 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_country,ca_zip] + CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt index 371254b08..de75d46ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt @@ -7,16 +7,16 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] + CometHashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,21 +24,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [s_state,s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] WholeStageCodegen (4) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] @@ -47,23 +47,23 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk] #3 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #8 + CometBroadcastExchange [s_store_sk] #8 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [i_item_sk,i_item_id] #6 WholeStageCodegen (6) @@ -73,21 +73,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [agg1,agg2,agg3,agg4] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk] #3 ReusedExchange [d_date_sk] #4 ReusedExchange [s_store_sk] #8 - CometBroadcastExchange #10 + CometBroadcastExchange [i_item_sk] #10 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt index 970562e5c..814e543ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt index a6b4add1f..0b55e23ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt index e3a91e471..8d73022ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,ca_state,cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometProject [c_current_cdemo_sk,ca_state] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,customsk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customsk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,27 +26,27 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #5 - CometUnion + CometBroadcastExchange [customsk] #5 + CometUnion [customsk] CometProject [ws_bill_customer_sk] [customsk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 CometProject [cs_ship_customer_sk] [customsk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #6 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state] #6 + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #7 - CometFilter [cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt index 9c35ee397..256fddfc8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt @@ -22,14 +22,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [i_category,i_class,sum,sum,ss_net_profit,ss_ext_sales_price] CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,18 +37,18 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #7 + CometBroadcastExchange [s_store_sk] #7 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt index a85302cc5..eeeb5ba36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter 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 8d7b158d0..0e6b65b06 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 @@ -27,14 +27,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] + CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] 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 + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #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] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_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 @@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] 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] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] @@ -71,18 +71,18 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] + CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] 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 #9 + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 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] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_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] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] ReusedExchange [d_date_sk] #6 WholeStageCodegen (15) @@ -105,17 +105,17 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] + CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] 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 #12 + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 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] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_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] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt index 78dd29c2a..e9c4d46ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt @@ -42,10 +42,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,d_date,ws_sales_price] + CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #6 @@ -53,11 +53,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter BroadcastExchange #8 @@ -96,10 +96,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (11) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,d_date,ss_sales_price] + CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt index c2c8e089f..968f7e1af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [cs_item_sk,cs_call_center_sk] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter 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 dd66c9582..1428c7ba5 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 @@ -22,14 +22,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,s_store_sk] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,18 +37,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #6 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #6 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (4) 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] @@ -57,23 +57,23 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #8 - CometFilter [cp_catalog_page_sk] + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (6) 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] @@ -82,27 +82,27 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] 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] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_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 #10 + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10 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] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #11 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_site_id] #11 + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] WholeStageCodegen (17) 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt index 9607ab887..cfb74b3b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt @@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,ca_state,state] ColumnarToRow InputAdapter CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #2 - CometFilter [c_current_addr_sk,c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #3 - CometFilter [ss_customer_sk,ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,ca_state,state] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] Subquery #2 WholeStageCodegen (2) HashAggregate [d_month_seq] @@ -41,12 +41,12 @@ TakeOrderedAndProject [cnt,ca_state,state] InputAdapter CometHashAggregate [d_month_seq] CometProject [d_month_seq] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter @@ -56,7 +56,7 @@ TakeOrderedAndProject [cnt,ca_state,state] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow InputAdapter - CometFilter [i_current_price,i_category,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #8 @@ -68,6 +68,6 @@ TakeOrderedAndProject [cnt,ca_state,state] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_current_price] - CometFilter [i_category] + CometHashAggregate [i_category,sum,count,i_current_price] + CometFilter [i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt index 2a0bc5bce..512a74f9f 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 @@ -54,19 +54,19 @@ WholeStageCodegen (52) 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] - 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] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #4 + CometFilter [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] 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] + CometFilter [d_date_sk,d_year] 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] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (8) @@ -89,7 +89,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -100,7 +100,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] 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 @@ -116,14 +116,14 @@ WholeStageCodegen (52) WholeStageCodegen (11) 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] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter BroadcastExchange #11 WholeStageCodegen (12) ColumnarToRow InputAdapter - CometFilter [d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #11 @@ -157,7 +157,7 @@ WholeStageCodegen (52) WholeStageCodegen (19) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] InputAdapter ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 @@ -176,7 +176,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] - CometFilter [i_current_price,i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter WholeStageCodegen (50) @@ -227,19 +227,19 @@ WholeStageCodegen (52) 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] - 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] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #20 + CometFilter [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] 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] + CometFilter [d_date_sk,d_year] 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] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (33) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt index a3a99e312..dfa9ae66d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt @@ -15,14 +15,14 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty,ss_sales_price,ss_quantity] CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,17 +30,17 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ ColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] WholeStageCodegen (5) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt index 9e72edbb4..feca96b18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt @@ -26,8 +26,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -35,11 +35,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #6 @@ -47,7 +47,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #7 @@ -64,16 +64,16 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_state,ss_net_profit] + CometHashAggregate [s_state,sum,ss_net_profit] CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #9 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #9 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [d_date_sk] #5 WholeStageCodegen (13) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt index bea1fd4a1..6cba2d0e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt @@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #4 - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange #7 + CometBroadcastExchange [cd_demo_sk] #7 CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange #8 + CometBroadcastExchange [hd_demo_sk] #8 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #10 - CometFilter [d_week_seq,d_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #11 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #12 @@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt index 283fe5bde..e5cf23f50 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt @@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt index fb78d64b1..2aecc9e98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt @@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #7 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] @@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] @@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_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 WholeStageCodegen (38) @@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange #18 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (23) @@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt index 752b8c854..34972d290 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt @@ -24,12 +24,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] + CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit] CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,13 +37,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter @@ -55,12 +55,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,sr_return_amt,sr_net_loss] + CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss] CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -77,9 +77,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit] CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -89,9 +89,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometHashAggregate [cr_return_amount,cr_net_loss] + CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -104,16 +104,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit] CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #13 + CometBroadcastExchange [wp_web_page_sk] #13 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter @@ -125,12 +125,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] + CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss] CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt index 49bd173f6..b8e6e0605 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -45,7 +45,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -83,7 +83,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -109,7 +109,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt index 34e47dcba..0793067df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt @@ -39,7 +39,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -47,7 +47,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -58,7 +58,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -67,7 +67,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #8 @@ -75,7 +75,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [i_item_sk] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #9 @@ -83,7 +83,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [p_promo_sk] - CometFilter [p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (20) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] @@ -109,7 +109,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -130,7 +130,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cp_catalog_page_sk] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter ReusedExchange [i_item_sk] #8 @@ -160,7 +160,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (21) ColumnarToRow InputAdapter - CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -172,7 +172,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -181,7 +181,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (26) ColumnarToRow InputAdapter - CometFilter [web_site_sk] + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] InputAdapter ReusedExchange [i_item_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt index 0b7ad4726..5adee59be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt @@ -22,12 +22,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,ws_net_paid] + CometHashAggregate [i_category,i_class,sum,ws_net_paid] CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -35,14 +35,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt index 6484c2dcb..64b162608 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt @@ -17,12 +17,12 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,12 +30,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] From 15e7baa5d1f76a1a0720e4f848cd01a14c09e147 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 9 Jul 2024 15:07:01 -0600 Subject: [PATCH 16/16] feat: Upgrade to DataFusion 40.0.0-rc1 (#644) * Partial upgrade to DataFusion 40.0.0 * fix * implement more udaf * update bitwise agg * add func names * remove unused imports * remove arrow-string dep * fix copy and paste error * use 40.0.0-rc1 and temporarily ignore failing test * clippy * fall back to Spark for count windows aggregate * address feedback --- native/Cargo.lock | 128 ++++++++++-------- native/core/Cargo.toml | 24 ++-- .../execution/datafusion/expressions/abs.rs | 2 +- .../execution/datafusion/expressions/avg.rs | 2 +- .../execution/datafusion/operators/expand.rs | 4 + .../core/src/execution/datafusion/planner.rs | 119 ++++++++++++---- .../execution/datafusion/shuffle_writer.rs | 4 + native/core/src/execution/operators/copy.rs | 4 + native/core/src/execution/operators/scan.rs | 4 + .../apache/comet/serde/QueryPlanSerde.scala | 5 +- .../apache/comet/exec/CometExecSuite.scala | 2 +- 11 files changed, 202 insertions(+), 96 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 6136e0339..df1828ee0 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -114,8 +114,9 @@ checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" [[package]] name = "arrow" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6127ea5e585a12ec9f742232442828ebaf264dfa5eefdd71282376c599562b77" dependencies = [ "arrow-arith", "arrow-array", @@ -134,8 +135,9 @@ dependencies = [ [[package]] name = "arrow-arith" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7add7f39210b7d726e2a8efc0083e7bf06e8f2d15bdb4896b564dce4410fbf5d" dependencies = [ "arrow-array", "arrow-buffer", @@ -148,8 +150,9 @@ dependencies = [ [[package]] name = "arrow-array" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "81c16ec702d3898c2f5cfdc148443c6cd7dbe5bac28399859eb0a3d38f072827" dependencies = [ "ahash", "arrow-buffer", @@ -164,8 +167,9 @@ dependencies = [ [[package]] name = "arrow-buffer" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cae6970bab043c4fbc10aee1660ceb5b306d0c42c8cc5f6ae564efcd9759b663" dependencies = [ "bytes", "half", @@ -174,8 +178,9 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1c7ef44f26ef4f8edc392a048324ed5d757ad09135eff6d5509e6450d39e0398" dependencies = [ "arrow-array", "arrow-buffer", @@ -194,8 +199,9 @@ dependencies = [ [[package]] name = "arrow-csv" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5f843490bd258c5182b66e888161bb6f198f49f3792f7c7f98198b924ae0f564" dependencies = [ "arrow-array", "arrow-buffer", @@ -212,8 +218,9 @@ dependencies = [ [[package]] name = "arrow-data" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a769666ffac256dd301006faca1ca553d0ae7cffcf4cd07095f73f95eb226514" dependencies = [ "arrow-buffer", "arrow-schema", @@ -223,8 +230,9 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dbf9c3fb57390a1af0b7bb3b5558c1ee1f63905f3eccf49ae7676a8d1e6e5a72" dependencies = [ "arrow-array", "arrow-buffer", @@ -237,8 +245,9 @@ dependencies = [ [[package]] name = "arrow-json" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "654e7f3724176b66ddfacba31af397c48e106fbe4d281c8144e7d237df5acfd7" dependencies = [ "arrow-array", "arrow-buffer", @@ -256,8 +265,9 @@ dependencies = [ [[package]] name = "arrow-ord" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e8008370e624e8e3c68174faaf793540287106cfda8ad1da862fdc53d8e096b4" dependencies = [ "arrow-array", "arrow-buffer", @@ -270,8 +280,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ca5e3a6b7fda8d9fe03f3b18a2d946354ea7f3c8e4076dbdb502ad50d9d44824" dependencies = [ "ahash", "arrow-array", @@ -284,16 +295,18 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dab1c12b40e29d9f3b699e0203c2a73ba558444c05e388a4377208f8f9c97eee" dependencies = [ "bitflags 2.6.0", ] [[package]] name = "arrow-select" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e80159088ffe8c48965cb9b1a7c968b2729f29f37363df7eca177fc3281fe7c3" dependencies = [ "ahash", "arrow-array", @@ -305,8 +318,9 @@ dependencies = [ [[package]] name = "arrow-string" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0fd04a6ea7de183648edbcb7a6dd925bbd04c210895f6384c780e27a9b54afcd" dependencies = [ "arrow-array", "arrow-buffer", @@ -790,8 +804,8 @@ dependencies = [ [[package]] name = "datafusion" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "ahash", "arrow", @@ -890,8 +904,8 @@ dependencies = [ [[package]] name = "datafusion-common" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "ahash", "arrow", @@ -910,16 +924,16 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "tokio", ] [[package]] name = "datafusion-execution" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "arrow", "chrono", @@ -938,8 +952,8 @@ dependencies = [ [[package]] name = "datafusion-expr" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "ahash", "arrow", @@ -956,8 +970,8 @@ dependencies = [ [[package]] name = "datafusion-functions" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "arrow", "base64", @@ -967,7 +981,6 @@ dependencies = [ "datafusion-common", "datafusion-execution", "datafusion-expr", - "datafusion-physical-expr", "hashbrown", "hex", "itertools 0.12.1", @@ -982,8 +995,8 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "ahash", "arrow", @@ -999,8 +1012,8 @@ dependencies = [ [[package]] name = "datafusion-optimizer" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "arrow", "async-trait", @@ -1012,13 +1025,14 @@ dependencies = [ "indexmap", "itertools 0.12.1", "log", + "paste", "regex-syntax", ] [[package]] name = "datafusion-physical-expr" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "ahash", "arrow", @@ -1032,7 +1046,6 @@ dependencies = [ "datafusion-common", "datafusion-execution", "datafusion-expr", - "datafusion-functions-aggregate", "datafusion-physical-expr-common", "half", "hashbrown", @@ -1047,19 +1060,21 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ + "ahash", "arrow", "datafusion-common", "datafusion-expr", + "hashbrown", "rand", ] [[package]] name = "datafusion-physical-plan" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "ahash", "arrow", @@ -1091,8 +1106,8 @@ dependencies = [ [[package]] name = "datafusion-sql" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "arrow", "arrow-array", @@ -2015,8 +2030,9 @@ dependencies = [ [[package]] name = "parquet" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0f22ba0d95db56dde8685e3fadcb915cdaadda31ab8abbe3ff7f0ad1ef333267" dependencies = [ "ahash", "bytes", diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index bd0a3d5e4..160db2949 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -32,12 +32,12 @@ include = [ ] [dependencies] -arrow = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1", features = ["prettyprint", "ffi", "chrono-tz"] } -arrow-array = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } -arrow-buffer = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } -arrow-data = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } -arrow-schema = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } -parquet = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1", default-features = false, features = ["experimental"] } +arrow = { version = "52.1.0", features = ["prettyprint", "ffi", "chrono-tz"] } +arrow-array = { version = "52.1.0" } +arrow-buffer = { version = "52.1.0" } +arrow-data = { version = "52.1.0" } +arrow-schema = { version = "52.1.0" } +parquet = { version = "52.1.0", default-features = false, features = ["experimental"] } half = { version = "2.4.1", default-features = false } futures = "0.3.28" mimalloc = { version = "*", default-features = false, optional = true } @@ -64,12 +64,12 @@ itertools = "0.11.0" chrono = { version = "0.4", default-features = false, features = ["clock"] } chrono-tz = { version = "0.8" } paste = "1.0.14" -datafusion-common = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1" } -datafusion = { default-features = false, git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1", features = ["unicode_expressions", "crypto_expressions"] } -datafusion-functions = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1", features = ["crypto_expressions"] } -datafusion-expr = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1", default-features = false } -datafusion-physical-expr-common = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1", default-features = false } -datafusion-physical-expr = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1", default-features = false } +datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1" } +datafusion = { default-features = false, git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", features = ["unicode_expressions", "crypto_expressions"] } +datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", features = ["crypto_expressions"] } +datafusion-expr = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", default-features = false } +datafusion-physical-expr-common = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", default-features = false } +datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", default-features = false } unicode-segmentation = "^1.10.1" once_cell = "1.18.0" regex = "1.9.6" diff --git a/native/core/src/execution/datafusion/expressions/abs.rs b/native/core/src/execution/datafusion/expressions/abs.rs index 4eb8c7c1e..a037e5cbc 100644 --- a/native/core/src/execution/datafusion/expressions/abs.rs +++ b/native/core/src/execution/datafusion/expressions/abs.rs @@ -37,7 +37,7 @@ impl CometAbsFunc { pub fn new(eval_mode: EvalMode, data_type_name: String) -> Result { if let EvalMode::Legacy | EvalMode::Ansi = eval_mode { Ok(Self { - inner_abs_func: math::abs().inner(), + inner_abs_func: math::abs().inner().clone(), eval_mode, data_type_name, }) diff --git a/native/core/src/execution/datafusion/expressions/avg.rs b/native/core/src/execution/datafusion/expressions/avg.rs index 1ff276e5d..3c8865bd1 100644 --- a/native/core/src/execution/datafusion/expressions/avg.rs +++ b/native/core/src/execution/datafusion/expressions/avg.rs @@ -47,7 +47,7 @@ pub struct Avg { impl Avg { /// Create a new AVG aggregate function pub fn new(expr: Arc, name: impl Into, data_type: DataType) -> Self { - let result_data_type = avg_return_type(&data_type).unwrap(); + let result_data_type = avg_return_type("avg", &data_type).unwrap(); Self { name: name.into(), diff --git a/native/core/src/execution/datafusion/operators/expand.rs b/native/core/src/execution/datafusion/operators/expand.rs index 5285dfb46..67171212f 100644 --- a/native/core/src/execution/datafusion/operators/expand.rs +++ b/native/core/src/execution/datafusion/operators/expand.rs @@ -126,6 +126,10 @@ impl ExecutionPlan for CometExpandExec { fn properties(&self) -> &PlanProperties { &self.cache } + + fn name(&self) -> &str { + "CometExpandExec" + } } pub struct ExpandStream { diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index 40515c0c4..360380400 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -20,19 +20,22 @@ use std::{collections::HashMap, sync::Arc}; use arrow_schema::{DataType, Field, Schema, TimeUnit}; +use datafusion::functions_aggregate::bit_and_or_xor::{bit_and_udaf, bit_or_udaf, bit_xor_udaf}; +use datafusion::functions_aggregate::count::count_udaf; +use datafusion::functions_aggregate::sum::sum_udaf; use datafusion::physical_plan::windows::BoundedWindowAggExec; use datafusion::physical_plan::InputOrderMode; use datafusion::{ arrow::{compute::SortOptions, datatypes::SchemaRef}, common::DataFusionError, execution::FunctionRegistry, + functions_aggregate::first_last::{FirstValue, LastValue}, logical_expr::Operator as DataFusionOperator, physical_expr::{ execution_props::ExecutionProps, expressions::{ - in_list, BinaryExpr, BitAnd, BitOr, BitXor, CaseExpr, CastExpr, Column, Count, - FirstValue, IsNotNullExpr, IsNullExpr, LastValue, Literal as DataFusionLiteral, Max, - Min, NotExpr, Sum, + in_list, BinaryExpr, CaseExpr, CastExpr, Column, IsNotNullExpr, IsNullExpr, + Literal as DataFusionLiteral, Max, Min, NotExpr, }, AggregateExpr, PhysicalExpr, PhysicalSortExpr, ScalarFunctionExpr, }, @@ -647,7 +650,7 @@ impl PhysicalPlanner { let left = self.create_expr(left, input_schema.clone())?; let right = self.create_expr(right, input_schema.clone())?; match ( - op, + &op, left.data_type(&input_schema), right.data_type(&input_schema), ) { @@ -1208,11 +1211,19 @@ impl PhysicalPlanner { .iter() .map(|child| self.create_expr(child, schema.clone())) .collect::, _>>()?; - Ok(Arc::new(Count::new_with_multiple_exprs( - children, + + create_aggregate_expr( + &count_udaf(), + &children, + &[], + &[], + &[], + schema.as_ref(), "count", - DataType::Int64, - ))) + false, + false, + ) + .map_err(|e| ExecutionError::DataFusionError(e.to_string())) } AggExprStruct::Min(expr) => { let child = self.create_expr(expr.child.as_ref().unwrap(), schema)?; @@ -1236,7 +1247,18 @@ impl PhysicalPlanner { // cast to the result data type of SUM if necessary, we should not expect // a cast failure since it should have already been checked at Spark side let child = Arc::new(CastExpr::new(child, datatype.clone(), None)); - Ok(Arc::new(Sum::new(child, "sum", datatype))) + create_aggregate_expr( + &sum_udaf(), + &[child], + &[], + &[], + &[], + schema.as_ref(), + "sum", + false, + false, + ) + .map_err(|e| ExecutionError::DataFusionError(e.to_string())) } } } @@ -1263,31 +1285,79 @@ impl PhysicalPlanner { AggExprStruct::First(expr) => { let child = self.create_expr(expr.child.as_ref().unwrap(), schema.clone())?; let func = datafusion_expr::AggregateUDF::new_from_impl(FirstValue::new()); - - create_aggregate_expr(&func, &[child], &[], &[], &schema, "first", false, false) - .map_err(|e| e.into()) + create_aggregate_expr( + &func, + &[child], + &[], + &[], + &[], + &schema, + "first", + false, + false, + ) + .map_err(|e| e.into()) } AggExprStruct::Last(expr) => { let child = self.create_expr(expr.child.as_ref().unwrap(), schema.clone())?; let func = datafusion_expr::AggregateUDF::new_from_impl(LastValue::new()); - - create_aggregate_expr(&func, &[child], &[], &[], &schema, "last", false, false) - .map_err(|e| e.into()) + create_aggregate_expr( + &func, + &[child], + &[], + &[], + &[], + &schema, + "last", + false, + false, + ) + .map_err(|e| e.into()) } AggExprStruct::BitAndAgg(expr) => { - let child = self.create_expr(expr.child.as_ref().unwrap(), schema)?; - let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap()); - Ok(Arc::new(BitAnd::new(child, "bit_and", datatype))) + let child = self.create_expr(expr.child.as_ref().unwrap(), schema.clone())?; + create_aggregate_expr( + &bit_and_udaf(), + &[child], + &[], + &[], + &[], + &schema, + "bit_and", + false, + false, + ) + .map_err(|e| e.into()) } AggExprStruct::BitOrAgg(expr) => { - let child = self.create_expr(expr.child.as_ref().unwrap(), schema)?; - let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap()); - Ok(Arc::new(BitOr::new(child, "bit_or", datatype))) + let child = self.create_expr(expr.child.as_ref().unwrap(), schema.clone())?; + create_aggregate_expr( + &bit_or_udaf(), + &[child], + &[], + &[], + &[], + &schema, + "bit_or", + false, + false, + ) + .map_err(|e| e.into()) } AggExprStruct::BitXorAgg(expr) => { - let child = self.create_expr(expr.child.as_ref().unwrap(), schema)?; - let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap()); - Ok(Arc::new(BitXor::new(child, "bit_xor", datatype))) + let child = self.create_expr(expr.child.as_ref().unwrap(), schema.clone())?; + create_aggregate_expr( + &bit_xor_udaf(), + &[child], + &[], + &[], + &[], + &schema, + "bit_xor", + false, + false, + ) + .map_err(|e| e.into()) } AggExprStruct::Covariance(expr) => { let child1 = self.create_expr(expr.child1.as_ref().unwrap(), schema.clone())?; @@ -1483,6 +1553,7 @@ impl PhysicalPlanner { &window_func, window_func_name, &window_args, + &[], partition_by, sort_exprs, window_frame.into(), diff --git a/native/core/src/execution/datafusion/shuffle_writer.rs b/native/core/src/execution/datafusion/shuffle_writer.rs index 5afc9a53e..6e59ce53a 100644 --- a/native/core/src/execution/datafusion/shuffle_writer.rs +++ b/native/core/src/execution/datafusion/shuffle_writer.rs @@ -160,6 +160,10 @@ impl ExecutionPlan for ShuffleWriterExec { fn properties(&self) -> &PlanProperties { &self.cache } + + fn name(&self) -> &str { + "ShuffleWriterExec" + } } impl ShuffleWriterExec { diff --git a/native/core/src/execution/operators/copy.rs b/native/core/src/execution/operators/copy.rs index d011b3cb2..68c91aafc 100644 --- a/native/core/src/execution/operators/copy.rs +++ b/native/core/src/execution/operators/copy.rs @@ -126,6 +126,10 @@ impl ExecutionPlan for CopyExec { fn properties(&self) -> &PlanProperties { &self.cache } + + fn name(&self) -> &str { + "CopyExec" + } } struct CopyStream { diff --git a/native/core/src/execution/operators/scan.rs b/native/core/src/execution/operators/scan.rs index de5328210..68dd773cf 100644 --- a/native/core/src/execution/operators/scan.rs +++ b/native/core/src/execution/operators/scan.rs @@ -270,6 +270,10 @@ impl ExecutionPlan for ScanExec { fn properties(&self) -> &PlanProperties { &self.cache } + + fn name(&self) -> &str { + "ScanExec" + } } impl DisplayAs for ScanExec { 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 65de37c83..da534b02c 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -208,7 +208,10 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim expr match { case agg: AggregateExpression => agg.aggregateFunction match { - case _: Min | _: Max | _: Count => + // TODO add support for Count (this was removed when upgrading + // to DataFusion 40 because it is no longer a built-in window function) + // https://github.com/apache/datafusion-comet/issues/645 + case _: Min | _: Max => Some(agg) case _ => withInfo(windowExpr, "Unsupported aggregate", expr) 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 9cc4e7f78..e657af9b9 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -1438,7 +1438,7 @@ class CometExecSuite extends CometTestBase { SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled) { withParquetTable((0 until 10).map(i => (i, 10 - i)), "t1") { // TODO: test nulls val aggregateFunctions = - List("COUNT(_1)", "MAX(_1)", "MIN(_1)") // TODO: Test all the aggregates + List("MAX(_1)", "MIN(_1)") // TODO: Test all the aggregates aggregateFunctions.foreach { function => val queries = Seq(