diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java index 2f143f77fa4ae..3aac2d2441d2a 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java @@ -188,6 +188,7 @@ public void handle(ResponseMessage message) throws Exception { if (listener == null) { logger.warn("Ignoring response for RPC {} from {} ({} bytes) since it is not outstanding", resp.requestId, getRemoteAddress(channel), resp.body().size()); + resp.body().release(); } else { outstandingRpcs.remove(resp.requestId); try { diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java index ec2e3dce661d9..0b7eaa6225a41 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java @@ -21,7 +21,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; -import java.util.HashMap; +import java.util.LinkedHashMap; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; @@ -81,7 +81,6 @@ public OneForOneBlockFetcher( TransportConf transportConf, DownloadFileManager downloadFileManager) { this.client = client; - this.blockIds = blockIds; this.listener = listener; this.chunkCallback = new ChunkCallback(); this.transportConf = transportConf; @@ -90,8 +89,10 @@ public OneForOneBlockFetcher( throw new IllegalArgumentException("Zero-sized blockIds array"); } if (!transportConf.useOldFetchProtocol() && isShuffleBlocks(blockIds)) { - this.message = createFetchShuffleBlocksMsg(appId, execId, blockIds); + this.blockIds = new String[blockIds.length]; + this.message = createFetchShuffleBlocksMsgAndBuildBlockIds(appId, execId, blockIds); } else { + this.blockIds = blockIds; this.message = new OpenBlocks(appId, execId, blockIds); } } @@ -106,17 +107,16 @@ private boolean isShuffleBlocks(String[] blockIds) { } /** - * Analyze the pass in blockIds and create FetchShuffleBlocks message. - * The blockIds has been sorted by mapId and reduceId. It's produced in - * org.apache.spark.MapOutputTracker.convertMapStatuses. + * Create FetchShuffleBlocks message and rebuild internal blockIds by + * analyzing the pass in blockIds. */ - private FetchShuffleBlocks createFetchShuffleBlocksMsg( + private FetchShuffleBlocks createFetchShuffleBlocksMsgAndBuildBlockIds( String appId, String execId, String[] blockIds) { String[] firstBlock = splitBlockId(blockIds[0]); int shuffleId = Integer.parseInt(firstBlock[1]); boolean batchFetchEnabled = firstBlock.length == 5; - HashMap> mapIdToReduceIds = new HashMap<>(); + LinkedHashMap mapIdToBlocksInfo = new LinkedHashMap<>(); for (String blockId : blockIds) { String[] blockIdParts = splitBlockId(blockId); if (Integer.parseInt(blockIdParts[1]) != shuffleId) { @@ -124,23 +124,36 @@ private FetchShuffleBlocks createFetchShuffleBlocksMsg( ", got:" + blockId); } long mapId = Long.parseLong(blockIdParts[2]); - if (!mapIdToReduceIds.containsKey(mapId)) { - mapIdToReduceIds.put(mapId, new ArrayList<>()); + if (!mapIdToBlocksInfo.containsKey(mapId)) { + mapIdToBlocksInfo.put(mapId, new BlocksInfo()); } - mapIdToReduceIds.get(mapId).add(Integer.parseInt(blockIdParts[3])); + BlocksInfo blocksInfoByMapId = mapIdToBlocksInfo.get(mapId); + blocksInfoByMapId.blockIds.add(blockId); + blocksInfoByMapId.reduceIds.add(Integer.parseInt(blockIdParts[3])); if (batchFetchEnabled) { // When we read continuous shuffle blocks in batch, we will reuse reduceIds in // FetchShuffleBlocks to store the start and end reduce id for range // [startReduceId, endReduceId). assert(blockIdParts.length == 5); - mapIdToReduceIds.get(mapId).add(Integer.parseInt(blockIdParts[4])); + blocksInfoByMapId.reduceIds.add(Integer.parseInt(blockIdParts[4])); } } - long[] mapIds = Longs.toArray(mapIdToReduceIds.keySet()); + long[] mapIds = Longs.toArray(mapIdToBlocksInfo.keySet()); int[][] reduceIdArr = new int[mapIds.length][]; + int blockIdIndex = 0; for (int i = 0; i < mapIds.length; i++) { - reduceIdArr[i] = Ints.toArray(mapIdToReduceIds.get(mapIds[i])); + BlocksInfo blocksInfoByMapId = mapIdToBlocksInfo.get(mapIds[i]); + reduceIdArr[i] = Ints.toArray(blocksInfoByMapId.reduceIds); + + // The `blockIds`'s order must be same with the read order specified in in FetchShuffleBlocks + // because the shuffle data's return order should match the `blockIds`'s order to ensure + // blockId and data match. + for (int j = 0; j < blocksInfoByMapId.blockIds.size(); j++) { + this.blockIds[blockIdIndex++] = blocksInfoByMapId.blockIds.get(j); + } } + assert(blockIdIndex == this.blockIds.length); + return new FetchShuffleBlocks( appId, execId, shuffleId, mapIds, reduceIdArr, batchFetchEnabled); } @@ -157,6 +170,18 @@ private String[] splitBlockId(String blockId) { return blockIdParts; } + /** The reduceIds and blocks in a single mapId */ + private class BlocksInfo { + + final ArrayList reduceIds; + final ArrayList blockIds; + + BlocksInfo() { + this.reduceIds = new ArrayList<>(); + this.blockIds = new ArrayList<>(); + } + } + /** Callback invoked on receipt of each chunk. We equate a single chunk to a single block. */ private class ChunkCallback implements ChunkReceivedCallback { @Override diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index 0e2355646465d..9363efc58d7c6 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -30,7 +30,6 @@ import java.util.Arrays; import java.util.Collection; import java.util.Iterator; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentMap; @@ -445,9 +444,9 @@ static class PushBlockStreamCallback implements StreamCallbackWithID { private final AppShufflePartitionInfo partitionInfo; private int length = 0; // This indicates that this stream got the opportunity to write the blocks to the merged file. - // Once this is set to true and the stream encounters a failure then it will take necessary - // action to overwrite any partial written data. This is reset to false when the stream - // completes without any failures. + // Once this is set to true and the stream encounters a failure then it will unset the + // currentMapId of the partition so that another stream can start merging the blocks to the + // partition. This is reset to false when the stream completes. private boolean isWriting = false; // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly private List deferredBufs; @@ -477,16 +476,11 @@ public String getID() { */ private void writeBuf(ByteBuffer buf) throws IOException { while (buf.hasRemaining()) { - if (partitionInfo.isEncounteredFailure()) { - long updatedPos = partitionInfo.getDataFilePos() + length; - logger.debug( - "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}", - partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId, - partitionInfo.reduceId, partitionInfo.getDataFilePos(), updatedPos); - length += partitionInfo.dataChannel.write(buf, updatedPos); - } else { - length += partitionInfo.dataChannel.write(buf); - } + long updatedPos = partitionInfo.getDataFilePos() + length; + logger.debug("{} shuffleId {} reduceId {} current pos {} updated pos {}", + partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId, + partitionInfo.reduceId, partitionInfo.getDataFilePos(), updatedPos); + length += partitionInfo.dataChannel.write(buf, updatedPos); } } @@ -581,7 +575,6 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { } // Check whether we can write to disk if (allowedToWrite()) { - isWriting = true; // Identify duplicate block generated by speculative tasks. We respond success to // the client in cases of duplicate even though no data is written. if (isDuplicateBlock()) { @@ -598,6 +591,7 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { // If we got here, it's safe to write the block data to the merged shuffle file. We // first write any deferred block. + isWriting = true; try { if (deferredBufs != null && !deferredBufs.isEmpty()) { writeDeferredBufs(); @@ -609,16 +603,6 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { // back to the client so the block could be retried. throw ioe; } - // If we got here, it means we successfully write the current chunk of block to merged - // shuffle file. If we encountered failure while writing the previous block, we should - // reset the file channel position and the status of partitionInfo to indicate that we - // have recovered from previous disk write failure. However, we do not update the - // position tracked by partitionInfo here. That is only updated while the entire block - // is successfully written to merged shuffle file. - if (partitionInfo.isEncounteredFailure()) { - partitionInfo.dataChannel.position(partitionInfo.getDataFilePos() + length); - partitionInfo.setEncounteredFailure(false); - } } else { logger.trace("{} shuffleId {} reduceId {} onData deferred", partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId, @@ -639,7 +623,7 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { // written to disk due to this reason. We thus decide to optimize for server // throughput and memory usage. if (deferredBufs == null) { - deferredBufs = new LinkedList<>(); + deferredBufs = new ArrayList<>(); } // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger // byte buffer, we cache only the relevant bytes not the entire large buffer to save @@ -670,7 +654,6 @@ public void onComplete(String streamId) throws IOException { } // Check if we can commit this block if (allowedToWrite()) { - isWriting = true; // Identify duplicate block generated by speculative tasks. We respond success to // the client in cases of duplicate even though no data is written. if (isDuplicateBlock()) { @@ -681,6 +664,7 @@ public void onComplete(String streamId) throws IOException { try { if (deferredBufs != null && !deferredBufs.isEmpty()) { abortIfNecessary(); + isWriting = true; writeDeferredBufs(); } } catch (IOException ioe) { @@ -738,14 +722,14 @@ public void onFailure(String streamId, Throwable throwable) throws IOException { Map shufflePartitions = mergeManager.partitions.get(partitionInfo.appShuffleId); if (shufflePartitions != null && shufflePartitions.containsKey(partitionInfo.reduceId)) { - logger.debug("{} shuffleId {} reduceId {} set encountered failure", + logger.debug("{} shuffleId {} reduceId {} encountered failure", partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId); partitionInfo.setCurrentMapIndex(-1); - partitionInfo.setEncounteredFailure(true); } } } + isWriting = false; } @VisibleForTesting @@ -802,8 +786,6 @@ public static class AppShufflePartitionInfo { public FileChannel dataChannel; // Location offset of the last successfully merged block for this shuffle partition private long dataFilePos; - // Indicating whether failure was encountered when merging the previous block - private boolean encounteredFailure; // Track the map index whose block is being merged for this shuffle partition private int currentMapIndex; // Bitmap tracking which mapper's blocks have been merged for this shuffle partition @@ -836,7 +818,6 @@ public static class AppShufflePartitionInfo { // Writing 0 offset so that we can reuse ShuffleIndexInformation.getIndex() updateChunkInfo(0L, -1); this.dataFilePos = 0; - this.encounteredFailure = false; this.mapTracker = new RoaringBitmap(); this.chunkTracker = new RoaringBitmap(); } @@ -851,14 +832,6 @@ public void setDataFilePos(long dataFilePos) { this.dataFilePos = dataFilePos; } - boolean isEncounteredFailure() { - return encounteredFailure; - } - - void setEncounteredFailure(boolean encounteredFailure) { - this.encounteredFailure = encounteredFailure; - } - int getCurrentMapIndex() { return currentMapIndex; } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java index 285eedb39c65c..a7eb59d366966 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java @@ -201,6 +201,48 @@ public void testEmptyBlockFetch() { } } + @Test + public void testFetchShuffleBlocksOrder() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("shuffle_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[1]))); + blocks.put("shuffle_0_2_1", new NioManagedBuffer(ByteBuffer.wrap(new byte[2]))); + blocks.put("shuffle_0_10_2", new NettyManagedBuffer(Unpooled.wrappedBuffer(new byte[3]))); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); + + BlockFetchingListener listener = fetchBlocks( + blocks, + blockIds, + new FetchShuffleBlocks("app-id", "exec-id", 0, + new long[]{0, 2, 10}, new int[][]{{0}, {1}, {2}}, false), + conf); + + for (int chunkIndex = 0; chunkIndex < blockIds.length; chunkIndex++) { + String blockId = blockIds[chunkIndex]; + verify(listener).onBlockFetchSuccess(blockId, blocks.get(blockId)); + } + } + + @Test + public void testBatchFetchShuffleBlocksOrder() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("shuffle_0_0_1_2", new NioManagedBuffer(ByteBuffer.wrap(new byte[1]))); + blocks.put("shuffle_0_2_2_3", new NioManagedBuffer(ByteBuffer.wrap(new byte[2]))); + blocks.put("shuffle_0_10_3_4", new NettyManagedBuffer(Unpooled.wrappedBuffer(new byte[3]))); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); + + BlockFetchingListener listener = fetchBlocks( + blocks, + blockIds, + new FetchShuffleBlocks("app-id", "exec-id", 0, + new long[]{0, 2, 10}, new int[][]{{1, 2}, {2, 3}, {3, 4}}, true), + conf); + + for (int chunkIndex = 0; chunkIndex < blockIds.length; chunkIndex++) { + String blockId = blockIds[chunkIndex]; + verify(listener).onBlockFetchSuccess(blockId, blocks.get(blockId)); + } + } + /** * Begins a fetch on the given set of blocks by mocking out the server side of the RPC which * simply returns the given (BlockId, Block) pairs. diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java index 8c6f7434748ec..565d433ff3203 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java @@ -28,6 +28,7 @@ import java.nio.file.Paths; import java.util.Arrays; import java.util.concurrent.Semaphore; +import java.util.concurrent.ThreadLocalRandom; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -292,18 +293,32 @@ public void testTooLateArrival() throws IOException { @Test public void testIncompleteStreamsAreOverwritten() throws IOException { registerExecutor(TEST_APP, prepareLocalDirs(localDirs)); + byte[] expectedBytes = new byte[4]; + ThreadLocalRandom.current().nextBytes(expectedBytes); + StreamCallbackWithID stream1 = pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0)); - stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[4])); + byte[] data = new byte[10]; + ThreadLocalRandom.current().nextBytes(data); + stream1.onData(stream1.getID(), ByteBuffer.wrap(data)); // There is a failure stream1.onFailure(stream1.getID(), new RuntimeException("forced error")); StreamCallbackWithID stream2 = pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 1, 0, 0)); - stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[5])); + ByteBuffer nextBuf= ByteBuffer.wrap(expectedBytes, 0, 2); + stream2.onData(stream2.getID(), nextBuf); stream2.onComplete(stream2.getID()); + StreamCallbackWithID stream3 = + pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 2, 0, 0)); + nextBuf = ByteBuffer.wrap(expectedBytes, 2, 2); + stream3.onData(stream3.getID(), nextBuf); + stream3.onComplete(stream3.getID()); pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); - validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{5}, new int[][]{{1}}); + validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4}, new int[][]{{1, 2}}); + FileSegmentManagedBuffer mb = + (FileSegmentManagedBuffer) pushResolver.getMergedBlockData(TEST_APP, 0, 0, 0); + assertArrayEquals(expectedBytes, mb.nioByteBuffer().array()); } @Test (expected = RuntimeException.class) @@ -740,6 +755,72 @@ public void testFailureWhileTruncatingFiles() throws IOException { validateChunks(TEST_APP, 0, 1, meta, new int[]{5, 3}, new int[][]{{0},{1}}); } + @Test + public void testOnFailureInvokedMoreThanOncePerBlock() throws IOException { + StreamCallbackWithID stream1 = + pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0)); + stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2])); + stream1.onFailure(stream1.getID(), new RuntimeException("forced error")); + StreamCallbackWithID stream2 = + pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 1, 0, 0)); + stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[5])); + // On failure on stream1 gets invoked again and should cause no interference + stream1.onFailure(stream1.getID(), new RuntimeException("2nd forced error")); + StreamCallbackWithID stream3 = + pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 3, 0, 0)); + // This should be deferred as stream 2 is still the active stream + stream3.onData(stream3.getID(), ByteBuffer.wrap(new byte[2])); + // Stream 2 writes more and completes + stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[4])); + stream2.onComplete(stream2.getID()); + stream3.onComplete(stream3.getID()); + pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); + MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); + validateChunks(TEST_APP, 0, 0, blockMeta, new int[] {9, 2}, new int[][] {{1},{3}}); + removeApplication(TEST_APP); + } + + @Test (expected = RuntimeException.class) + public void testFailureAfterDuplicateBlockDoesNotInterfereActiveStream() throws IOException { + StreamCallbackWithID stream1 = + pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0)); + StreamCallbackWithID stream1Duplicate = + pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0)); + stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2])); + stream1.onComplete(stream1.getID()); + stream1Duplicate.onData(stream1.getID(), ByteBuffer.wrap(new byte[2])); + + StreamCallbackWithID stream2 = + pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 1, 0, 0)); + stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[5])); + // Should not change the current map id of the reduce partition + stream1Duplicate.onFailure(stream2.getID(), new RuntimeException("forced error")); + + StreamCallbackWithID stream3 = + pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 2, 0, 0)); + // This should be deferred as stream 2 is still the active stream + stream3.onData(stream3.getID(), ByteBuffer.wrap(new byte[2])); + RuntimeException failedEx = null; + try { + stream3.onComplete(stream3.getID()); + } catch (RuntimeException re) { + assertEquals( + "Couldn't find an opportunity to write block shufflePush_0_2_0 to merged shuffle", + re.getMessage()); + failedEx = re; + } + // Stream 2 writes more and completes + stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[4])); + stream2.onComplete(stream2.getID()); + pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); + MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); + validateChunks(TEST_APP, 0, 0, blockMeta, new int[] {11}, new int[][] {{0, 1}}); + removeApplication(TEST_APP); + if (failedEx != null) { + throw failedEx; + } + } + private void useTestFiles(boolean useTestIndexFile, boolean useTestMetaFile) throws IOException { pushResolver = new RemoteBlockPushResolver(conf) { @Override diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java index dc8d6e3a2f0ba..12867627379fb 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java @@ -45,30 +45,20 @@ public final class Platform { private static final boolean unaligned; - // Access fields and constructors once and store them, for performance: - - private static final Constructor DBB_CONSTRUCTOR; - private static final Field DBB_CLEANER_FIELD; - static { - try { - Class cls = Class.forName("java.nio.DirectByteBuffer"); - Constructor constructor = cls.getDeclaredConstructor(Long.TYPE, Integer.TYPE); - constructor.setAccessible(true); - Field cleanerField = cls.getDeclaredField("cleaner"); - cleanerField.setAccessible(true); - DBB_CONSTRUCTOR = constructor; - DBB_CLEANER_FIELD = cleanerField; - } catch (ClassNotFoundException | NoSuchMethodException | NoSuchFieldException e) { - throw new IllegalStateException(e); - } - } - // Split java.version on non-digit chars: private static final int majorVersion = Integer.parseInt(System.getProperty("java.version").split("\\D+")[0]); + // Access fields and constructors once and store them, for performance: + private static final Constructor DBB_CONSTRUCTOR; + private static final Field DBB_CLEANER_FIELD; private static final Method CLEANER_CREATE_METHOD; + static { + // At the end of this block, CLEANER_CREATE_METHOD should be non-null iff it's possible to use + // reflection to invoke it, which is not necessarily possible by default in Java 9+. + // Code below can test for null to see whether to use it. + // The implementation of Cleaner changed from JDK 8 to 9 String cleanerClassName; if (majorVersion < 9) { @@ -77,28 +67,53 @@ public final class Platform { cleanerClassName = "jdk.internal.ref.Cleaner"; } try { - Class cleanerClass = Class.forName(cleanerClassName); - Method createMethod = cleanerClass.getMethod("create", Object.class, Runnable.class); - // Accessing jdk.internal.ref.Cleaner should actually fail by default in JDK 9+, - // unfortunately, unless the user has allowed access with something like - // --add-opens java.base/java.lang=ALL-UNNAMED If not, we can't really use the Cleaner - // hack below. It doesn't break, just means the user might run into the default JVM limit - // on off-heap memory and increase it or set the flag above. This tests whether it's - // available: + Class cls = Class.forName("java.nio.DirectByteBuffer"); + Constructor constructor = cls.getDeclaredConstructor(Long.TYPE, Integer.TYPE); + Field cleanerField = cls.getDeclaredField("cleaner"); try { - createMethod.invoke(null, null, null); - } catch (IllegalAccessException e) { - // Don't throw an exception, but can't log here? - createMethod = null; - } catch (InvocationTargetException ite) { - // shouldn't happen; report it - throw new IllegalStateException(ite); + constructor.setAccessible(true); + cleanerField.setAccessible(true); + } catch (RuntimeException re) { + // This is a Java 9+ exception, so needs to be handled without importing it + if ("InaccessibleObjectException".equals(re.getClass().getSimpleName())) { + // Continue, but the constructor/field are not available + // See comment below for more context + constructor = null; + cleanerField = null; + } else { + throw re; + } } - CLEANER_CREATE_METHOD = createMethod; - } catch (ClassNotFoundException | NoSuchMethodException e) { + // Have to set these values no matter what: + DBB_CONSTRUCTOR = constructor; + DBB_CLEANER_FIELD = cleanerField; + + // no point continuing if the above failed: + if (DBB_CONSTRUCTOR != null && DBB_CLEANER_FIELD != null) { + Class cleanerClass = Class.forName(cleanerClassName); + Method createMethod = cleanerClass.getMethod("create", Object.class, Runnable.class); + // Accessing jdk.internal.ref.Cleaner should actually fail by default in JDK 9+, + // unfortunately, unless the user has allowed access with something like + // --add-opens java.base/java.lang=ALL-UNNAMED If not, we can't really use the Cleaner + // hack below. It doesn't break, just means the user might run into the default JVM limit + // on off-heap memory and increase it or set the flag above. This tests whether it's + // available: + try { + createMethod.invoke(null, null, null); + } catch (IllegalAccessException e) { + // Don't throw an exception, but can't log here? + createMethod = null; + } + CLEANER_CREATE_METHOD = createMethod; + } else { + CLEANER_CREATE_METHOD = null; + } + } catch (ClassNotFoundException | NoSuchMethodException | NoSuchFieldException e) { + // These are all fatal in any Java version - rethrow (have to wrap as this is a static block) throw new IllegalStateException(e); + } catch (InvocationTargetException ite) { + throw new IllegalStateException(ite.getCause()); } - } /** diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js index 91bf274aa467d..8b32fe7d3e20d 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -43,6 +43,23 @@ $.extend( $.fn.dataTable.ext.type.order, { a = ConvertDurationString( a ); b = ConvertDurationString( b ); return ((a < b) ? 1 : ((a > b) ? -1 : 0)); + }, + + "size-pre": function (data) { + var floatValue = parseFloat(data) + return isNaN(floatValue) ? 0 : floatValue; + }, + + "size-asc": function (a, b) { + a = parseFloat(a); + b = parseFloat(b); + return ((a < b) ? -1 : ((a > b) ? 1 : 0)); + }, + + "size-desc": function (a, b) { + a = parseFloat(a); + b = parseFloat(b); + return ((a < b) ? 1 : ((a > b) ? -1 : 0)); } } ); @@ -562,10 +579,27 @@ $(document).ready(function () { } ], "columnDefs": [ - { "visible": false, "targets": 15 }, - { "visible": false, "targets": 16 }, - { "visible": false, "targets": 17 }, - { "visible": false, "targets": 18 } + // SPARK-35087 [type:size] means String with structures like : 'size / records', + // they should be sorted as numerical-order instead of lexicographical-order by default. + // The targets: $id represents column id which comes from stagespage-template.html + // #summary-executor-table.If the relative position of the columns in the table + // #summary-executor-table has changed,please be careful to adjust the column index here + // Input Size / Records + {"type": "size", "targets": 9}, + // Output Size / Records + {"type": "size", "targets": 10}, + // Shuffle Read Size / Records + {"type": "size", "targets": 11}, + // Shuffle Write Size / Records + {"type": "size", "targets": 12}, + // Peak JVM Memory OnHeap / OffHeap + {"visible": false, "targets": 15}, + // Peak Execution Memory OnHeap / OffHeap + {"visible": false, "targets": 16}, + // Peak Storage Memory OnHeap / OffHeap + {"visible": false, "targets": 17}, + // Peak Pool Memory Direct / Mapped + {"visible": false, "targets": 18} ], "deferRender": true, "order": [[0, "asc"]], @@ -746,7 +780,7 @@ $(document).ready(function () { "paging": true, "info": true, "processing": true, - "lengthMenu": [[20, 40, 60, 100, totalTasksToShow], [20, 40, 60, 100, "All"]], + "lengthMenu": [[20, 40, 60, 100, -1], [20, 40, 60, 100, "All"]], "orderMulti": false, "bAutoWidth": false, "ajax": { @@ -762,6 +796,9 @@ $(document).ready(function () { data.numTasks = totalTasksToShow; data.columnIndexToSort = columnIndexToSort; data.columnNameToSort = columnNameToSort; + if (data.length === -1) { + data.length = totalTasksToShow; + } }, "dataSrc": function (jsons) { var jsonStr = JSON.stringify(jsons); diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index 262cee7b58aff..fad28ce333675 100755 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -113,7 +113,7 @@ table.sortable td { box-shadow: inset 1px 0 0 rgba(0,0,0,.15), inset 0 -1px 0 rgba(0,0,0,.15); } -.progress.progress-started { +.progress .progress-bar.progress-started { background-color: #A0DFFF; background-image: -moz-linear-gradient(top, #A4EDFF, #94DDFF); background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#A4EDFF), to(#94DDFF)); @@ -124,7 +124,7 @@ table.sortable td { filter: progid:dximagetransform.microsoft.gradient(startColorstr='#FFA4EDFF', endColorstr='#FF94DDFF', GradientType=0); } -.progress .progress-bar { +.progress .progress-bar.progress-completed { background-color: #3EC0FF; background-image: -moz-linear-gradient(top, #44CBFF, #34B0EE); background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#44CBFF), to(#34B0EE)); diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index bdb768ed5a6ca..f2078f45f69bf 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -736,6 +736,7 @@ private[spark] class ExecutorAllocationManager( stageAttemptToTaskIndices -= stageAttempt stageAttemptToSpeculativeTaskIndices -= stageAttempt stageAttemptToExecutorPlacementHints -= stageAttempt + removeStageFromResourceProfileIfUnused(stageAttempt) // Update the executor placement hints updateExecutorPlacementHints() @@ -780,20 +781,7 @@ private[spark] class ExecutorAllocationManager( stageAttemptToNumRunningTask(stageAttempt) -= 1 if (stageAttemptToNumRunningTask(stageAttempt) == 0) { stageAttemptToNumRunningTask -= stageAttempt - if (!stageAttemptToNumTasks.contains(stageAttempt)) { - val rpForStage = resourceProfileIdToStageAttempt.filter { case (k, v) => - v.contains(stageAttempt) - }.keys - if (rpForStage.size == 1) { - // be careful about the removal from here due to late tasks, make sure stage is - // really complete and no tasks left - resourceProfileIdToStageAttempt(rpForStage.head) -= stageAttempt - } else { - logWarning(s"Should have exactly one resource profile for stage $stageAttempt," + - s" but have $rpForStage") - } - } - + removeStageFromResourceProfileIfUnused(stageAttempt) } } if (taskEnd.taskInfo.speculative) { @@ -858,6 +846,28 @@ private[spark] class ExecutorAllocationManager( allocationManager.synchronized { // Clear unschedulableTaskSets since atleast one task becomes schedulable now unschedulableTaskSets.remove(stageAttempt) + removeStageFromResourceProfileIfUnused(stageAttempt) + } + } + + def removeStageFromResourceProfileIfUnused(stageAttempt: StageAttempt): Unit = { + if (!stageAttemptToNumRunningTask.contains(stageAttempt) && + !stageAttemptToNumTasks.contains(stageAttempt) && + !stageAttemptToNumSpeculativeTasks.contains(stageAttempt) && + !stageAttemptToTaskIndices.contains(stageAttempt) && + !stageAttemptToSpeculativeTaskIndices.contains(stageAttempt) + ) { + val rpForStage = resourceProfileIdToStageAttempt.filter { case (k, v) => + v.contains(stageAttempt) + }.keys + if (rpForStage.size == 1) { + // be careful about the removal from here due to late tasks, make sure stage is + // really complete and no tasks left + resourceProfileIdToStageAttempt(rpForStage.head) -= stageAttempt + } else { + logWarning(s"Should have exactly one resource profile for stage $stageAttempt," + + s" but have $rpForStage") + } } } @@ -931,7 +941,7 @@ private[spark] class ExecutorAllocationManager( val attempts = resourceProfileIdToStageAttempt.getOrElse(rp, Set.empty).toSeq // attempts is a Set, change to Seq so we keep all values attempts.map { attempt => - stageAttemptToNumRunningTask.getOrElseUpdate(attempt, 0) + stageAttemptToNumRunningTask.getOrElse(attempt, 0) }.sum } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 971b6f2b041b3..03c0fbfef3b94 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -17,7 +17,7 @@ package org.apache.spark -import java.io.{ByteArrayInputStream, ObjectInputStream, ObjectOutputStream} +import java.io.{ByteArrayInputStream, IOException, ObjectInputStream, ObjectOutputStream} import java.util.concurrent.{ConcurrentHashMap, LinkedBlockingQueue, ThreadPoolExecutor, TimeUnit} import java.util.concurrent.locks.ReentrantReadWriteLock @@ -100,7 +100,7 @@ private class ShuffleStatus(numPartitions: Int) extends Logging { * broadcast variable in order to keep it from being garbage collected and to allow for it to be * explicitly destroyed later on when the ShuffleMapStage is garbage-collected. */ - private[this] var cachedSerializedBroadcast: Broadcast[Array[Byte]] = _ + private[spark] var cachedSerializedBroadcast: Broadcast[Array[Byte]] = _ /** * Counter tracking the number of partitions that have output. This is a performance optimization @@ -843,7 +843,14 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr if (fetchedStatuses == null) { logInfo("Doing the fetch; tracker endpoint = " + trackerEndpoint) val fetchedBytes = askTracker[Array[Byte]](GetMapOutputStatuses(shuffleId)) - fetchedStatuses = MapOutputTracker.deserializeMapStatuses(fetchedBytes, conf) + try { + fetchedStatuses = MapOutputTracker.deserializeMapStatuses(fetchedBytes, conf) + } catch { + case e: SparkException => + throw new MetadataFetchFailedException(shuffleId, -1, + s"Unable to deserialize broadcasted map statuses for shuffle $shuffleId: " + + e.getCause) + } logInfo("Got the output locations") mapStatuses.put(shuffleId, fetchedStatuses) } @@ -953,13 +960,19 @@ private[spark] object MapOutputTracker extends Logging { case DIRECT => deserializeObject(bytes, 1, bytes.length - 1).asInstanceOf[Array[MapStatus]] case BROADCAST => - // deserialize the Broadcast, pull .value array out of it, and then deserialize that - val bcast = deserializeObject(bytes, 1, bytes.length - 1). - asInstanceOf[Broadcast[Array[Byte]]] - logInfo("Broadcast mapstatuses size = " + bytes.length + - ", actual size = " + bcast.value.length) - // Important - ignore the DIRECT tag ! Start from offset 1 - deserializeObject(bcast.value, 1, bcast.value.length - 1).asInstanceOf[Array[MapStatus]] + try { + // deserialize the Broadcast, pull .value array out of it, and then deserialize that + val bcast = deserializeObject(bytes, 1, bytes.length - 1). + asInstanceOf[Broadcast[Array[Byte]]] + logInfo("Broadcast mapstatuses size = " + bytes.length + + ", actual size = " + bcast.value.length) + // Important - ignore the DIRECT tag ! Start from offset 1 + deserializeObject(bcast.value, 1, bcast.value.length - 1).asInstanceOf[Array[MapStatus]] + } catch { + case e: IOException => + logWarning("Exception encountered during deserializing broadcasted map statuses: ", e) + throw new SparkException("Unable to deserialize broadcasted map statuses", e) + } case _ => throw new IllegalArgumentException("Unexpected byte tag = " + bytes(0)) } } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index a51133f295dc6..9f28232c8bc4a 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1586,11 +1586,7 @@ class SparkContext(config: SparkConf) extends Logging { private def addFile( path: String, recursive: Boolean, addedOnSubmit: Boolean, isArchive: Boolean = false ): Unit = { - val uri = if (!isArchive) { - new Path(path).toUri - } else { - Utils.resolveURI(path) - } + val uri = Utils.resolveURI(path) val schemeCorrectedURI = uri.getScheme match { case null => new File(path).getCanonicalFile.toURI case "local" => @@ -1622,10 +1618,8 @@ class SparkContext(config: SparkConf) extends Logging { env.rpcEnv.fileServer.addFile(new File(uri.getPath)) } else if (uri.getScheme == null) { schemeCorrectedURI.toString - } else if (isArchive) { - uri.toString } else { - path + uri.toString } val timestamp = if (addedOnSubmit) startTime else System.currentTimeMillis @@ -1980,7 +1974,7 @@ class SparkContext(config: SparkConf) extends Logging { // For local paths with backslashes on Windows, URI throws an exception addLocalJarFile(new File(path)) } else { - val uri = new Path(path).toUri + val uri = Utils.resolveURI(path) // SPARK-17650: Make sure this is a valid URL before adding it to the list of dependencies Utils.validateURL(uri) uri.getScheme match { diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 1024d9b5060bc..e35a079746a64 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -73,6 +73,10 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) /** Size of each block. Default value is 4MB. This value is only read by the broadcaster. */ @transient private var blockSize: Int = _ + + /** Whether to generate checksum for blocks or not. */ + private var checksumEnabled: Boolean = false + private def setConf(conf: SparkConf): Unit = { compressionCodec = if (conf.get(config.BROADCAST_COMPRESS)) { Some(CompressionCodec.createCodec(conf)) @@ -90,8 +94,6 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) /** Total number of blocks this broadcast variable contains. */ private val numBlocks: Int = writeBlocks(obj) - /** Whether to generate checksum for blocks or not. */ - private var checksumEnabled: Boolean = false /** The checksum for all the blocks. */ private var checksums: Array[Int] = _ diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index 7c5ab43a9e1b3..15cca6668bf6f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -190,13 +190,15 @@ private class ClientEndpoint( logDebug(s"State of driver $submittedDriverID is ${state.get}, " + s"continue monitoring driver status.") } - } - } - } else { + } + } + } else if (exception.exists(e => Utils.responseFromBackup(e.getMessage))) { + logDebug(s"The status response is reported from a backup spark instance. So, ignored.") + } else { logError(s"ERROR: Cluster master did not recognize $submittedDriverID") System.exit(-1) - } } + } override def receive: PartialFunction[Any, Unit] = { case SubmitDriverResponse(master, success, driverId, message) => diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index acdddbcb89401..5a9c550fc2068 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -956,6 +956,15 @@ private[spark] class SparkSubmit extends Logging { } catch { case t: Throwable => throw findCause(t) + } finally { + if (args.master.startsWith("k8s") && !isShell(args.primaryResource) && + !isSqlShell(args.mainClass) && !isThriftServer(args.mainClass)) { + try { + SparkContext.getActive.foreach(_.stop()) + } catch { + case e: Throwable => logError(s"Failed to close SparkContext: $e") + } + } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala index b4771c80a175f..b21c67a2823af 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala @@ -27,6 +27,7 @@ import org.apache.hadoop.hdfs.DFSInputStream import org.apache.spark.SparkConf import org.apache.spark.deploy.history.EventLogFileWriter.codecName +import org.apache.spark.internal.Logging import org.apache.spark.io.CompressionCodec import org.apache.spark.util.Utils @@ -96,7 +97,7 @@ abstract class EventLogFileReader( def totalSize: Long } -object EventLogFileReader { +object EventLogFileReader extends Logging { // A cache for compression codecs to avoid creating the same codec many times private val codecMap = new ConcurrentHashMap[String, CompressionCodec]() @@ -118,7 +119,12 @@ object EventLogFileReader { if (isSingleEventLog(status)) { Some(new SingleFileEventLogFileReader(fs, status.getPath, Option(status))) } else if (isRollingEventLogs(status)) { - Some(new RollingEventLogFilesFileReader(fs, status.getPath)) + if (fs.listStatus(status.getPath).exists(RollingEventLogFilesWriter.isEventLogFile)) { + Some(new RollingEventLogFilesFileReader(fs, status.getPath)) + } else { + logDebug(s"Rolling event log directory have no event log file at ${status.getPath}") + None + } } else { None } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 5ee5c7dcdc665..8a70c6edc44cf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -967,6 +967,7 @@ private[deploy] class Master( app.driver.send(WorkerRemoved(worker.id, worker.host, msg)) } persistenceEngine.removeWorker(worker) + schedule() } private def relaunchDriver(driver: DriverInfo): Unit = { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index 9e1f753b51e5a..1dda6831b2214 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -309,7 +309,9 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { {UIUtils.formatDate(app.submitDate)} {app.desc.user} {app.state.toString} - {UIUtils.formatDuration(app.duration)} + + {UIUtils.formatDuration(app.duration)} + } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 2e26ccf671d88..40d940778e51b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -83,7 +83,7 @@ private[deploy] class ExecutorRunner( shutdownHook = ShutdownHookManager.addShutdownHook { () => // It's possible that we arrive here before calling `fetchAndRunExecutor`, then `state` will // be `ExecutorState.LAUNCHING`. In this case, we should set `state` to `FAILED`. - if (state == ExecutorState.LAUNCHING) { + if (state == ExecutorState.LAUNCHING || state == ExecutorState.RUNNING) { state = ExecutorState.FAILED } killProcess(Some("Worker shutting down")) } @@ -185,11 +185,11 @@ private[deploy] class ExecutorRunner( // Redirect its stdout and stderr to files val stdout = new File(executorDir, "stdout") - stdoutAppender = FileAppender(process.getInputStream, stdout, conf) + stdoutAppender = FileAppender(process.getInputStream, stdout, conf, true) val stderr = new File(executorDir, "stderr") Files.write(header, stderr, StandardCharsets.UTF_8) - stderrAppender = FileAppender(process.getErrorStream, stderr, conf) + stderrAppender = FileAppender(process.getErrorStream, stderr, conf, true) state = ExecutorState.RUNNING worker.send(ExecutorStateChanged(appId, execId, state, None, None)) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala index 23efcab6caad1..b7a5728dd00d5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala @@ -17,6 +17,8 @@ package org.apache.spark.deploy.worker +import java.util.concurrent.atomic.AtomicBoolean + import org.apache.spark.internal.Logging import org.apache.spark.rpc._ @@ -26,7 +28,10 @@ import org.apache.spark.rpc._ * Provides fate sharing between a worker and its associated child processes. */ private[spark] class WorkerWatcher( - override val rpcEnv: RpcEnv, workerUrl: String, isTesting: Boolean = false) + override val rpcEnv: RpcEnv, + workerUrl: String, + isTesting: Boolean = false, + isChildProcessStopping: AtomicBoolean = new AtomicBoolean(false)) extends RpcEndpoint with Logging { logInfo(s"Connecting to worker $workerUrl") @@ -45,7 +50,17 @@ private[spark] class WorkerWatcher( private val expectedAddress = RpcAddress.fromURIString(workerUrl) private def isWorker(address: RpcAddress) = expectedAddress == address - private def exitNonZero() = if (isTesting) isShutDown = true else System.exit(-1) + private def exitNonZero() = + if (isTesting) { + isShutDown = true + } else if (isChildProcessStopping.compareAndSet(false, true)) { + // SPARK-35714: avoid the duplicate call of `System.exit` to avoid the dead lock. + // Same as SPARK-14180, we should run `System.exit` in a separate thread to avoid + // dead lock since `System.exit` will trigger the shutdown hook of `executor.stop`. + new Thread("WorkerWatcher-exit-executor") { + override def run(): Unit = System.exit(-1) + }.start() + } override def receive: PartialFunction[Any, Unit] = { case e => logWarning(s"Received unexpected message: $e") diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 40a8f7dc59a59..f12f5d996fc5f 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -61,7 +61,7 @@ private[spark] class CoarseGrainedExecutorBackend( private implicit val formats = DefaultFormats - private[this] val stopping = new AtomicBoolean(false) + private[executor] val stopping = new AtomicBoolean(false) var executor: Executor = null @volatile var driver: Option[RpcEndpointRef] = None @@ -454,10 +454,11 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { val env = SparkEnv.createExecutorEnv(driverConf, arguments.executorId, arguments.bindAddress, arguments.hostname, arguments.cores, cfg.ioEncryptionKey, isLocal = false) - env.rpcEnv.setupEndpoint("Executor", - backendCreateFn(env.rpcEnv, arguments, env, cfg.resourceProfile)) + val backend = backendCreateFn(env.rpcEnv, arguments, env, cfg.resourceProfile) + env.rpcEnv.setupEndpoint("Executor", backend) arguments.workerUrl.foreach { url => - env.rpcEnv.setupEndpoint("WorkerWatcher", new WorkerWatcher(env.rpcEnv, url)) + env.rpcEnv.setupEndpoint("WorkerWatcher", + new WorkerWatcher(env.rpcEnv, url, isChildProcessStopping = backend.stopping)) } env.rpcEnv.awaitTermination() } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index e7f1b8f3cf17a..a40a61ecb592a 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -72,7 +72,7 @@ private[spark] class Executor( logInfo(s"Starting executor ID $executorId on host $executorHostname") private val executorShutdown = new AtomicBoolean(false) - ShutdownHookManager.addShutdownHook( + val stopHookReference = ShutdownHookManager.addShutdownHook( () => stop() ) // Application dependencies (added through SparkContext) that we've fetched so far on this node. @@ -312,6 +312,7 @@ private[spark] class Executor( def stop(): Unit = { if (!executorShutdown.getAndSet(true)) { + ShutdownHookManager.removeShutdownHook(stopHookReference) env.metricsSystem.report() try { metricsPoller.stop() @@ -995,7 +996,7 @@ private[spark] class Executor( try { val response = heartbeatReceiverRef.askSync[HeartbeatResponse]( message, new RpcTimeout(HEARTBEAT_INTERVAL_MS.millis, EXECUTOR_HEARTBEAT_INTERVAL.key)) - if (response.reregisterBlockManager) { + if (!executorShutdown.get && response.reregisterBlockManager) { logInfo("Told to re-register on heartbeat") env.blockManager.reregister() } diff --git a/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala b/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala index 80ef757332e43..5682a21e9560d 100644 --- a/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala +++ b/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala @@ -101,7 +101,8 @@ private[spark] class ProcfsMetricsGetter(procfsDir: String = "/proc/") extends L } } - private def computeProcessTree(): Set[Int] = { + // Exposed for testing + private[executor] def computeProcessTree(): Set[Int] = { if (!isAvailable || testing) { return Set() } @@ -159,7 +160,8 @@ private[spark] class ProcfsMetricsGetter(procfsDir: String = "/proc/") extends L } } - def addProcfsMetricsFromOneProcess( + // Exposed for testing + private[executor] def addProcfsMetricsFromOneProcess( allMetrics: ProcfsMetrics, pid: Int): ProcfsMetrics = { @@ -199,7 +201,7 @@ private[spark] class ProcfsMetricsGetter(procfsDir: String = "/proc/") extends L case f: IOException => logWarning("There was a problem with reading" + " the stat file of the process. ", f) - ProcfsMetrics(0, 0, 0, 0, 0, 0) + throw f } } @@ -210,11 +212,16 @@ private[spark] class ProcfsMetricsGetter(procfsDir: String = "/proc/") extends L val pids = computeProcessTree var allMetrics = ProcfsMetrics(0, 0, 0, 0, 0, 0) for (p <- pids) { - allMetrics = addProcfsMetricsFromOneProcess(allMetrics, p) - // if we had an error getting any of the metrics, we don't want to report partial metrics, as - // that would be misleading. - if (!isAvailable) { - return ProcfsMetrics(0, 0, 0, 0, 0, 0) + try { + allMetrics = addProcfsMetricsFromOneProcess(allMetrics, p) + // if we had an error getting any of the metrics, we don't want to + // report partial metrics, as that would be misleading. + if (!isAvailable) { + return ProcfsMetrics(0, 0, 0, 0, 0, 0) + } + } catch { + case _: IOException => + return ProcfsMetrics(0, 0, 0, 0, 0, 0) } } allMetrics diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 954c055ce63f1..6d9bf85f3d0cd 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -1013,7 +1013,7 @@ package object config { "like YARN and event logs.") .version("2.1.2") .regexConf - .createWithDefault("(?i)secret|password|token".r) + .createWithDefault("(?i)secret|password|token|access[.]key".r) private[spark] val STRING_REDACTION_PATTERN = ConfigBuilder("spark.redaction.string.regex") diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala index 30f9a650a69c9..c061d617fce4b 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala @@ -188,13 +188,18 @@ class HadoopMapReduceCommitProtocol( val filesToMove = allAbsPathFiles.foldLeft(Map[String, String]())(_ ++ _) logDebug(s"Committing files staged for absolute locations $filesToMove") + val absParentPaths = filesToMove.values.map(new Path(_).getParent).toSet if (dynamicPartitionOverwrite) { - val absPartitionPaths = filesToMove.values.map(new Path(_).getParent).toSet - logDebug(s"Clean up absolute partition directories for overwriting: $absPartitionPaths") - absPartitionPaths.foreach(fs.delete(_, true)) + logDebug(s"Clean up absolute partition directories for overwriting: $absParentPaths") + absParentPaths.foreach(fs.delete(_, true)) } + logDebug(s"Create absolute parent directories: $absParentPaths") + absParentPaths.foreach(fs.mkdirs) for ((src, dst) <- filesToMove) { - fs.rename(new Path(src), new Path(dst)) + if (!fs.rename(new Path(src), new Path(dst))) { + throw new IOException(s"Failed to rename $src to $dst when committing files staged for " + + s"absolute locations") + } } if (dynamicPartitionOverwrite) { @@ -213,7 +218,11 @@ class HadoopMapReduceCommitProtocol( // a parent that exists, otherwise we may get unexpected result on the rename. fs.mkdirs(finalPartPath.getParent) } - fs.rename(new Path(stagingDir, part), finalPartPath) + val stagingPartPath = new Path(stagingDir, part) + if (!fs.rename(stagingPartPath, finalPartPath)) { + throw new IOException(s"Failed to rename $stagingPartPath to $finalPartPath when " + + s"committing files staged for overwriting dynamic partitions") + } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 608f5a4efdae8..3e14ec839296a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -700,6 +700,35 @@ private[spark] class DAGScheduler( missing.toList } + /** Invoke `.partitions` on the given RDD and all of its ancestors */ + private def eagerlyComputePartitionsForRddAndAncestors(rdd: RDD[_]): Unit = { + val startTime = System.nanoTime + val visitedRdds = new HashSet[RDD[_]] + // We are manually maintaining a stack here to prevent StackOverflowError + // caused by recursively visiting + val waitingForVisit = new ListBuffer[RDD[_]] + waitingForVisit += rdd + + def visit(rdd: RDD[_]): Unit = { + if (!visitedRdds(rdd)) { + visitedRdds += rdd + + // Eagerly compute: + rdd.partitions + + for (dep <- rdd.dependencies) { + waitingForVisit.prepend(dep.rdd) + } + } + } + + while (waitingForVisit.nonEmpty) { + visit(waitingForVisit.remove(0)) + } + logDebug("eagerlyComputePartitionsForRddAndAncestors for RDD %d took %f seconds" + .format(rdd.id, (System.nanoTime - startTime) / 1e9)) + } + /** * Registers the given jobId among the jobs that need the given stage and * all of that stage's ancestors. @@ -809,6 +838,11 @@ private[spark] class DAGScheduler( "Total number of partitions: " + maxPartitions) } + // SPARK-23626: `RDD.getPartitions()` can be slow, so we eagerly compute + // `.partitions` on every RDD in the DAG to ensure that `getPartitions()` + // is evaluated outside of the DAGScheduler's single-threaded event loop: + eagerlyComputePartitionsForRddAndAncestors(rdd) + val jobId = nextJobId.getAndIncrement() if (partitions.isEmpty) { val clonedProperties = Utils.cloneProperties(properties) @@ -892,18 +926,25 @@ private[spark] class DAGScheduler( timeout: Long, properties: Properties): PartialResult[R] = { val jobId = nextJobId.getAndIncrement() + val clonedProperties = Utils.cloneProperties(properties) if (rdd.partitions.isEmpty) { // Return immediately if the job is running 0 tasks val time = clock.getTimeMillis() - listenerBus.post(SparkListenerJobStart(jobId, time, Seq[StageInfo](), properties)) + listenerBus.post(SparkListenerJobStart(jobId, time, Seq[StageInfo](), clonedProperties)) listenerBus.post(SparkListenerJobEnd(jobId, time, JobSucceeded)) return new PartialResult(evaluator.currentResult(), true) } + + // SPARK-23626: `RDD.getPartitions()` can be slow, so we eagerly compute + // `.partitions` on every RDD in the DAG to ensure that `getPartitions()` + // is evaluated outside of the DAGScheduler's single-threaded event loop: + eagerlyComputePartitionsForRddAndAncestors(rdd) + val listener = new ApproximateActionListener(rdd, func, evaluator, timeout) val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] eventProcessLoop.post(JobSubmitted( jobId, rdd, func2, rdd.partitions.indices.toArray, callSite, listener, - Utils.cloneProperties(properties))) + clonedProperties)) listener.awaitResult() // Will throw an exception if the job fails } @@ -931,6 +972,11 @@ private[spark] class DAGScheduler( throw new SparkException("Can't run submitMapStage on RDD with 0 partitions") } + // SPARK-23626: `RDD.getPartitions()` can be slow, so we eagerly compute + // `.partitions` on every RDD in the DAG to ensure that `getPartitions()` + // is evaluated outside of the DAGScheduler's single-threaded event loop: + eagerlyComputePartitionsForRddAndAncestors(rdd) + // We create a JobWaiter with only one "task", which will be marked as complete when the whole // map stage has completed, and will be passed the MapOutputStatistics for that stage. // This makes it easier to avoid race conditions between the user code and the map output @@ -1164,7 +1210,8 @@ private[spark] class DAGScheduler( val stageIds = jobIdToStageIds(jobId).toArray val stageInfos = stageIds.flatMap(id => stageIdToStage.get(id).map(_.latestInfo)) listenerBus.post( - SparkListenerJobStart(job.jobId, jobSubmissionTime, stageInfos, properties)) + SparkListenerJobStart(job.jobId, jobSubmissionTime, stageInfos, + Utils.cloneProperties(properties))) submitStage(finalStage) } @@ -1202,7 +1249,8 @@ private[spark] class DAGScheduler( val stageIds = jobIdToStageIds(jobId).toArray val stageInfos = stageIds.flatMap(id => stageIdToStage.get(id).map(_.latestInfo)) listenerBus.post( - SparkListenerJobStart(job.jobId, jobSubmissionTime, stageInfos, properties)) + SparkListenerJobStart(job.jobId, jobSubmissionTime, stageInfos, + Utils.cloneProperties(properties))) submitStage(finalStage) // If the whole stage has already finished, tell the listener and remove it @@ -1336,7 +1384,8 @@ private[spark] class DAGScheduler( } catch { case NonFatal(e) => stage.makeNewStageAttempt(partitionsToCompute.size) - listenerBus.post(SparkListenerStageSubmitted(stage.latestInfo, properties)) + listenerBus.post(SparkListenerStageSubmitted(stage.latestInfo, + Utils.cloneProperties(properties))) abortStage(stage, s"Task creation failed: $e\n${Utils.exceptionString(e)}", Some(e)) runningStages -= stage return @@ -1350,7 +1399,8 @@ private[spark] class DAGScheduler( if (partitionsToCompute.nonEmpty) { stage.latestInfo.submissionTime = Some(clock.getTimeMillis()) } - listenerBus.post(SparkListenerStageSubmitted(stage.latestInfo, properties)) + listenerBus.post(SparkListenerStageSubmitted(stage.latestInfo, + Utils.cloneProperties(properties))) // TODO: Maybe we can keep the taskBinary in Stage to avoid serializing it multiple times. // Broadcasted binary for the task, used to dispatch tasks to executors. Note that we broadcast diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index ccb5eb115882e..548cab928e140 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -409,8 +409,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp totalCoreCount.addAndGet(-executorInfo.totalCores) totalRegisteredExecutors.addAndGet(-1) scheduler.executorLost(executorId, lossReason) - listenerBus.post( - SparkListenerExecutorRemoved(System.currentTimeMillis(), executorId, reason.toString)) + listenerBus.post(SparkListenerExecutorRemoved( + System.currentTimeMillis(), executorId, lossReason.toString)) case None => // SPARK-15262: If an executor is still alive even after the scheduler has removed // its metadata, we may receive a heartbeat from that executor and tell its block diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index 077b035f3d079..3c134011f11a0 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -87,8 +87,8 @@ private object JavaDeserializationStream { "long" -> classOf[Long], "float" -> classOf[Float], "double" -> classOf[Double], - "void" -> classOf[Void] - ) + "void" -> classOf[Unit]) + } private[spark] class JavaSerializerInstance( diff --git a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala index bc2a0fbc36d5b..30c752960d5da 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -51,15 +51,17 @@ private[spark] class BlockStoreShuffleReader[K, C]( true } val useOldFetchProtocol = conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL) + // SPARK-34790: Fetching continuous blocks in batch is incompatible with io encryption. + val ioEncryption = conf.get(config.IO_ENCRYPTION_ENABLED) val doBatchFetch = shouldBatchFetch && serializerRelocatable && - (!compressed || codecConcatenation) && !useOldFetchProtocol + (!compressed || codecConcatenation) && !useOldFetchProtocol && !ioEncryption if (shouldBatchFetch && !doBatchFetch) { logDebug("The feature tag of continuous shuffle block fetching is set to true, but " + "we can not enable the feature because other conditions are not satisfied. " + s"Shuffle compress: $compressed, serializer relocatable: $serializerRelocatable, " + s"codec concatenation: $codecConcatenation, use old shuffle fetch protocol: " + - s"$useOldFetchProtocol.") + s"$useOldFetchProtocol, io encryption: $ioEncryption.") } doBatchFetch } diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 5f0bb42108c56..7112ef5ddbec9 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -246,7 +246,7 @@ private[spark] class IndexShuffleBlockResolver( throw new FileNotFoundException("Index file is deleted already.") } if (dataFile.exists()) { - List((indexBlockId, indexBlockData), (dataBlockId, dataBlockData)) + List((dataBlockId, dataBlockData), (indexBlockId, indexBlockData)) } else { List((indexBlockId, indexBlockData)) } diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index 52d41cdd72664..4245243c52be7 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -1018,7 +1018,7 @@ private[spark] class AppStatusListener( */ def activeStages(): Seq[v1.StageData] = { liveStages.values.asScala - .filter(_.info.submissionTime.isDefined) + .filter(s => Option(s.info).exists(_.submissionTime.isDefined)) .map(_.toApi()) .toList .sortBy(_.stageId) @@ -1179,7 +1179,7 @@ private[spark] class AppStatusListener( private def getOrCreateStage(info: StageInfo): LiveStage = { val stage = liveStages.computeIfAbsent((info.stageId, info.attemptNumber), - (_: (Int, Int)) => new LiveStage()) + (_: (Int, Int)) => new LiveStage(info)) stage.info = info stage } diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala index 38f1f25f2fcaa..9f0720bb9d9b6 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -393,14 +393,13 @@ private class LiveExecutorStageSummary( } -private class LiveStage extends LiveEntity { +private class LiveStage(var info: StageInfo) extends LiveEntity { import LiveEntityHelpers._ var jobs = Seq[LiveJob]() var jobIds = Set[Int]() - var info: StageInfo = null var status = v1.StageStatus.PENDING var description: Option[String] = None @@ -557,7 +556,7 @@ private class LiveRDDDistribution(exec: LiveExecutor) { def toApi(): v1.RDDDataDistribution = { if (lastUpdate == null) { lastUpdate = new v1.RDDDataDistribution( - weakIntern(exec.hostPort), + weakIntern(if (exec.hostPort != null) exec.hostPort else exec.host), memoryUsed, exec.maxMemory - exec.memoryUsed, diskUsed, diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index 7e3597c19dea4..04d092dd28d56 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -117,12 +117,20 @@ class BlockManagerMasterEndpoint( case _updateBlockInfo @ UpdateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size) => - val isSuccess = updateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size) - context.reply(isSuccess) - // SPARK-30594: we should not post `SparkListenerBlockUpdated` when updateBlockInfo - // returns false since the block info would be updated again later. - if (isSuccess) { - listenerBus.post(SparkListenerBlockUpdated(BlockUpdatedInfo(_updateBlockInfo))) + + @inline def handleResult(success: Boolean): Unit = { + // SPARK-30594: we should not post `SparkListenerBlockUpdated` when updateBlockInfo + // returns false since the block info would be updated again later. + if (success) { + listenerBus.post(SparkListenerBlockUpdated(BlockUpdatedInfo(_updateBlockInfo))) + } + context.reply(success) + } + + if (blockId.isShuffle) { + updateShuffleBlockInfo(blockId, blockManagerId).foreach(handleResult) + } else { + handleResult(updateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size)) } case GetLocations(blockId) => @@ -569,6 +577,33 @@ class BlockManagerMasterEndpoint( id } + private def updateShuffleBlockInfo(blockId: BlockId, blockManagerId: BlockManagerId) + : Future[Boolean] = { + blockId match { + case ShuffleIndexBlockId(shuffleId, mapId, _) => + // SPARK-36782: Invoke `MapOutputTracker.updateMapOutput` within the thread + // `dispatcher-BlockManagerMaster` could lead to the deadlock when + // `MapOutputTracker.serializeOutputStatuses` broadcasts the serialized mapstatues under + // the acquired write lock. The broadcast block would report its status to + // `BlockManagerMasterEndpoint`, while the `BlockManagerMasterEndpoint` is occupied by + // `updateMapOutput` since it's waiting for the write lock. Thus, we use `Future` to call + // `updateMapOutput` in a separate thread to avoid the deadlock. + Future { + // We need to update this at index file because there exists the index-only block + logDebug(s"Received shuffle index block update for ${shuffleId} ${mapId}, updating.") + mapOutputTracker.updateMapOutput(shuffleId, mapId, blockManagerId) + true + } + case ShuffleDataBlockId(shuffleId: Int, mapId: Long, reduceId: Int) => + logDebug(s"Received shuffle data block update for ${shuffleId} ${mapId}, ignore.") + Future.successful(true) + case _ => + logDebug(s"Unexpected shuffle block type ${blockId}" + + s"as ${blockId.getClass().getSimpleName()}") + Future.successful(false) + } + } + private def updateBlockInfo( blockManagerId: BlockManagerId, blockId: BlockId, @@ -577,23 +612,6 @@ class BlockManagerMasterEndpoint( diskSize: Long): Boolean = { logDebug(s"Updating block info on master ${blockId} for ${blockManagerId}") - if (blockId.isShuffle) { - blockId match { - case ShuffleIndexBlockId(shuffleId, mapId, _) => - // Don't update the map output on just the index block - logDebug(s"Received shuffle index block update for ${shuffleId} ${mapId}, ignoring.") - return true - case ShuffleDataBlockId(shuffleId: Int, mapId: Long, reduceId: Int) => - logDebug(s"Received shuffle data block update for ${shuffleId} ${mapId}, updating.") - mapOutputTracker.updateMapOutput(shuffleId, mapId, blockManagerId) - return true - case _ => - logDebug(s"Unexpected shuffle block type ${blockId}" + - s"as ${blockId.getClass().getSimpleName()}") - return false - } - } - if (!blockManagerInfo.contains(blockManagerId)) { if (blockManagerId.isDriver && !isLocal) { // We intentionally do not register the master (except in local mode), diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala index e55c09274cd9a..38b845eb5ebeb 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala @@ -17,7 +17,7 @@ package org.apache.spark.storage -import java.io.{BufferedOutputStream, File, FileOutputStream, OutputStream} +import java.io.{BufferedOutputStream, File, FileOutputStream, IOException, OutputStream} import java.nio.channels.{ClosedByInterruptException, FileChannel} import org.apache.spark.internal.Logging @@ -62,7 +62,16 @@ private[spark] class DiskBlockObjectWriter( } def manualClose(): Unit = { - super.close() + try { + super.close() + } catch { + // The output stream may have been closed when the task thread is interrupted, then we + // get IOException when flushing the buffered data. We should catch and log the exception + // to ensure the revertPartialWritesAndClose() function doesn't throw an exception. + case e: IOException => + logError("Exception occurred while manually close the output stream to file " + + file + ", " + e.getMessage) + } } } diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index fa4e46590aa5e..0c37a5b21a870 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -317,7 +317,10 @@ final class ShuffleBlockFetcherIterator( hostLocalBlockBytes += mergedBlockInfos.map(_.size).sum } else { remoteBlockBytes += blockInfos.map(_._2).sum - collectFetchRequests(address, blockInfos, collectedRemoteRequests) + val (_, timeCost) = Utils.timeTakenMs[Unit] { + collectFetchRequests(address, blockInfos, collectedRemoteRequests) + } + logDebug(s"Collected remote fetch requests for $address in $timeCost ms") } } val numRemoteBlocks = collectedRemoteRequests.map(_.blocks.size).sum @@ -345,10 +348,10 @@ final class ShuffleBlockFetcherIterator( curBlocks: Seq[FetchBlockInfo], address: BlockManagerId, isLast: Boolean, - collectedRemoteRequests: ArrayBuffer[FetchRequest]): Seq[FetchBlockInfo] = { + collectedRemoteRequests: ArrayBuffer[FetchRequest]): ArrayBuffer[FetchBlockInfo] = { val mergedBlocks = mergeContinuousShuffleBlockIdsIfNeeded(curBlocks, doBatchFetch) numBlocksToFetch += mergedBlocks.size - var retBlocks = Seq.empty[FetchBlockInfo] + val retBlocks = new ArrayBuffer[FetchBlockInfo] if (mergedBlocks.length <= maxBlocksInFlightPerAddress) { collectedRemoteRequests += createFetchRequest(mergedBlocks, address) } else { @@ -358,7 +361,7 @@ final class ShuffleBlockFetcherIterator( } else { // The last group does not exceed `maxBlocksInFlightPerAddress`. Put it back // to `curBlocks`. - retBlocks = blocks + retBlocks ++= blocks numBlocksToFetch -= blocks.size } } @@ -372,26 +375,24 @@ final class ShuffleBlockFetcherIterator( collectedRemoteRequests: ArrayBuffer[FetchRequest]): Unit = { val iterator = blockInfos.iterator var curRequestSize = 0L - var curBlocks = Seq.empty[FetchBlockInfo] + var curBlocks = new ArrayBuffer[FetchBlockInfo]() while (iterator.hasNext) { val (blockId, size, mapIndex) = iterator.next() assertPositiveBlockSize(blockId, size) - curBlocks = curBlocks ++ Seq(FetchBlockInfo(blockId, size, mapIndex)) + curBlocks += FetchBlockInfo(blockId, size, mapIndex) curRequestSize += size // For batch fetch, the actual block in flight should count for merged block. val mayExceedsMaxBlocks = !doBatchFetch && curBlocks.size >= maxBlocksInFlightPerAddress if (curRequestSize >= targetRemoteRequestSize || mayExceedsMaxBlocks) { - curBlocks = createFetchRequests(curBlocks, address, isLast = false, + curBlocks = createFetchRequests(curBlocks.toSeq, address, isLast = false, collectedRemoteRequests) curRequestSize = curBlocks.map(_.size).sum } } // Add in the final request if (curBlocks.nonEmpty) { - curBlocks = createFetchRequests(curBlocks, address, isLast = true, - collectedRemoteRequests) - curRequestSize = curBlocks.map(_.size).sum + createFetchRequests(curBlocks.toSeq, address, isLast = true, collectedRemoteRequests) } } @@ -889,7 +890,7 @@ object ShuffleBlockFetcherIterator { blocks: Seq[FetchBlockInfo], doBatchFetch: Boolean): Seq[FetchBlockInfo] = { val result = if (doBatchFetch) { - var curBlocks = new ArrayBuffer[FetchBlockInfo] + val curBlocks = new ArrayBuffer[FetchBlockInfo] val mergedBlockInfo = new ArrayBuffer[FetchBlockInfo] def mergeFetchBlockInfo(toBeMerged: ArrayBuffer[FetchBlockInfo]): FetchBlockInfo = { diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 5e3406037a72b..05b6fea38a9ab 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -460,13 +460,14 @@ private[spark] object UIUtils extends Logging { skipped: Int, reasonToNumKilled: Map[String, Int], total: Int): Seq[Node] = { - val ratio = if (total == 0) 100.0 else (completed.toDouble/total)*100 + val ratio = if (total == 0) 100.0 else (completed.toDouble / total) * 100 val completeWidth = "width: %s%%".format(ratio) // started + completed can be > total when there are speculative tasks val boundedStarted = math.min(started, total - completed) - val startWidth = "width: %s%%".format((boundedStarted.toDouble/total)*100) + val startRatio = if (total == 0) 0.0 else (boundedStarted.toDouble / total) * 100 + val startWidth = "width: %s%%".format(startRatio) -
0) s"progress progress-started" else s"progress" }> +
{completed}/{total} { if (failed == 0 && skipped == 0 && started > 0) s"($started running)" } @@ -477,7 +478,8 @@ private[spark] object UIUtils extends Logging { } } -
+
+
} diff --git a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala index 1453840b834f2..e5a8146447076 100644 --- a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala +++ b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala @@ -155,6 +155,10 @@ abstract class AccumulatorV2[IN, OUT] extends Serializable { */ def value: OUT + // Serialize the buffer of this accumulator before sending back this accumulator to the driver. + // By default this method does nothing. + protected def withBufferSerialized(): AccumulatorV2[IN, OUT] = this + // Called by Java when serializing an object final protected def writeReplace(): Any = { if (atDriverSide) { @@ -177,7 +181,7 @@ abstract class AccumulatorV2[IN, OUT] extends Serializable { } copyAcc } else { - this + withBufferSerialized() } } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index fa38903e423b3..b94549f74f7d4 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2072,6 +2072,17 @@ private[spark] object Utils extends Logging { } } + /** Check whether a path is an absolute URI. */ + def isAbsoluteURI(path: String): Boolean = { + try { + val uri = new URI(path: String) + uri.isAbsolute + } catch { + case _: URISyntaxException => + false + } + } + /** Return all non-local paths from a comma-separated list of paths. */ def nonLocalPaths(paths: String, testWindows: Boolean = false): Array[String] = { val windows = isWindows || testWindows @@ -2867,6 +2878,34 @@ private[spark] object Utils extends Logging { Hex.encodeHexString(secretBytes) } + /** + * Returns true if and only if the underlying class is a member class. + * + * Note: jdk8u throws a "Malformed class name" error if a given class is a deeply-nested + * inner class (See SPARK-34607 for details). This issue has already been fixed in jdk9+, so + * we can remove this helper method safely if we drop the support of jdk8u. + */ + def isMemberClass(cls: Class[_]): Boolean = { + try { + cls.isMemberClass + } catch { + case _: InternalError => + // We emulate jdk8u `Class.isMemberClass` below: + // public boolean isMemberClass() { + // return getSimpleBinaryName() != null && !isLocalOrAnonymousClass(); + // } + // `getSimpleBinaryName()` returns null if a given class is a top-level class, + // so we replace it with `cls.getEnclosingClass != null`. The second condition checks + // if a given class is not a local or an anonymous class, so we replace it with + // `cls.getEnclosingMethod == null` because `cls.getEnclosingMethod()` return a value + // only in either case (JVM Spec 4.8.6). + // + // Note: The newer jdk evaluates `!isLocalOrAnonymousClass()` first, + // we reorder the conditions to follow it. + cls.getEnclosingMethod == null && cls.getEnclosingClass != null + } + } + /** * Safer than Class obj's getSimpleName which may throw Malformed class name error in scala. * This method mimics scalatest's getSimpleNameOfAnObjectsClass. @@ -2979,6 +3018,9 @@ private[spark] object Utils extends Logging { /** Create a new properties object with the same values as `props` */ def cloneProperties(props: Properties): Properties = { + if (props == null) { + return props + } val resultProps = new Properties() props.forEach((k, v) => resultProps.put(k, v)) resultProps diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index dc39170ecf382..6e95bce8107a8 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -307,14 +307,13 @@ private[spark] class ExternalSorter[K, V, C]( } if (objectsWritten > 0) { flush() + writer.close() } else { writer.revertPartialWritesAndClose() } success = true } finally { - if (success) { - writer.close() - } else { + if (!success) { // This code path only happens if an exception was thrown above before we set success; // close our stuff and let the exception be thrown further writer.revertPartialWritesAndClose() diff --git a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStream.scala b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStream.scala index a625b3289538a..34d36655a6069 100644 --- a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStream.scala +++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStream.scala @@ -48,7 +48,7 @@ private[spark] class ChunkedByteBufferOutputStream( * This can also never be 0. */ private[this] var position = chunkSize - private[this] var _size = 0 + private[this] var _size = 0L private[this] var closed: Boolean = false def size: Long = _size @@ -120,4 +120,5 @@ private[spark] class ChunkedByteBufferOutputStream( new ChunkedByteBuffer(ret) } } + } diff --git a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala index 7107be25eb505..2243239dce6fd 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala @@ -26,8 +26,12 @@ import org.apache.spark.util.{IntParam, Utils} /** * Continuously appends the data from an input stream into the given file. */ -private[spark] class FileAppender(inputStream: InputStream, file: File, bufferSize: Int = 8192) - extends Logging { +private[spark] class FileAppender( + inputStream: InputStream, + file: File, + bufferSize: Int = 8192, + closeStreams: Boolean = false +) extends Logging { @volatile private var outputStream: FileOutputStream = null @volatile private var markedForStop = false // has the appender been asked to stopped @@ -76,7 +80,13 @@ private[spark] class FileAppender(inputStream: InputStream, file: File, bufferSi } } } { - closeFile() + try { + if (closeStreams) { + inputStream.close() + } + } finally { + closeFile() + } } } catch { case e: Exception => @@ -113,7 +123,12 @@ private[spark] class FileAppender(inputStream: InputStream, file: File, bufferSi private[spark] object FileAppender extends Logging { /** Create the right appender based on Spark configuration */ - def apply(inputStream: InputStream, file: File, conf: SparkConf): FileAppender = { + def apply( + inputStream: InputStream, + file: File, + conf: SparkConf, + closeStreams: Boolean = false + ) : FileAppender = { val rollingStrategy = conf.get(config.EXECUTOR_LOGS_ROLLING_STRATEGY) val rollingSizeBytes = conf.get(config.EXECUTOR_LOGS_ROLLING_MAX_SIZE) @@ -141,9 +156,10 @@ private[spark] object FileAppender extends Logging { validatedParams.map { case (interval, pattern) => new RollingFileAppender( - inputStream, file, new TimeBasedRollingPolicy(interval, pattern), conf) + inputStream, file, new TimeBasedRollingPolicy(interval, pattern), conf, + closeStreams = closeStreams) }.getOrElse { - new FileAppender(inputStream, file) + new FileAppender(inputStream, file, closeStreams = closeStreams) } } @@ -151,17 +167,18 @@ private[spark] object FileAppender extends Logging { rollingSizeBytes match { case IntParam(bytes) => logInfo(s"Rolling executor logs enabled for $file with rolling every $bytes bytes") - new RollingFileAppender(inputStream, file, new SizeBasedRollingPolicy(bytes), conf) + new RollingFileAppender( + inputStream, file, new SizeBasedRollingPolicy(bytes), conf, closeStreams = closeStreams) case _ => logWarning( s"Illegal size [$rollingSizeBytes] for rolling executor logs, rolling logs not enabled") - new FileAppender(inputStream, file) + new FileAppender(inputStream, file, closeStreams = closeStreams) } } rollingStrategy match { case "" => - new FileAppender(inputStream, file) + new FileAppender(inputStream, file, closeStreams = closeStreams) case "time" => createTimeBasedAppender() case "size" => @@ -170,7 +187,7 @@ private[spark] object FileAppender extends Logging { logWarning( s"Illegal strategy [$rollingStrategy] for rolling executor logs, " + s"rolling logs not enabled") - new FileAppender(inputStream, file) + new FileAppender(inputStream, file, closeStreams = closeStreams) } } } diff --git a/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala index b73f422649312..68a59232c7a96 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala @@ -36,14 +36,16 @@ import org.apache.spark.internal.config * @param rollingPolicy Policy based on which files will be rolled over. * @param conf SparkConf that is used to pass on extra configurations * @param bufferSize Optional buffer size. Used mainly for testing. + * @param closeStreams Option flag: whether to close the inputStream at the end. */ private[spark] class RollingFileAppender( inputStream: InputStream, activeFile: File, val rollingPolicy: RollingPolicy, conf: SparkConf, - bufferSize: Int = RollingFileAppender.DEFAULT_BUFFER_SIZE - ) extends FileAppender(inputStream, activeFile, bufferSize) { + bufferSize: Int = RollingFileAppender.DEFAULT_BUFFER_SIZE, + closeStreams: Boolean = false + ) extends FileAppender(inputStream, activeFile, bufferSize, closeStreams) { private val maxRetainedFiles = conf.get(config.EXECUTOR_LOGS_ROLLING_MAX_RETAINED_FILES) private val enableCompression = conf.get(config.EXECUTOR_LOGS_ROLLING_ENABLE_COMPRESSION) diff --git a/core/src/test/scala/org/apache/spark/DebugFilesystem.scala b/core/src/test/scala/org/apache/spark/DebugFilesystem.scala index 1d3e28b39548f..8f220801f41e3 100644 --- a/core/src/test/scala/org/apache/spark/DebugFilesystem.scala +++ b/core/src/test/scala/org/apache/spark/DebugFilesystem.scala @@ -57,8 +57,14 @@ object DebugFilesystem extends Logging { } /** - * DebugFilesystem wraps file open calls to track all open connections. This can be used in tests - * to check that connections are not leaked. + * DebugFilesystem wraps + * 1) file open calls to track all open connections. This can be used in tests to check that + * connections are not leaked; + * 2) rename calls to return false when destination's parent path does not exist. When + * destination parent does not exist, LocalFileSystem uses FileUtil#copy to copy the + * file and returns true if succeed, while many other hadoop file systems (e.g. HDFS, S3A) + * return false without renaming any file. This helps to test that Spark can work with the + * latter file systems. */ // TODO(ekl) we should consider always interposing this to expose num open conns as a metric class DebugFilesystem extends LocalFileSystem { @@ -120,4 +126,8 @@ class DebugFilesystem extends LocalFileSystem { override def hashCode(): Int = wrapped.hashCode() } } + + override def rename(src: Path, dst: Path): Boolean = { + exists(dst.getParent) && super.rename(src, dst) + } } diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index b5b68f639ffc9..33e1113774663 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -333,4 +333,45 @@ class MapOutputTrackerSuite extends SparkFunSuite { rpcEnv.shutdown() } + test("SPARK-34939: remote fetch using broadcast if broadcasted value is destroyed") { + val newConf = new SparkConf + newConf.set(RPC_MESSAGE_MAX_SIZE, 1) + newConf.set(RPC_ASK_TIMEOUT, "1") // Fail fast + newConf.set(SHUFFLE_MAPOUTPUT_MIN_SIZE_FOR_BROADCAST, 10240L) // 10 KiB << 1MiB framesize + + // needs TorrentBroadcast so need a SparkContext + withSpark(new SparkContext("local", "MapOutputTrackerSuite", newConf)) { sc => + val masterTracker = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] + val rpcEnv = sc.env.rpcEnv + val masterEndpoint = new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, newConf) + rpcEnv.stop(masterTracker.trackerEndpoint) + rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, masterEndpoint) + + masterTracker.registerShuffle(20, 100) + (0 until 100).foreach { i => + masterTracker.registerMapOutput(20, i, new CompressedMapStatus( + BlockManagerId("999", "mps", 1000), Array.fill[Long](4000000)(0), 5)) + } + + val mapWorkerRpcEnv = createRpcEnv("spark-worker", "localhost", 0, new SecurityManager(conf)) + val mapWorkerTracker = new MapOutputTrackerWorker(conf) + mapWorkerTracker.trackerEndpoint = + mapWorkerRpcEnv.setupEndpointRef(rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) + + val fetchedBytes = mapWorkerTracker.trackerEndpoint + .askSync[Array[Byte]](GetMapOutputStatuses(20)) + assert(fetchedBytes(0) == 1) + + // Normally `unregisterMapOutput` triggers the destroy of broadcasted value. + // But the timing of destroying broadcasted value is indeterminate, we manually destroy + // it by blocking. + masterTracker.shuffleStatuses.get(20).foreach { shuffleStatus => + shuffleStatus.cachedSerializedBroadcast.destroy(true) + } + val err = intercept[SparkException] { + MapOutputTracker.deserializeMapStatuses(fetchedBytes, conf) + } + assert(err.getMessage.contains("Unable to deserialize broadcasted map statuses")) + } + } } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 0c0a9b86ac603..c4bcccfd1d45a 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -1069,6 +1069,46 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu assert(sc.hadoopConfiguration.get(bufferKey).toInt === 65536, "spark configs have higher priority than spark.hadoop configs") } + + test("SPARK-34225: addFile/addJar shouldn't further encode URI if a URI form string is passed") { + withTempDir { dir => + val jar1 = File.createTempFile("testprefix", "test jar.jar", dir) + val jarUrl1 = jar1.toURI.toString + val file1 = File.createTempFile("testprefix", "test file.txt", dir) + val fileUrl1 = file1.toURI.toString + val jar2 = File.createTempFile("testprefix", "test %20jar.jar", dir) + val file2 = File.createTempFile("testprefix", "test %20file.txt", dir) + + try { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + sc.addJar(jarUrl1) + sc.addFile(fileUrl1) + sc.addJar(jar2.toString) + sc.addFile(file2.toString) + sc.parallelize(Array(1), 1).map { x => + val gottenJar1 = new File(SparkFiles.get(jar1.getName)) + if (!gottenJar1.exists()) { + throw new SparkException("file doesn't exist : " + jar1) + } + val gottenFile1 = new File(SparkFiles.get(file1.getName)) + if (!gottenFile1.exists()) { + throw new SparkException("file doesn't exist : " + file1) + } + val gottenJar2 = new File(SparkFiles.get(jar2.getName)) + if (!gottenJar2.exists()) { + throw new SparkException("file doesn't exist : " + jar2) + } + val gottenFile2 = new File(SparkFiles.get(file2.getName)) + if (!gottenFile2.exists()) { + throw new SparkException("file doesn't exist : " + file2) + } + x + }.collect() + } finally { + sc.stop() + } + } + } } object SparkContextSuite { diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala index 8eab2da1a37b7..0bdc015349036 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala @@ -77,6 +77,8 @@ abstract class EventLogFileReadersSuite extends SparkFunSuite with LocalSparkCon } } else { fileSystem.mkdirs(path) + fileSystem.create(getAppStatusFilePath(path, "app", None, true)) + fileSystem.create(getEventLogFilePath(path, "app", None, 1, None)) } val reader = EventLogFileReader(fileSystem, path) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 3caa0a8ba7d4e..4aaafa353731b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -1524,6 +1524,35 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { } } + test("SPARK-36354: EventLogFileReader should skip rolling event log directories with no logs") { + withTempDir { dir => + val conf = createTestConf(true) + conf.set(HISTORY_LOG_DIR, dir.getAbsolutePath) + val hadoopConf = SparkHadoopUtil.newConfiguration(conf) + val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) + + val provider = new FsHistoryProvider(conf) + + val writer = new RollingEventLogFilesWriter("app", None, dir.toURI, conf, hadoopConf) + writer.start() + + writeEventsToRollingWriter(writer, Seq( + SparkListenerApplicationStart("app", Some("app"), 0, "user", None), + SparkListenerJobStart(1, 0, Seq.empty)), rollFile = false) + provider.checkForLogs() + provider.cleanLogs() + assert(dir.listFiles().size === 1) + assert(provider.getListing.length === 1) + + // Manually delete event log files and create event log file reader + val eventLogDir = dir.listFiles().head + eventLogDir.listFiles + .filter(f => RollingEventLogFilesWriter.isEventLogFile(f.getName)) + .foreach(f => f.delete()) + EventLogFileReader(fs, new Path(eventLogDir.getAbsolutePath)).map(_.lastIndex) + } + } + test("SPARK-33215: check ui view permissions without retrieving ui") { val conf = createTestConf() .set(HISTORY_SERVER_UI_ACLS_ENABLE, true) diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index 97ffb36062dbc..a237447b0fa2d 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -270,6 +270,17 @@ class ExecutorSuite extends SparkFunSuite heartbeatZeroAccumulatorUpdateTest(false) } + private def withMockHeartbeatReceiverRef(executor: Executor) + (func: RpcEndpointRef => Unit): Unit = { + val executorClass = classOf[Executor] + val mockReceiverRef = mock[RpcEndpointRef] + val receiverRef = executorClass.getDeclaredField("heartbeatReceiverRef") + receiverRef.setAccessible(true) + receiverRef.set(executor, mockReceiverRef) + + func(mockReceiverRef) + } + private def withHeartbeatExecutor(confs: (String, String)*) (f: (Executor, ArrayBuffer[Heartbeat]) => Unit): Unit = { val conf = new SparkConf @@ -277,22 +288,18 @@ class ExecutorSuite extends SparkFunSuite val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) withExecutor("id", "localhost", SparkEnv.get) { executor => - val executorClass = classOf[Executor] - - // Save all heartbeats sent into an ArrayBuffer for verification - val heartbeats = ArrayBuffer[Heartbeat]() - val mockReceiver = mock[RpcEndpointRef] - when(mockReceiver.askSync(any[Heartbeat], any[RpcTimeout])(any)) - .thenAnswer((invocation: InvocationOnMock) => { - val args = invocation.getArguments() - heartbeats += args(0).asInstanceOf[Heartbeat] - HeartbeatResponse(false) - }) - val receiverRef = executorClass.getDeclaredField("heartbeatReceiverRef") - receiverRef.setAccessible(true) - receiverRef.set(executor, mockReceiver) + withMockHeartbeatReceiverRef(executor) { mockReceiverRef => + // Save all heartbeats sent into an ArrayBuffer for verification + val heartbeats = ArrayBuffer[Heartbeat]() + when(mockReceiverRef.askSync(any[Heartbeat], any[RpcTimeout])(any)) + .thenAnswer((invocation: InvocationOnMock) => { + val args = invocation.getArguments() + heartbeats += args(0).asInstanceOf[Heartbeat] + HeartbeatResponse(false) + }) - f(executor, heartbeats) + f(executor, heartbeats) + } } } @@ -416,6 +423,35 @@ class ExecutorSuite extends SparkFunSuite assert(taskMetrics.getMetricValue("JVMHeapMemory") > 0) } + test("SPARK-34949: do not re-register BlockManager when executor is shutting down") { + val reregisterInvoked = new AtomicBoolean(false) + val mockBlockManager = mock[BlockManager] + when(mockBlockManager.reregister()).thenAnswer { (_: InvocationOnMock) => + reregisterInvoked.getAndSet(true) + } + val conf = new SparkConf(false).setAppName("test").setMaster("local[2]") + val mockEnv = createMockEnv(conf, new JavaSerializer(conf)) + when(mockEnv.blockManager).thenReturn(mockBlockManager) + + withExecutor("id", "localhost", mockEnv) { executor => + withMockHeartbeatReceiverRef(executor) { mockReceiverRef => + when(mockReceiverRef.askSync(any[Heartbeat], any[RpcTimeout])(any)).thenAnswer { + (_: InvocationOnMock) => HeartbeatResponse(reregisterBlockManager = true) + } + val reportHeartbeat = PrivateMethod[Unit](Symbol("reportHeartBeat")) + executor.invokePrivate(reportHeartbeat()) + assert(reregisterInvoked.get(), "BlockManager.reregister should be invoked " + + "on HeartbeatResponse(reregisterBlockManager = true) when executor is not shutting down") + + reregisterInvoked.getAndSet(false) + executor.stop() + executor.invokePrivate(reportHeartbeat()) + assert(!reregisterInvoked.get(), + "BlockManager.reregister should not be invoked when executor is shutting down") + } + } + } + test("SPARK-33587: isFatalError") { def errorInThreadPool(e: => Throwable): Throwable = { intercept[Throwable] { diff --git a/core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala b/core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala index 9836697e1647c..ff0374da1bcfe 100644 --- a/core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.executor +import org.mockito.Mockito.{spy, when} + import org.apache.spark.SparkFunSuite @@ -38,4 +40,26 @@ class ProcfsMetricsGetterSuite extends SparkFunSuite { assert(r.jvmVmemTotal == 4769947648L) assert(r.jvmRSSTotal == 262610944) } + + test("SPARK-34845: partial metrics shouldn't be returned") { + val p = new ProcfsMetricsGetter(getTestResourcePath("ProcfsMetrics")) + val mockedP = spy(p) + + var ptree: Set[Int] = Set(26109, 22763) + when(mockedP.computeProcessTree).thenReturn(ptree) + var r = mockedP.computeAllMetrics + assert(r.jvmVmemTotal == 4769947648L) + assert(r.jvmRSSTotal == 262610944) + assert(r.pythonVmemTotal == 360595456) + assert(r.pythonRSSTotal == 7831552) + + // proc file of pid 22764 doesn't exist, so partial metrics shouldn't be returned + ptree = Set(26109, 22764, 22763) + when(mockedP.computeProcessTree).thenReturn(ptree) + r = mockedP.computeAllMetrics + assert(r.jvmVmemTotal == 0) + assert(r.jvmRSSTotal == 0) + assert(r.pythonVmemTotal == 0) + assert(r.pythonRSSTotal == 0) + } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/HealthTrackerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/HealthTrackerIntegrationSuite.scala index 29a8f4be8b72b..fd05ff9dfe8ad 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/HealthTrackerIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/HealthTrackerIntegrationSuite.scala @@ -112,7 +112,7 @@ class HealthTrackerIntegrationSuite extends SchedulerIntegrationSuite[MultiExecu backend.taskFailed(taskDescription, new RuntimeException("test task failure")) } withBackend(runBackend _) { - val jobFuture = submit(new MockRDD(sc, 10, Nil), (0 until 10).toArray) + val jobFuture = submit(new MockRDD(sc, 10, Nil, Nil), (0 until 10).toArray) awaitJobTermination(jobFuture, duration) val pattern = ( s"""|Aborting TaskSet 0.0 because task .* @@ -150,7 +150,7 @@ class MockRDDWithLocalityPrefs( sc: SparkContext, numPartitions: Int, shuffleDeps: Seq[ShuffleDependency[Int, Int, Nothing]], - val preferredLoc: String) extends MockRDD(sc, numPartitions, shuffleDeps) { + val preferredLoc: String) extends MockRDD(sc, numPartitions, shuffleDeps, Nil) { override def getPreferredLocations(split: Partition): Seq[String] = { Seq(preferredLoc) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala index 88d2868b957f9..874abce68c11f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler import java.util.Properties -import java.util.concurrent.{TimeoutException, TimeUnit} +import java.util.concurrent.{CountDownLatch, TimeoutException, TimeUnit} import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference} import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} @@ -205,7 +205,13 @@ abstract class SchedulerIntegrationSuite[T <: MockBackend: ClassTag] extends Spa def shuffle(nParts: Int, input: MockRDD): MockRDD = { val partitioner = new HashPartitioner(nParts) val shuffleDep = new ShuffleDependency[Int, Int, Nothing](input, partitioner) - new MockRDD(sc, nParts, List(shuffleDep)) + new MockRDD(sc, nParts, List(shuffleDep), Nil) + } + + /** models a one-to-one dependency within a stage, like a map or filter */ + def oneToOne(input: MockRDD): MockRDD = { + val dep = new OneToOneDependency[(Int, Int)](input) + new MockRDD(sc, input.numPartitions, Nil, Seq(dep)) } /** models a stage boundary with multiple dependencies, like a join */ @@ -214,7 +220,7 @@ abstract class SchedulerIntegrationSuite[T <: MockBackend: ClassTag] extends Spa val shuffleDeps = inputs.map { inputRDD => new ShuffleDependency[Int, Int, Nothing](inputRDD, partitioner) } - new MockRDD(sc, nParts, shuffleDeps) + new MockRDD(sc, nParts, shuffleDeps, Nil) } val backendException = new AtomicReference[Exception](null) @@ -449,10 +455,11 @@ case class ExecutorTaskStatus(host: String, executorId: String, var freeCores: I class MockRDD( sc: SparkContext, val numPartitions: Int, - val shuffleDeps: Seq[ShuffleDependency[Int, Int, Nothing]] -) extends RDD[(Int, Int)](sc, shuffleDeps) with Serializable { + val shuffleDeps: Seq[ShuffleDependency[Int, Int, Nothing]], + val oneToOneDeps: Seq[OneToOneDependency[(Int, Int)]] +) extends RDD[(Int, Int)](sc, deps = shuffleDeps ++ oneToOneDeps) with Serializable { - MockRDD.validate(numPartitions, shuffleDeps) + MockRDD.validate(numPartitions, shuffleDeps, oneToOneDeps) override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = throw new RuntimeException("should not be reached") @@ -468,14 +475,25 @@ class MockRDD( object MockRDD extends AssertionsHelper with TripleEquals with Assertions { /** * make sure all the shuffle dependencies have a consistent number of output partitions + * and that one-to-one dependencies have the same partition counts as their parents * (mostly to make sure the test setup makes sense, not that Spark itself would get this wrong) */ - def validate(numPartitions: Int, dependencies: Seq[ShuffleDependency[_, _, _]]): Unit = { - dependencies.foreach { dependency => + def validate( + numPartitions: Int, + shuffleDependencies: Seq[ShuffleDependency[_, _, _]], + oneToOneDependencies: Seq[OneToOneDependency[_]]): Unit = { + shuffleDependencies.foreach { dependency => val partitioner = dependency.partitioner assert(partitioner != null) assert(partitioner.numPartitions === numPartitions) } + oneToOneDependencies.foreach { dependency => + // In order to support the SPARK-23626 testcase, we cast to MockRDD + // and access `numPartitions` instead of just calling `getNumPartitions`: + // `getNumPartitions` would call `getPartitions`, undermining the intention + // of the SPARK-23626 testcase. + assert(dependency.rdd.asInstanceOf[MockRDD].numPartitions === numPartitions) + } } } @@ -539,7 +557,7 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor backend.taskSuccess(taskDescription, 42) } withBackend(runBackend _) { - val jobFuture = submit(new MockRDD(sc, 10, Nil), (0 until 10).toArray) + val jobFuture = submit(new MockRDD(sc, 10, Nil, Nil), (0 until 10).toArray) awaitJobTermination(jobFuture, duration) } assert(results === (0 until 10).map { _ -> 42 }.toMap) @@ -564,7 +582,7 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor } } - val a = new MockRDD(sc, 2, Nil) + val a = new MockRDD(sc, 2, Nil, Nil) val b = shuffle(10, a) val c = shuffle(20, a) val d = join(30, b, c) @@ -604,7 +622,7 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor * (b) we get a second attempt for stage 0 & stage 1 */ testScheduler("job with fetch failure") { - val input = new MockRDD(sc, 2, Nil) + val input = new MockRDD(sc, 2, Nil, Nil) val shuffledRdd = shuffle(10, input) val shuffleId = shuffledRdd.shuffleDeps.head.shuffleId @@ -646,10 +664,88 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor backend.taskFailed(taskDescription, new RuntimeException("test task failure")) } withBackend(runBackend _) { - val jobFuture = submit(new MockRDD(sc, 10, Nil), (0 until 10).toArray) + val jobFuture = submit(new MockRDD(sc, 10, Nil, Nil), (0 until 10).toArray) awaitJobTermination(jobFuture, duration) assert(failure.getMessage.contains("test task failure")) } assertDataStructuresEmpty(noFailure = false) } + + testScheduler("SPARK-23626: RDD with expensive getPartitions() doesn't block scheduler loop") { + // Before SPARK-23626, expensive `RDD.getPartitions()` calls might occur inside of the + // DAGScheduler event loop, causing concurrently-submitted jobs to block. This test case + // reproduces a scenario where that blocking could occur. + + // We'll use latches to simulate an RDD with a slow getPartitions() call. + import MockRDDWithSlowGetPartitions._ + + // DAGScheduler.submitJob calls `.partitions` on the RDD passed to it. + // Therefore to write a proper regression test for SPARK-23626 we must + // ensure that the slow getPartitions() call occurs deeper in the RDD DAG: + val rddWithSlowGetPartitions = oneToOne(new MockRDDWithSlowGetPartitions(sc, 1)) + + // A RDD whose execution should not be blocked by the other RDD's slow getPartitions(): + val simpleRdd = new MockRDD(sc, 1, Nil, Nil) + + getPartitionsShouldNotHaveBeenCalledYet.set(false) + + def runBackend(): Unit = { + val (taskDescription, _) = backend.beginTask() + backend.taskSuccess(taskDescription, 42) + } + + withBackend(runBackend _) { + // Submit a job containing an RDD which will hang in getPartitions() until we release + // the countdown latch: + import scala.concurrent.ExecutionContext.Implicits.global + val slowJobFuture = Future { submit(rddWithSlowGetPartitions, Array(0)) }.flatten + + // Block the current thread until the other thread has started the getPartitions() call: + beginGetPartitionsLatch.await(duration.toSeconds, SECONDS) + + // Submit a concurrent job. This job's execution should not be blocked by the other job: + val fastJobFuture = submit(simpleRdd, Array(0)) + awaitJobTermination(fastJobFuture, duration) + + // The slow job should still be blocked in the getPartitions() call: + assert(!slowJobFuture.isCompleted) + + // Allow it to complete: + endGetPartitionsLatch.countDown() + awaitJobTermination(slowJobFuture, duration) + } + + assertDataStructuresEmpty() + } +} + +/** Helper class used in SPARK-23626 test case */ +private object MockRDDWithSlowGetPartitions { + // Latch for blocking the test execution thread until getPartitions() has been called: + val beginGetPartitionsLatch = new CountDownLatch(1) + + // Latch for blocking the getPartitions() call from completing: + val endGetPartitionsLatch = new CountDownLatch(1) + + // Atomic boolean which is used to fail the test in case getPartitions() is called earlier + // than expected. This guards against false-negatives (e.g. the test passing because + // `.getPartitions()` was called in the test setup before we even submitted a job): + val getPartitionsShouldNotHaveBeenCalledYet = new AtomicBoolean(true) +} + +/** Helper class used in SPARK-23626 test case */ +private class MockRDDWithSlowGetPartitions( + sc: SparkContext, + numPartitions: Int) extends MockRDD(sc, numPartitions, Nil, Nil) { + import MockRDDWithSlowGetPartitions._ + + override def getPartitions: Array[Partition] = { + if (getPartitionsShouldNotHaveBeenCalledYet.get()) { + throw new Exception("getPartitions() should not have been called at this point") + } + beginGetPartitionsLatch.countDown() + val partitions = super.getPartitions + endGetPartitionsLatch.await() + partitions + } } diff --git a/core/src/test/scala/org/apache/spark/serializer/JavaSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/JavaSerializerSuite.scala index 6a6ea42797fb6..03349f8aef4fc 100644 --- a/core/src/test/scala/org/apache/spark/serializer/JavaSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/JavaSerializerSuite.scala @@ -47,5 +47,5 @@ private class ContainsPrimitiveClass extends Serializable { val floatClass = classOf[Float] val booleanClass = classOf[Boolean] val byteClass = classOf[Byte] - val voidClass = classOf[Void] + val voidClass = classOf[Unit] } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala index e461474294f38..7bd1b6421b191 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala @@ -98,12 +98,22 @@ class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalS runDecomTest(true, true, JobEnded) } + test(s"SPARK-36782 not deadlock if MapOutput uses broadcast") { + runDecomTest(false, true, JobEnded, forceMapOutputBroadcast = true) + } + private def runDecomTest( persist: Boolean, shuffle: Boolean, - whenToDecom: String): Unit = { + whenToDecom: String, + forceMapOutputBroadcast: Boolean = false): Unit = { val migrateDuring = whenToDecom != JobEnded val master = s"local-cluster[${numExecs}, 1, 1024]" + val minBroadcastSize = if (forceMapOutputBroadcast) { + 0 + } else { + config.SHUFFLE_MAPOUTPUT_MIN_SIZE_FOR_BROADCAST.defaultValue.get + } val conf = new SparkConf().setAppName("test").setMaster(master) .set(config.DECOMMISSION_ENABLED, true) .set(config.STORAGE_DECOMMISSION_ENABLED, true) @@ -114,6 +124,7 @@ class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalS // Just replicate blocks quickly during testing, there isn't another // workload we need to worry about. .set(config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL, 10L) + .set(config.SHUFFLE_MAPOUTPUT_MIN_SIZE_FOR_BROADCAST, minBroadcastSize) if (whenToDecom == TaskStarted) { // We are using accumulators below, make sure those are reported frequently. diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 09678c77e10b3..cd319daccc0db 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -1975,6 +1975,54 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } } + test("SPARK-35589: test migration of index-only shuffle blocks during decommissioning") { + val shuffleManager1 = makeSortShuffleManager() + val bm1 = makeBlockManager(3500, "exec1", shuffleManager = shuffleManager1) + shuffleManager1.shuffleBlockResolver._blockManager = bm1 + + val shuffleManager2 = makeSortShuffleManager() + val bm2 = makeBlockManager(3500, "exec2", shuffleManager = shuffleManager2) + shuffleManager2.shuffleBlockResolver._blockManager = bm2 + + val blockSize = 5 + val shuffleDataBlockContent = Array[Byte](0, 1, 2, 3, 4) + val shuffleData = ShuffleDataBlockId(0, 0, 0) + Files.write(bm1.diskBlockManager.getFile(shuffleData).toPath(), shuffleDataBlockContent) + val shuffleIndexBlockContent = Array[Byte](5, 6, 7, 8, 9) + val shuffleIndex = ShuffleIndexBlockId(0, 0, 0) + val shuffleIndexOnly = ShuffleIndexBlockId(0, 1, 0) + Files.write(bm1.diskBlockManager.getFile(shuffleIndex).toPath(), shuffleIndexBlockContent) + Files.write(bm1.diskBlockManager.getFile(shuffleIndexOnly).toPath(), shuffleIndexBlockContent) + + mapOutputTracker.registerShuffle(0, 2) + val decomManager = new BlockManagerDecommissioner(conf, bm1) + try { + mapOutputTracker.registerMapOutput(0, 0, MapStatus(bm1.blockManagerId, Array(blockSize), 0)) + mapOutputTracker.registerMapOutput(0, 1, MapStatus(bm1.blockManagerId, Array(blockSize), 1)) + assert(mapOutputTracker.shuffleStatuses(0).mapStatuses(0).location === bm1.blockManagerId) + assert(mapOutputTracker.shuffleStatuses(0).mapStatuses(1).location === bm1.blockManagerId) + + val env = mock(classOf[SparkEnv]) + when(env.conf).thenReturn(conf) + SparkEnv.set(env) + + decomManager.refreshOffloadingShuffleBlocks() + + eventually(timeout(1.second), interval(10.milliseconds)) { + assert(mapOutputTracker.shuffleStatuses(0).mapStatuses(0).location === bm2.blockManagerId) + assert(mapOutputTracker.shuffleStatuses(0).mapStatuses(1).location === bm2.blockManagerId) + } + assert(Files.readAllBytes(bm2.diskBlockManager.getFile(shuffleData).toPath()) + === shuffleDataBlockContent) + assert(Files.readAllBytes(bm2.diskBlockManager.getFile(shuffleIndex).toPath()) + === shuffleIndexBlockContent) + } finally { + mapOutputTracker.unregisterShuffle(0) + // Avoid thread leak + decomManager.stopOffloadingShuffleBlocks() + } + } + test("SPARK-32919: Shuffle push merger locations should be bounded with in" + " spark.shuffle.push.retainedMergerLocations") { assert(master.getShufflePushMergerLocations(10, Set.empty).isEmpty) diff --git a/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala b/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala index d4368c882d660..9d040bb4e1ec7 100644 --- a/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala @@ -113,7 +113,8 @@ class UIUtilsSuite extends SparkFunSuite { test("SPARK-11906: Progress bar should not overflow because of speculative tasks") { val generated = makeProgressBar(2, 3, 0, 0, Map.empty, 4).head.child.filter(_.label == "div") val expected = Seq( -
+
, +
) assert(generated.sameElements(expected), s"\nRunning progress bar should round down\n\nExpected:\n$expected\nGenerated:\n$generated") diff --git a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala index 12d97573ff6ee..71010a10cb23c 100644 --- a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala @@ -61,6 +61,15 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { assert(Files.toString(testFile, StandardCharsets.UTF_8) === header + testString) } + test("SPARK-35027: basic file appender - close stream") { + val inputStream = mock(classOf[InputStream]) + val appender = new FileAppender(inputStream, testFile, closeStreams = true) + Thread.sleep(10) + appender.stop() + appender.awaitTermination() + verify(inputStream).close() + } + test("rolling file appender - time-based rolling") { // setup input stream and appender val testOutputStream = new PipedOutputStream() @@ -96,6 +105,32 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { appender, testOutputStream, textToAppend, rolloverIntervalMillis, isCompressed = true) } + test("SPARK-35027: rolling file appender - time-based rolling close stream") { + val inputStream = mock(classOf[InputStream]) + val sparkConf = new SparkConf() + sparkConf.set(config.EXECUTOR_LOGS_ROLLING_STRATEGY.key, "time") + val appender = FileAppender(inputStream, testFile, sparkConf, closeStreams = true) + assert( + appender.asInstanceOf[RollingFileAppender].rollingPolicy.isInstanceOf[TimeBasedRollingPolicy]) + Thread.sleep(10) + appender.stop() + appender.awaitTermination() + verify(inputStream).close() + } + + test("SPARK-35027: rolling file appender - size-based rolling close stream") { + val inputStream = mock(classOf[InputStream]) + val sparkConf = new SparkConf() + sparkConf.set(config.EXECUTOR_LOGS_ROLLING_STRATEGY.key, "size") + val appender = FileAppender(inputStream, testFile, sparkConf, closeStreams = true) + assert( + appender.asInstanceOf[RollingFileAppender].rollingPolicy.isInstanceOf[SizeBasedRollingPolicy]) + Thread.sleep(10) + appender.stop() + appender.awaitTermination() + verify(inputStream).close() + } + test("rolling file appender - size-based rolling") { // setup input stream and appender val testOutputStream = new PipedOutputStream() diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 18ff96021153f..208e7297f06aa 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -1024,11 +1024,13 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { // Set some secret keys val secretKeys = Seq( "spark.executorEnv.HADOOP_CREDSTORE_PASSWORD", + "spark.hadoop.fs.s3a.access.key", "spark.my.password", "spark.my.sECreT") secretKeys.foreach { key => sparkConf.set(key, "sensitive_value") } // Set a non-secret key sparkConf.set("spark.regular.property", "regular_value") + sparkConf.set("spark.hadoop.fs.s3a.access_key", "regular_value") // Set a property with a regular key but secret in the value sparkConf.set("spark.sensitive.property", "has_secret_in_value") @@ -1039,7 +1041,8 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { secretKeys.foreach { key => assert(redactedConf(key) === Utils.REDACTION_REPLACEMENT_TEXT) } assert(redactedConf("spark.regular.property") === "regular_value") assert(redactedConf("spark.sensitive.property") === Utils.REDACTION_REPLACEMENT_TEXT) - + assert(redactedConf("spark.hadoop.fs.s3a.access.key") === Utils.REDACTION_REPLACEMENT_TEXT) + assert(redactedConf("spark.hadoop.fs.s3a.access_key") === "regular_value") } test("redact sensitive information in command line args") { diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSpillSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSpillSuite.scala new file mode 100644 index 0000000000000..959d5d813df81 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSpillSuite.scala @@ -0,0 +1,147 @@ +/* + * 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.util.collection + +import java.io.{File, IOException} +import java.util.UUID + +import scala.collection.mutable.ArrayBuffer + +import org.mockito.ArgumentMatchers.{any, anyInt} +import org.mockito.Mockito.{mock, when} +import org.mockito.invocation.InvocationOnMock +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.{SparkConf, SparkEnv, SparkFunSuite, TaskContext} +import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.internal.config +import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} +import org.apache.spark.serializer.{KryoSerializer, SerializerInstance, SerializerManager} +import org.apache.spark.storage.{BlockId, BlockManager, DiskBlockManager, DiskBlockObjectWriter, TempShuffleBlockId} +import org.apache.spark.util.{Utils => UUtils} + +class ExternalSorterSpillSuite extends SparkFunSuite with BeforeAndAfterEach { + + private val spillFilesCreated = ArrayBuffer.empty[File] + + private var tempDir: File = _ + private var conf: SparkConf = _ + private var taskMemoryManager: TaskMemoryManager = _ + + private var blockManager: BlockManager = _ + private var diskBlockManager: DiskBlockManager = _ + private var taskContext: TaskContext = _ + + override protected def beforeEach(): Unit = { + tempDir = UUtils.createTempDir(null, "test") + spillFilesCreated.clear() + + val env: SparkEnv = mock(classOf[SparkEnv]) + SparkEnv.set(env) + + conf = new SparkConf() + when(SparkEnv.get.conf).thenReturn(conf) + + val serializer = new KryoSerializer(conf) + when(SparkEnv.get.serializer).thenReturn(serializer) + + blockManager = mock(classOf[BlockManager]) + when(SparkEnv.get.blockManager).thenReturn(blockManager) + + val manager = new SerializerManager(serializer, conf) + when(blockManager.serializerManager).thenReturn(manager) + + diskBlockManager = mock(classOf[DiskBlockManager]) + when(blockManager.diskBlockManager).thenReturn(diskBlockManager) + + taskContext = mock(classOf[TaskContext]) + val memoryManager = new TestMemoryManager(conf) + taskMemoryManager = new TaskMemoryManager(memoryManager, 0) + when(taskContext.taskMemoryManager()).thenReturn(taskMemoryManager) + + when(diskBlockManager.createTempShuffleBlock()) + .thenAnswer((_: InvocationOnMock) => { + val blockId = TempShuffleBlockId(UUID.randomUUID) + val file = File.createTempFile("spillFile", ".spill", tempDir) + spillFilesCreated += file + (blockId, file) + }) + } + + override protected def afterEach(): Unit = { + UUtils.deleteRecursively(tempDir) + SparkEnv.set(null) + + val leakedMemory = taskMemoryManager.cleanUpAllAllocatedMemory + if (leakedMemory != 0) { + fail("Test leaked " + leakedMemory + " bytes of managed memory") + } + } + + test("SPARK-36242 Spill File should not exists if writer close fails") { + // Prepare the data and ensure that the amount of data let the `spill()` method + // to enter the `objectsWritten > 0` branch + val writeSize = conf.get(config.SHUFFLE_SPILL_BATCH_SIZE) + 1 + val dataBuffer = new PartitionedPairBuffer[Int, Int] + (0 until writeSize.toInt).foreach(i => dataBuffer.insert(0, 0, i)) + + val externalSorter = new TestExternalSorter[Int, Int, Int](taskContext) + + // Mock the answer of `blockManager.getDiskWriter` and let the `close()` method of + // `DiskBlockObjectWriter` throw IOException. + val errorMessage = "Spill file close failed" + when(blockManager.getDiskWriter( + any(classOf[BlockId]), + any(classOf[File]), + any(classOf[SerializerInstance]), + anyInt(), + any(classOf[ShuffleWriteMetrics]) + )).thenAnswer((invocation: InvocationOnMock) => { + val args = invocation.getArguments + new DiskBlockObjectWriter( + args(1).asInstanceOf[File], + blockManager.serializerManager, + args(2).asInstanceOf[SerializerInstance], + args(3).asInstanceOf[Int], + false, + args(4).asInstanceOf[ShuffleWriteMetrics], + args(0).asInstanceOf[BlockId] + ) { + override def close(): Unit = throw new IOException(errorMessage) + } + }) + + val ioe = intercept[IOException] { + externalSorter.spill(dataBuffer) + } + + ioe.getMessage.equals(errorMessage) + // The `TempShuffleBlock` create by diskBlockManager + // will remain before SPARK-36242 + assert(!spillFilesCreated(0).exists()) + } +} + +/** + * `TestExternalSorter` used to expand the access scope of the spill method. + */ +private[this] class TestExternalSorter[K, V, C](context: TaskContext) + extends ExternalSorter[K, V, C](context) { + override def spill(collection: WritablePartitionedPairCollection[K, C]): Unit = + super.spill(collection) +} diff --git a/core/src/test/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStreamSuite.scala b/core/src/test/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStreamSuite.scala index 86961745673c6..29443e275f769 100644 --- a/core/src/test/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStreamSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStreamSuite.scala @@ -119,4 +119,14 @@ class ChunkedByteBufferOutputStreamSuite extends SparkFunSuite { assert(arrays(1).toSeq === ref.slice(10, 20)) assert(arrays(2).toSeq === ref.slice(20, 30)) } + + test("SPARK-36464: size returns correct positive number even with over 2GB data") { + val ref = new Array[Byte](1024 * 1024 * 1024) + val o = new ChunkedByteBufferOutputStream(1024 * 1024, ByteBuffer.allocate) + o.write(ref) + o.write(ref) + o.close() + assert(o.size > 0L) // make sure it is not overflowing + assert(o.size == ref.length.toLong * 2) + } } diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroScanSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroScanSuite.scala new file mode 100644 index 0000000000000..98a7190ba984e --- /dev/null +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroScanSuite.scala @@ -0,0 +1,30 @@ +/* + * 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.avro + +import org.apache.spark.sql.FileScanSuiteBase +import org.apache.spark.sql.v2.avro.AvroScan + +class AvroScanSuite extends FileScanSuiteBase { + val scanBuilders = Seq[(String, ScanBuilder, Seq[String])]( + ("AvroScan", + (s, fi, ds, rds, rps, f, o, pf, df) => AvroScan(s, fi, ds, rds, rps, o, f, pf, df), + Seq.empty)) + + run(scanBuilders) +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 2eeffd840b7db..de9d15850e1c8 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -235,9 +235,21 @@ private[spark] object Config extends Logging { .checkValue(value => value > 0, "Allocation batch delay must be a positive time value.") .createWithDefaultString("1s") + val KUBERNETES_ALLOCATION_DRIVER_READINESS_TIMEOUT = + ConfigBuilder("spark.kubernetes.allocation.driver.readinessTimeout") + .doc("Time to wait for driver pod to get ready before creating executor pods. This wait " + + "only happens on application start. If timeout happens, executor pods will still be " + + "created.") + .version("3.1.3") + .timeConf(TimeUnit.SECONDS) + .checkValue(value => value > 0, "Allocation driver readiness timeout must be a positive " + + "time value.") + .createWithDefaultString("1s") + val KUBERNETES_ALLOCATION_EXECUTOR_TIMEOUT = ConfigBuilder("spark.kubernetes.allocation.executor.timeout") - .doc("Time to wait before considering a pending executor timedout.") + .doc("Time to wait before a newly created executor POD request, which does not reached " + + "the POD pending state yet, considered timedout and will be deleted.") .version("3.1.0") .timeConf(TimeUnit.MILLISECONDS) .checkValue(value => value > 0, "Allocation executor timeout must be a positive time value.") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala index cec8272beed57..7f34f30d59982 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala @@ -96,7 +96,7 @@ private[spark] class BasicDriverFeatureStep(conf: KubernetesDriverConf) val driverPort = conf.sparkConf.getInt(DRIVER_PORT.key, DEFAULT_DRIVER_PORT) val driverBlockManagerPort = conf.sparkConf.getInt( DRIVER_BLOCK_MANAGER_PORT.key, - DEFAULT_BLOCKMANAGER_PORT + conf.sparkConf.getInt(BLOCK_MANAGER_PORT.key, DEFAULT_BLOCKMANAGER_PORT) ) val driverUIPort = SparkUI.getUIPort(conf.sparkConf) val driverContainer = new ContainerBuilder(pod.container) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala index 250dd8238d9ea..a0a17cecf9a8e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala @@ -44,7 +44,10 @@ private[spark] class BasicExecutorFeatureStep( .getOrElse(throw new SparkException("Must specify the executor container image")) private val blockManagerPort = kubernetesConf .sparkConf - .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT) + .getInt(BLOCK_MANAGER_PORT.key, DEFAULT_BLOCKMANAGER_PORT) + + require(blockManagerPort == 0 || (1024 <= blockManagerPort && blockManagerPort < 65536), + "port number must be 0 or in [1024, 65535]") private val executorPodNamePrefix = kubernetesConf.resourceNamePrefix @@ -171,14 +174,17 @@ private[spark] class BasicExecutorFeatureStep( .replaceAll(ENV_EXECUTOR_ID, kubernetesConf.executorId)) } - val requiredPorts = Seq( - (BLOCK_MANAGER_PORT_NAME, blockManagerPort)) - .map { case (name, port) => - new ContainerPortBuilder() - .withName(name) - .withContainerPort(port) - .build() - } + // 0 is invalid as kubernetes containerPort request, we shall leave it unmounted + val requiredPorts = if (blockManagerPort != 0) { + Seq( + (BLOCK_MANAGER_PORT_NAME, blockManagerPort)) + .map { case (name, port) => + new ContainerPortBuilder() + .withName(name) + .withContainerPort(port) + .build() + } + } else Nil if (!isDefaultProfile) { if (pod.container != null && pod.container.getResources() != null) { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index eb35de8759593..9e1794fd3e8fc 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -18,7 +18,8 @@ package org.apache.spark.scheduler.cluster.k8s import java.time.Instant import java.util.concurrent.ConcurrentHashMap -import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong} +import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} import scala.collection.JavaConverters._ import scala.collection.mutable @@ -61,6 +62,8 @@ private[spark] class ExecutorPodsAllocator( podAllocationDelay * 5, conf.get(KUBERNETES_ALLOCATION_EXECUTOR_TIMEOUT)) + private val driverPodReadinessTimeout = conf.get(KUBERNETES_ALLOCATION_DRIVER_READINESS_TIMEOUT) + private val executorIdleTimeout = conf.get(DYN_ALLOCATION_EXECUTOR_IDLE_TIMEOUT) * 1000 private val namespace = conf.get(KUBERNETES_NAMESPACE) @@ -70,21 +73,26 @@ private[spark] class ExecutorPodsAllocator( private val shouldDeleteExecutors = conf.get(KUBERNETES_DELETE_EXECUTORS) - private val driverPod = kubernetesDriverPodName + val driverPod = kubernetesDriverPodName .map(name => Option(kubernetesClient.pods() .withName(name) .get()) .getOrElse(throw new SparkException( - s"No pod was found named $kubernetesDriverPodName in the cluster in the " + + s"No pod was found named $name in the cluster in the " + s"namespace $namespace (this was supposed to be the driver pod.)."))) // Executor IDs that have been requested from Kubernetes but have not been detected in any // snapshot yet. Mapped to the (ResourceProfile id, timestamp) when they were created. private val newlyCreatedExecutors = mutable.LinkedHashMap.empty[Long, (Int, Long)] + // Executor IDs that have been requested from Kubernetes but have not been detected in any POD + // snapshot yet but already known by the scheduler backend. Mapped to the ResourceProfile id. + private val schedulerKnownNewlyCreatedExecs = mutable.LinkedHashMap.empty[Long, Int] + private val dynamicAllocationEnabled = Utils.isDynamicAllocationEnabled(conf) - private val hasPendingPods = new AtomicBoolean() + // visible for tests + private[k8s] val numOutstandingPods = new AtomicInteger() private var lastSnapshot = ExecutorPodsSnapshot() @@ -93,9 +101,19 @@ private[spark] class ExecutorPodsAllocator( // if they happen to come up before the deletion takes effect. @volatile private var deletedExecutorIds = Set.empty[Long] - def start(applicationId: String): Unit = { + def start(applicationId: String, schedulerBackend: KubernetesClusterSchedulerBackend): Unit = { + driverPod.foreach { pod => + // Wait until the driver pod is ready before starting executors, as the headless service won't + // be resolvable by DNS until the driver pod is ready. + Utils.tryLogNonFatalError { + kubernetesClient + .pods() + .withName(pod.getMetadata.getName) + .waitUntilReady(driverPodReadinessTimeout, TimeUnit.SECONDS) + } + } snapshotsStore.addSubscriber(podAllocationDelay) { - onNewSnapshots(applicationId, _) + onNewSnapshots(applicationId, schedulerBackend, _) } } @@ -105,7 +123,7 @@ private[spark] class ExecutorPodsAllocator( totalExpectedExecutorsPerResourceProfileId.put(rp.id, numExecs) } logDebug(s"Set total expected execs to $totalExpectedExecutorsPerResourceProfileId") - if (!hasPendingPods.get()) { + if (numOutstandingPods.get() == 0) { snapshotsStore.notifySubscribers() } } @@ -114,8 +132,19 @@ private[spark] class ExecutorPodsAllocator( private def onNewSnapshots( applicationId: String, + schedulerBackend: KubernetesClusterSchedulerBackend, snapshots: Seq[ExecutorPodsSnapshot]): Unit = { - newlyCreatedExecutors --= snapshots.flatMap(_.executorPods.keys) + val k8sKnownExecIds = snapshots.flatMap(_.executorPods.keys) + newlyCreatedExecutors --= k8sKnownExecIds + schedulerKnownNewlyCreatedExecs --= k8sKnownExecIds + + // transfer the scheduler backend known executor requests from the newlyCreatedExecutors + // to the schedulerKnownNewlyCreatedExecs + val schedulerKnownExecs = schedulerBackend.getExecutorIds().map(_.toLong).toSet + schedulerKnownNewlyCreatedExecs ++= + newlyCreatedExecutors.filterKeys(schedulerKnownExecs.contains(_)).mapValues(_._1) + newlyCreatedExecutors --= schedulerKnownNewlyCreatedExecs.keySet + // For all executors we've created against the API but have not seen in a snapshot // yet - check the current time. If the current time has exceeded some threshold, // assume that the pod was either never created (the API server never properly @@ -164,15 +193,16 @@ private[spark] class ExecutorPodsAllocator( _deletedExecutorIds = _deletedExecutorIds.filter(existingExecs.contains) } + val notDeletedPods = lastSnapshot.executorPods.filterKeys(!_deletedExecutorIds.contains(_)) // Map the pods into per ResourceProfile id so we can check per ResourceProfile, // add a fast path if not using other ResourceProfiles. val rpIdToExecsAndPodState = mutable.HashMap[Int, mutable.HashMap[Long, ExecutorPodState]]() if (totalExpectedExecutorsPerResourceProfileId.size <= 1) { rpIdToExecsAndPodState(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) = - mutable.HashMap.empty ++= lastSnapshot.executorPods + mutable.HashMap.empty ++= notDeletedPods } else { - lastSnapshot.executorPods.foreach { case (execId, execPodState) => + notDeletedPods.foreach { case (execId, execPodState) => val rpId = execPodState.pod.getMetadata.getLabels.get(SPARK_RESOURCE_PROFILE_ID_LABEL).toInt val execPods = rpIdToExecsAndPodState.getOrElseUpdate(rpId, mutable.HashMap[Long, ExecutorPodState]()) @@ -190,24 +220,33 @@ private[spark] class ExecutorPodsAllocator( case _ => false } - val currentPendingExecutors = podsForRpId.filter { + val (schedulerKnownPendingExecsForRpId, currentPendingExecutorsForRpId) = podsForRpId.filter { case (_, PodPending(_)) => true case _ => false + }.partition { case (k, _) => + schedulerKnownExecs.contains(k) } // This variable is used later to print some debug logs. It's updated when cleaning up - // excess pod requests, since currentPendingExecutors is immutable. - var knownPendingCount = currentPendingExecutors.size + // excess pod requests, since currentPendingExecutorsForRpId is immutable. + var knownPendingCount = currentPendingExecutorsForRpId.size val newlyCreatedExecutorsForRpId = newlyCreatedExecutors.filter { case (_, (waitingRpId, _)) => rpId == waitingRpId } + val schedulerKnownNewlyCreatedExecsForRpId = + schedulerKnownNewlyCreatedExecs.filter { case (_, waitingRpId) => + rpId == waitingRpId + } + if (podsForRpId.nonEmpty) { logDebug(s"ResourceProfile Id: $rpId " + s"pod allocation status: $currentRunningCount running, " + - s"${currentPendingExecutors.size} pending. " + - s"${newlyCreatedExecutorsForRpId.size} unacknowledged.") + s"${currentPendingExecutorsForRpId.size} unknown pending, " + + s"${schedulerKnownPendingExecsForRpId.size} scheduler backend known pending, " + + s"${newlyCreatedExecutorsForRpId.size} unknown newly created, " + + s"${schedulerKnownNewlyCreatedExecsForRpId.size} scheduler backend known newly created.") } // It's possible that we have outstanding pods that are outdated when dynamic allocation @@ -218,8 +257,9 @@ private[spark] class ExecutorPodsAllocator( // // TODO: with dynamic allocation off, handle edge cases if we end up with more running // executors than expected. - val knownPodCount = currentRunningCount + currentPendingExecutors.size + - newlyCreatedExecutorsForRpId.size + val knownPodCount = currentRunningCount + + currentPendingExecutorsForRpId.size + schedulerKnownPendingExecsForRpId.size + + newlyCreatedExecutorsForRpId.size + schedulerKnownNewlyCreatedExecsForRpId.size if (knownPodCount > targetNum) { val excess = knownPodCount - targetNum @@ -227,7 +267,7 @@ private[spark] class ExecutorPodsAllocator( .filter { case (_, (_, createTime)) => currentTime - createTime > executorIdleTimeout }.keys.take(excess).toList - val knownPendingToDelete = currentPendingExecutors + val knownPendingToDelete = currentPendingExecutorsForRpId .filter(x => isExecutorIdleTimedOut(x._2, currentTime)) .take(excess - newlyCreatedToDelete.size) .map { case (id, _) => id } @@ -245,7 +285,7 @@ private[spark] class ExecutorPodsAllocator( .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) .withLabelIn(SPARK_EXECUTOR_ID_LABEL, toDelete.sorted.map(_.toString): _*) .delete() - newlyCreatedExecutors --= toDelete + newlyCreatedExecutors --= newlyCreatedToDelete knownPendingCount -= knownPendingToDelete.size } } @@ -276,8 +316,9 @@ private[spark] class ExecutorPodsAllocator( deletedExecutorIds = _deletedExecutorIds // Update the flag that helps the setTotalExpectedExecutors() callback avoid triggering this - // update method when not needed. - hasPendingPods.set(totalPendingCount + newlyCreatedExecutors.size > 0) + // update method when not needed. PODs known by the scheduler backend are not counted here as + // they considered running PODs and they should not block upscaling. + numOutstandingPods.set(totalPendingCount + newlyCreatedExecutors.size) } private def requestNewExecutors( @@ -328,11 +369,11 @@ private[spark] class ExecutorPodsAllocator( private def isExecutorIdleTimedOut(state: ExecutorPodState, currentTime: Long): Boolean = { try { - val startTime = Instant.parse(state.pod.getStatus.getStartTime).toEpochMilli() - currentTime - startTime > executorIdleTimeout + val creationTime = Instant.parse(state.pod.getMetadata.getCreationTimestamp).toEpochMilli() + currentTime - creationTime > executorIdleTimeout } catch { - case _: Exception => - logDebug(s"Cannot get startTime of pod ${state.pod}") + case e: Exception => + logError(s"Cannot get the creationTimestamp of the pod: ${state.pod}", e) true } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala index da7fe7cdda328..6fcb87655b56d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala @@ -22,12 +22,21 @@ import io.fabric8.kubernetes.client.KubernetesClient import scala.collection.JavaConverters._ import org.apache.spark.SparkConf +import org.apache.spark.annotation.{DeveloperApi, Since, Stable} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.internal.Logging import org.apache.spark.util.{ThreadUtils, Utils} -private[spark] class ExecutorPodsPollingSnapshotSource( +/** + * :: DeveloperApi :: + * + * A class used for polling K8s executor pods by ExternalClusterManagers. + * @since 3.1.3 + */ +@Stable +@DeveloperApi +class ExecutorPodsPollingSnapshotSource( conf: SparkConf, kubernetesClient: KubernetesClient, snapshotsStore: ExecutorPodsSnapshotsStore, @@ -37,6 +46,7 @@ private[spark] class ExecutorPodsPollingSnapshotSource( private var pollingFuture: Future[_] = _ + @Since("3.1.3") def start(applicationId: String): Unit = { require(pollingFuture == null, "Cannot start polling more than once.") logDebug(s"Starting to check for executor pod state every $pollingInterval ms.") @@ -44,6 +54,7 @@ private[spark] class ExecutorPodsPollingSnapshotSource( new PollRunnable(applicationId), pollingInterval, pollingInterval, TimeUnit.MILLISECONDS) } + @Since("3.1.3") def stop(): Unit = { if (pollingFuture != null) { pollingFuture.cancel(true) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSource.scala index a6749a644e00c..7ac70b589c698 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSource.scala @@ -22,16 +22,27 @@ import io.fabric8.kubernetes.api.model.Pod import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watcher} import io.fabric8.kubernetes.client.Watcher.Action +import org.apache.spark.annotation.{DeveloperApi, Since, Stable} import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.internal.Logging import org.apache.spark.util.Utils -private[spark] class ExecutorPodsWatchSnapshotSource( +/** + * :: DeveloperApi :: + * + * A class used for watching K8s executor pods by ExternalClusterManagers. + * + * @since 3.1.3 + */ +@Stable +@DeveloperApi +class ExecutorPodsWatchSnapshotSource( snapshotsStore: ExecutorPodsSnapshotsStore, kubernetesClient: KubernetesClient) extends Logging { private var watchConnection: Closeable = _ + @Since("3.1.3") def start(applicationId: String): Unit = { require(watchConnection == null, "Cannot start the watcher twice.") logDebug(s"Starting watch for pods with labels $SPARK_APP_ID_LABEL=$applicationId," + @@ -42,6 +53,7 @@ private[spark] class ExecutorPodsWatchSnapshotSource( .watch(new ExecutorPodsWatcher()) } + @Since("3.1.3") def stop(): Unit = { if (watchConnection != null) { Utils.tryLogNonFatalError { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index c35a434f83ec1..887afca05cb1c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -20,11 +20,13 @@ import java.util.concurrent.{ScheduledExecutorService, TimeUnit} import scala.concurrent.Future +import io.fabric8.kubernetes.api.model.Pod import io.fabric8.kubernetes.client.KubernetesClient import org.apache.spark.SparkContext import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.KubernetesUtils import org.apache.spark.deploy.k8s.submit.KubernetesClientUtils import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.internal.config.SCHEDULER_MIN_REGISTERED_RESOURCES_RATIO @@ -67,13 +69,14 @@ private[spark] class KubernetesClusterSchedulerBackend( } } - private def setUpExecutorConfigMap(): Unit = { + private def setUpExecutorConfigMap(driverPod: Option[Pod]): Unit = { val configMapName = KubernetesClientUtils.configMapNameExecutor val confFilesMap = KubernetesClientUtils .buildSparkConfDirFilesMap(configMapName, conf, Map.empty) val labels = Map(SPARK_APP_ID_LABEL -> applicationId(), SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) val configMap = KubernetesClientUtils.buildConfigMap(configMapName, confFilesMap, labels) + KubernetesUtils.addOwnerReference(driverPod.orNull, Seq(configMap)) kubernetesClient.configMaps().create(configMap) } @@ -93,10 +96,10 @@ private[spark] class KubernetesClusterSchedulerBackend( val initExecs = Map(defaultProfile -> initialExecutors) podAllocator.setTotalExpectedExecutors(initExecs) lifecycleEventHandler.start(this) - podAllocator.start(applicationId()) + podAllocator.start(applicationId(), this) watchEvents.start(applicationId()) pollEvents.start(applicationId()) - setUpExecutorConfigMap() + setUpExecutorConfigMap(podAllocator.driverPod) } override def stop(): Unit = { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala index 858b4f1494b8e..f0843225ea6e2 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala @@ -213,6 +213,25 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { } } + test("SPARK-35493: make spark.blockManager.port be able to be fallen back to in driver pod") { + val initPod = SparkPod.initialPod() + val sparkConf = new SparkConf() + .set(CONTAINER_IMAGE, "spark-driver:latest") + .set(BLOCK_MANAGER_PORT, 1234) + val driverConf1 = KubernetesTestConf.createDriverConf(sparkConf) + val pod1 = new BasicDriverFeatureStep(driverConf1).configurePod(initPod) + val portMap1 = + pod1.container.getPorts.asScala.map { cp => (cp.getName -> cp.getContainerPort) }.toMap + assert(portMap1(BLOCK_MANAGER_PORT_NAME) === 1234, s"fallback to $BLOCK_MANAGER_PORT.key") + + val driverConf2 = + KubernetesTestConf.createDriverConf(sparkConf.set(DRIVER_BLOCK_MANAGER_PORT, 1235)) + val pod2 = new BasicDriverFeatureStep(driverConf2).configurePod(initPod) + val portMap2 = + pod2.container.getPorts.asScala.map { cp => (cp.getName -> cp.getContainerPort) }.toMap + assert(portMap2(BLOCK_MANAGER_PORT_NAME) === 1235) + } + def containerPort(name: String, portNumber: Int): ContainerPort = new ContainerPortBuilder() .withName(name) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala index 66ece81aca646..df4693f313917 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala @@ -330,6 +330,35 @@ class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { SecretVolumeUtils.podHasVolume(podConfigured.pod, SPARK_CONF_VOLUME_EXEC) } + test("SPARK-35482: user correct block manager port for executor pods") { + try { + val initPod = SparkPod.initialPod() + val sm = new SecurityManager(baseConf) + val step1 = + new BasicExecutorFeatureStep(newExecutorConf(), sm, defaultProfile) + val containerPort1 = step1.configurePod(initPod).container.getPorts.get(0) + assert(containerPort1.getContainerPort === DEFAULT_BLOCKMANAGER_PORT, + s"should use port no. $DEFAULT_BLOCKMANAGER_PORT as default") + + baseConf.set(BLOCK_MANAGER_PORT, 12345) + val step2 = new BasicExecutorFeatureStep(newExecutorConf(), sm, defaultProfile) + val containerPort2 = step2.configurePod(initPod).container.getPorts.get(0) + assert(containerPort2.getContainerPort === 12345) + + baseConf.set(BLOCK_MANAGER_PORT, 1000) + val e = intercept[IllegalArgumentException] { + new BasicExecutorFeatureStep(newExecutorConf(), sm, defaultProfile) + } + assert(e.getMessage.contains("port number must be 0 or in [1024, 65535]")) + + baseConf.set(BLOCK_MANAGER_PORT, 0) + val step3 = new BasicExecutorFeatureStep(newExecutorConf(), sm, defaultProfile) + assert(step3.configurePod(initPod).container.getPorts.isEmpty, "random port") + } finally { + baseConf.remove(BLOCK_MANAGER_PORT) + } + } + // There is always exactly one controller reference, and it points to the driver pod. private def checkOwnerReferences(executor: Pod, driverPodUid: String): Unit = { assert(executor.getMetadata.getOwnerReferences.size() === 1) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala index 41cba573d89c2..0b3ce6d7eb274 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala @@ -62,9 +62,11 @@ object ExecutorLifecycleTestUtils { def pendingExecutor(executorId: Long, rpId: Int = DEFAULT_RESOURCE_PROFILE_ID): Pod = { new PodBuilder(podWithAttachedContainerForId(executorId, rpId)) + .editOrNewMetadata() + .withCreationTimestamp(Instant.now.toString) + .endMetadata() .editOrNewStatus() .withPhase("pending") - .withStartTime(Instant.now.toString) .endStatus() .build() } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala index 349bbcd6f7883..7ec17298d902e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -28,7 +28,7 @@ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfter -import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import org.apache.spark.{SecurityManager, SparkConf, SparkException, SparkFunSuite} import org.apache.spark.deploy.k8s.{KubernetesExecutorConf, KubernetesExecutorSpec} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ @@ -81,6 +81,9 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { @Mock private var executorBuilder: KubernetesExecutorBuilder = _ + @Mock + private var schedulerBackend: KubernetesClusterSchedulerBackend = _ + private var snapshotsStore: DeterministicExecutorPodsSnapshotsStore = _ private var podsAllocatorUnderTest: ExecutorPodsAllocator = _ @@ -90,18 +93,21 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { when(kubernetesClient.pods()).thenReturn(podOperations) when(podOperations.withName(driverPodName)).thenReturn(driverPodOperations) when(driverPodOperations.get).thenReturn(driverPod) + when(driverPodOperations.waitUntilReady(any(), any())).thenReturn(driverPod) when(executorBuilder.buildFromFeatures(any(classOf[KubernetesExecutorConf]), meq(secMgr), meq(kubernetesClient), any(classOf[ResourceProfile]))).thenAnswer(executorPodAnswer()) snapshotsStore = new DeterministicExecutorPodsSnapshotsStore() waitForExecutorPodsClock = new ManualClock(0L) podsAllocatorUnderTest = new ExecutorPodsAllocator( conf, secMgr, executorBuilder, kubernetesClient, snapshotsStore, waitForExecutorPodsClock) - podsAllocatorUnderTest.start(TEST_SPARK_APP_ID) + when(schedulerBackend.getExecutorIds).thenReturn(Seq.empty) + podsAllocatorUnderTest.start(TEST_SPARK_APP_ID, schedulerBackend) } test("Initially request executors in batches. Do not request another batch if the" + " first has not finished.") { podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> (podAllocationSize + 1))) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 5) for (nextId <- 1 to podAllocationSize) { verify(podOperations).create(podWithAttachedContainerForId(nextId)) } @@ -111,28 +117,34 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { test("Request executors in batches. Allow another batch to be requested if" + " all pending executors start running.") { podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> (podAllocationSize + 1))) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 5) for (execId <- 1 until podAllocationSize) { snapshotsStore.updatePod(runningExecutor(execId)) } snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 1) verify(podOperations, never()).create(podWithAttachedContainerForId(podAllocationSize + 1)) snapshotsStore.updatePod(runningExecutor(podAllocationSize)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 1) verify(podOperations).create(podWithAttachedContainerForId(podAllocationSize + 1)) snapshotsStore.updatePod(runningExecutor(podAllocationSize)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 1) verify(podOperations, times(podAllocationSize + 1)).create(any(classOf[Pod])) } test("When a current batch reaches error states immediately, re-request" + " them on the next batch.") { podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> podAllocationSize)) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 5) for (execId <- 1 until podAllocationSize) { snapshotsStore.updatePod(runningExecutor(execId)) } val failedPod = failedExecutorWithoutDeletion(podAllocationSize) snapshotsStore.updatePod(failedPod) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 1) verify(podOperations).create(podWithAttachedContainerForId(podAllocationSize + 1)) } @@ -148,9 +160,11 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { .withLabelIn(SPARK_EXECUTOR_ID_LABEL, "1")) .thenReturn(labeledPods) podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 1)) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 1) verify(podOperations).create(podWithAttachedContainerForId(1)) waitForExecutorPodsClock.setTime(podCreationTimeout + 1) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 1) verify(labeledPods).delete() verify(podOperations).create(podWithAttachedContainerForId(2)) } @@ -174,17 +188,20 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { // Target 1 executor, make sure it's requested, even with an empty initial snapshot. podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 1)) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 1) verify(podOperations).create(podWithAttachedContainerForId(1)) // Mark executor as running, verify that subsequent allocation cycle is a no-op. snapshotsStore.updatePod(runningExecutor(1)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) verify(podOperations, times(1)).create(any()) verify(podOperations, never()).delete() // Request 3 more executors, make sure all are requested. podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 4)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 3) verify(podOperations).create(podWithAttachedContainerForId(2)) verify(podOperations).create(podWithAttachedContainerForId(3)) verify(podOperations).create(podWithAttachedContainerForId(4)) @@ -193,6 +210,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { snapshotsStore.updatePod(runningExecutor(2)) snapshotsStore.updatePod(pendingExecutor(3)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 2) verify(podOperations, times(4)).create(any()) verify(podOperations, never()).delete() @@ -200,6 +218,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { waitForExecutorPodsClock.advance(executorIdleTimeout * 2) podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 1)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) verify(podOperations, times(4)).create(any()) verify(podOperations).withLabelIn(SPARK_EXECUTOR_ID_LABEL, "3", "4") verify(podOperations).delete() @@ -212,6 +231,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { snapshotsStore.updatePod(deletedExecutor(4)) snapshotsStore.removeDeletedExecutors() snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) assert(!podsAllocatorUnderTest.isDeleted("3")) assert(!podsAllocatorUnderTest.isDeleted("4")) } @@ -279,6 +299,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { waitForExecutorPodsClock.setTime(startTime) podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 5)) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 5) verify(podOperations).create(podWithAttachedContainerForId(1)) verify(podOperations).create(podWithAttachedContainerForId(2)) verify(podOperations).create(podWithAttachedContainerForId(3)) @@ -292,16 +313,139 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { // Newly created executors (both acknowledged and not) are protected by executorIdleTimeout podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 0)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 5) verify(podOperations, never()).withLabelIn(SPARK_EXECUTOR_ID_LABEL, "1", "2", "3", "4", "5") verify(podOperations, never()).delete() // Newly created executors (both acknowledged and not) are cleaned up. waitForExecutorPodsClock.advance(executorIdleTimeout * 2) + when(schedulerBackend.getExecutorIds).thenReturn(Seq("1", "3", "4")) snapshotsStore.notifySubscribers() - verify(podOperations).withLabelIn(SPARK_EXECUTOR_ID_LABEL, "1", "2", "3", "4", "5") + // SPARK-34361: even as 1, 3 and 4 are not timed out as they are considered as known PODs so + // this is why they are not counted into the outstanding PODs and /they are not removed even + // though executor 1 is still in pending state and executor 3 and 4 are new request without + // any state reported by kubernetes and all the three are already timed out + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) + verify(podOperations).withLabelIn(SPARK_EXECUTOR_ID_LABEL, "2", "5") verify(podOperations).delete() } + /** + * This test covers some downscaling and upscaling of dynamic allocation on kubernetes + * along with multiple resource profiles (default and rp) when some executors + * already know by the scheduler backend. + * + * Legend: + * + * N-: newly created not known by the scheduler backend + * N+: newly created known by the scheduler backend + * P- / P+ : pending (not know / known) by the scheduler backend + * D: deleted + * | default || rp | expected + * | || | outstanding + * | 1 | 2 | 3 || 4 | 5 | 6 | 7 | PODs + * ========================================================================================== + * 0) setTotalExpectedExecs with | N- | N- | N- || N- | N- | N- | N- | + * default->3, ro->4 | | | || | | | | 7 + * ------------------------------------------------------------------------------------------ + * 1) make 1 from each rp | N+ | N- | N- || N+ | N- | N- | N- | + * known by backend | | | || | | | | 5 + * ------------------------------------------------------------------------------------------- + * 2) some more backend known + pending | N+ | P+ | P- || N+ | P+ | P- | N- | 3 + * ------------------------------------------------------------------------------------------- + * 3) advance time with idle timeout | | | || | | | | + * setTotalExpectedExecs with | N+ | P+ | D || N+ | P+ | D | D | 0 + * default->1, rp->1 | | | || | | | | + * ------------------------------------------------------------------------------------------- + * 4) setTotalExpectedExecs with | N+ | P+ | D || N+ | P+ | D | D | 0 and + * default->2, rp->2 | | | || | | | | no new POD req. + * =========================================================================================== + * + * 5) setTotalExpectedExecs with default -> 3, rp -> 3 which will lead to creation of the new + * PODs: 8 and 9 + */ + test("SPARK-34361: scheduler backend known pods with multiple resource profiles at downscaling") { + when(podOperations + .withField("status.phase", "Pending")) + .thenReturn(podOperations) + when(podOperations + .withLabel(SPARK_APP_ID_LABEL, TEST_SPARK_APP_ID)) + .thenReturn(podOperations) + when(podOperations + .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE)) + .thenReturn(podOperations) + when(podOperations + .withLabelIn(meq(SPARK_EXECUTOR_ID_LABEL), any())) + .thenReturn(podOperations) + + val startTime = Instant.now.toEpochMilli + waitForExecutorPodsClock.setTime(startTime) + + val rpb = new ResourceProfileBuilder() + val ereq = new ExecutorResourceRequests() + val treq = new TaskResourceRequests() + ereq.cores(4).memory("2g") + treq.cpus(2) + rpb.require(ereq).require(treq) + val rp = rpb.build() + + // 0) request 3 PODs for the default and 4 PODs for the other resource profile + podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 3, rp -> 4)) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 7) + verify(podOperations).create(podWithAttachedContainerForId(1, defaultProfile.id)) + verify(podOperations).create(podWithAttachedContainerForId(2, defaultProfile.id)) + verify(podOperations).create(podWithAttachedContainerForId(3, defaultProfile.id)) + verify(podOperations).create(podWithAttachedContainerForId(4, rp.id)) + verify(podOperations).create(podWithAttachedContainerForId(5, rp.id)) + verify(podOperations).create(podWithAttachedContainerForId(6, rp.id)) + verify(podOperations).create(podWithAttachedContainerForId(7, rp.id)) + + // 1) make 1 POD known by the scheduler backend for each resource profile + when(schedulerBackend.getExecutorIds).thenReturn(Seq("1", "4")) + snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 5, + "scheduler backend known PODs are not outstanding") + verify(podOperations, times(7)).create(any()) + + // 2) make 1 extra POD known by the scheduler backend for each resource profile + // and make some to pending + when(schedulerBackend.getExecutorIds).thenReturn(Seq("1", "2", "4", "5")) + snapshotsStore.updatePod(pendingExecutor(2, defaultProfile.id)) + snapshotsStore.updatePod(pendingExecutor(3, defaultProfile.id)) + snapshotsStore.updatePod(pendingExecutor(5, rp.id)) + snapshotsStore.updatePod(pendingExecutor(6, rp.id)) + snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 3) + verify(podOperations, times(7)).create(any()) + + // 3) downscale to 1 POD for default and 1 POD for the other resource profile + waitForExecutorPodsClock.advance(executorIdleTimeout * 2) + podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 1, rp -> 1)) + snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) + verify(podOperations, times(7)).create(any()) + verify(podOperations, times(2)).delete() + assert(podsAllocatorUnderTest.isDeleted("3")) + assert(podsAllocatorUnderTest.isDeleted("6")) + assert(podsAllocatorUnderTest.isDeleted("7")) + + // 4) upscale to 2 PODs for default and 2 for the other resource profile but as there is still + // 2 PODs known by the scheduler backend there must be no new POD requested to be created + podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 2, rp -> 2)) + snapshotsStore.notifySubscribers() + verify(podOperations, times(7)).create(any()) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) + verify(podOperations, times(7)).create(any()) + + // 5) requesting 1 more executor for each resource + podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 3, rp -> 3)) + snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 2) + verify(podOperations, times(9)).create(any()) + verify(podOperations).create(podWithAttachedContainerForId(8, defaultProfile.id)) + verify(podOperations).create(podWithAttachedContainerForId(9, rp.id)) + } + test("SPARK-33288: multiple resource profiles") { when(podOperations .withField("status.phase", "Pending")) @@ -330,6 +474,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { // Target 1 executor for default profile, 2 for other profile, // make sure it's requested, even with an empty initial snapshot. podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 1, rp -> 2)) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 3) verify(podOperations).create(podWithAttachedContainerForId(1, defaultProfile.id)) verify(podOperations).create(podWithAttachedContainerForId(2, rp.id)) verify(podOperations).create(podWithAttachedContainerForId(3, rp.id)) @@ -339,6 +484,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { snapshotsStore.updatePod(runningExecutor(2, rp.id)) snapshotsStore.updatePod(runningExecutor(3, rp.id)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) verify(podOperations, times(3)).create(any()) verify(podOperations, never()).delete() @@ -346,6 +492,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { // make sure all are requested. podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 4, rp -> 3)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 4) verify(podOperations).create(podWithAttachedContainerForId(4, defaultProfile.id)) verify(podOperations).create(podWithAttachedContainerForId(5, defaultProfile.id)) verify(podOperations).create(podWithAttachedContainerForId(6, defaultProfile.id)) @@ -356,6 +503,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { snapshotsStore.updatePod(pendingExecutor(5, defaultProfile.id)) snapshotsStore.updatePod(pendingExecutor(7, rp.id)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 3) verify(podOperations, times(7)).create(any()) verify(podOperations, never()).delete() @@ -364,6 +512,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { waitForExecutorPodsClock.advance(executorIdleTimeout * 2) podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 1, rp -> 1)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) verify(podOperations, times(7)).create(any()) verify(podOperations).withLabelIn(SPARK_EXECUTOR_ID_LABEL, "5", "6") verify(podOperations).withLabelIn(SPARK_EXECUTOR_ID_LABEL, "7") @@ -379,6 +528,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { snapshotsStore.updatePod(deletedExecutor(7)) snapshotsStore.removeDeletedExecutors() snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) assert(!podsAllocatorUnderTest.isDeleted("5")) assert(!podsAllocatorUnderTest.isDeleted("6")) assert(!podsAllocatorUnderTest.isDeleted("7")) @@ -399,6 +549,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { .thenReturn(podOperations) podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 6)) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 5) // Initial request of pods verify(podOperations).create(podWithAttachedContainerForId(1)) verify(podOperations).create(podWithAttachedContainerForId(2)) @@ -414,10 +565,23 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { // We move forward one allocation cycle waitForExecutorPodsClock.setTime(podAllocationDelay + 1) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 2) // We request pod 6 verify(podOperations).create(podWithAttachedContainerForId(6)) } + test("print the pod name instead of Some(name) if pod is absent") { + val nonexistentPod = "i-do-not-exist" + val conf = new SparkConf().set(KUBERNETES_DRIVER_POD_NAME, nonexistentPod) + when(kubernetesClient.pods()).thenReturn(podOperations) + when(podOperations.withName(nonexistentPod)).thenReturn(driverPodOperations) + when(driverPodOperations.get()).thenReturn(null) + val e = intercept[SparkException](new ExecutorPodsAllocator( + conf, secMgr, executorBuilder, kubernetesClient, snapshotsStore, waitForExecutorPodsClock)) + assert(e.getMessage.contains("No pod was found named i-do-not-exist in the cluster in the" + + " namespace default")) + } + private def executorPodAnswer(): Answer[KubernetesExecutorSpec] = (invocation: InvocationOnMock) => { val k8sConf: KubernetesExecutorConf = invocation.getArgument(0) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index 861d41cc50ac5..3573ffc07d2c8 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -112,6 +112,7 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn .thenReturn(driverEndpointRef) when(kubernetesClient.pods()).thenReturn(podOperations) when(kubernetesClient.configMaps()).thenReturn(configMapsOperations) + when(podAllocator.driverPod).thenReturn(None) schedulerBackendUnderTest = new KubernetesClusterSchedulerBackend( taskScheduler, sc, @@ -127,7 +128,7 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn test("Start all components") { schedulerBackendUnderTest.start() verify(podAllocator).setTotalExpectedExecutors(Map(defaultProfile -> 3)) - verify(podAllocator).start(TEST_SPARK_APP_ID) + verify(podAllocator).start(TEST_SPARK_APP_ID, schedulerBackendUnderTest) verify(lifecycleEventHandler).start(schedulerBackendUnderTest) verify(watchEvents).start(TEST_SPARK_APP_ID) verify(pollEvents).start(TEST_SPARK_APP_ID) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala index 88c672f1cdf85..d58de77dc3dd2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala @@ -316,7 +316,10 @@ trait Row extends Serializable { * * @throws ClassCastException when data type does not match. */ - def getSeq[T](i: Int): Seq[T] = getAs[scala.collection.Seq[T]](i).toSeq + def getSeq[T](i: Int): Seq[T] = { + val res = getAs[scala.collection.Seq[T]](i) + if (res != null) res.toSeq else null + } /** * Returns the value at position i of array type as `java.util.List`. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index e9e8ba842a36f..5578838df97db 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -157,8 +157,10 @@ class Analyzer(override val catalogManager: CatalogManager) private val v1SessionCatalog: SessionCatalog = catalogManager.v1SessionCatalog - override protected def isPlanIntegral(plan: LogicalPlan): Boolean = { - !Utils.isTesting || LogicalPlanIntegrity.checkIfExprIdsAreGloballyUnique(plan) + override protected def isPlanIntegral( + previousPlan: LogicalPlan, + currentPlan: LogicalPlan): Boolean = { + !Utils.isTesting || LogicalPlanIntegrity.checkIfExprIdsAreGloballyUnique(currentPlan) } override def isView(nameParts: Seq[String]): Boolean = v1SessionCatalog.isView(nameParts) @@ -248,7 +250,7 @@ class Analyzer(override val catalogManager: CatalogManager) ResolvePartitionSpec :: AddMetadataColumns :: ResolveReferences :: - ResolveCreateNamedStruct :: + ResolveExpressionsWithNamePlaceholders :: ResolveDeserializer :: ResolveNewInstance :: ResolveUpCast :: @@ -539,7 +541,7 @@ class Analyzer(override val catalogManager: CatalogManager) aggregations: Seq[NamedExpression], groupByAliases: Seq[Alias], groupingAttrs: Seq[Expression], - gid: Attribute): Seq[NamedExpression] = aggregations.map { + gid: Attribute): Seq[NamedExpression] = aggregations.map { agg => // collect all the found AggregateExpression, so we can check an expression is part of // any AggregateExpression or not. val aggsBuffer = ArrayBuffer[Expression]() @@ -547,7 +549,7 @@ class Analyzer(override val catalogManager: CatalogManager) def isPartOfAggregation(e: Expression): Boolean = { aggsBuffer.exists(a => a.find(_ eq e).isDefined) } - replaceGroupingFunc(_, groupByExprs, gid).transformDown { + replaceGroupingFunc(agg, groupByExprs, gid).transformDown { // AggregateExpression should be computed on the unmodified value of its argument // expressions, so we should not replace any references to grouping expression // inside it. @@ -872,16 +874,16 @@ class Analyzer(override val catalogManager: CatalogManager) object ResolveTempViews extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { case u @ UnresolvedRelation(ident, _, isStreaming) => - lookupTempView(ident, isStreaming).getOrElse(u) + lookupAndResolveTempView(ident, isStreaming).getOrElse(u) case i @ InsertIntoStatement(UnresolvedRelation(ident, _, false), _, _, _, _, _) => - lookupTempView(ident) + lookupAndResolveTempView(ident) .map(view => i.copy(table = view)) .getOrElse(i) // TODO (SPARK-27484): handle streaming write commands when we have them. case write: V2WriteCommand => write.table match { case UnresolvedRelation(ident, _, false) => - lookupTempView(ident).map(EliminateSubqueryAliases(_)).map { + lookupAndResolveTempView(ident).map(EliminateSubqueryAliases(_)).map { case r: DataSourceV2Relation => write.withNewTable(r) case _ => throw new AnalysisException("Cannot write into temp view " + s"${ident.quoted} as it's not a data source v2 relation.") @@ -905,8 +907,9 @@ class Analyzer(override val catalogManager: CatalogManager) .getOrElse(u) } - def lookupTempView( - identifier: Seq[String], isStreaming: Boolean = false): Option[LogicalPlan] = { + private def lookupTempView( + identifier: Seq[String], + isStreaming: Boolean = false): Option[LogicalPlan] = { // Permanent View can't refer to temp views, no need to lookup at all. if (isResolvingView && !referredTempViewNames.contains(identifier)) return None @@ -920,7 +923,13 @@ class Analyzer(override val catalogManager: CatalogManager) throw new AnalysisException(s"${identifier.quoted} is not a temp view of streaming " + s"logical plan, please use batch API such as `DataFrameReader.table` to read it.") } - tmpView.map(ResolveRelations.resolveViews) + tmpView + } + + private def lookupAndResolveTempView( + identifier: Seq[String], + isStreaming: Boolean = false): Option[LogicalPlan] = { + lookupTempView(identifier, isStreaming).map(ResolveRelations.resolveViews) } } @@ -1093,6 +1102,9 @@ class Analyzer(override val catalogManager: CatalogManager) executeSameContext(child) } } + // Fail the analysis eagerly because outside AnalysisContext, the unresolved operators + // inside a view maybe resolved incorrectly. + checkAnalysis(newChild) view.copy(child = newChild) case p @ SubqueryAlias(_, view: View) => p.copy(child = resolveViews(view)) @@ -1293,7 +1305,11 @@ class Analyzer(override val catalogManager: CatalogManager) relation.output.flatMap { col => outputNameToStaticName.get(col.name).flatMap(staticPartitions.get) match { case Some(staticValue) => - Some(Alias(Cast(Literal(staticValue), col.dataType), col.name)()) + // SPARK-30844: try our best to follow StoreAssignmentPolicy for static partition + // values but not completely follow because we can't do static type checking due to + // the reason that the parser has erased the type info of static partition values + // and converted them to string. + Some(Alias(AnsiCast(Literal(staticValue), col.dataType), col.name)()) case _ if queryColumns.hasNext => Some(queryColumns.next) case _ => @@ -1321,7 +1337,9 @@ class Analyzer(override val catalogManager: CatalogManager) // ResolveOutputRelation runs, using the query's column names that will match the // table names at that point. because resolution happens after a future rule, create // an UnresolvedAttribute. - EqualNullSafe(UnresolvedAttribute(attr.name), Cast(Literal(value), attr.dataType)) + EqualNullSafe( + UnresolvedAttribute.quoted(attr.name), + Cast(Literal(value), attr.dataType)) case None => throw QueryCompilationErrors.unknownStaticPartitionColError(name) } @@ -1361,13 +1379,16 @@ class Analyzer(override val catalogManager: CatalogManager) case oldVersion: SerializeFromObject if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => - Seq((oldVersion, oldVersion.copy( - serializer = oldVersion.serializer.map(_.newInstance())))) + val newVersion = oldVersion.copy(serializer = oldVersion.serializer.map(_.newInstance())) + newVersion.copyTagsFrom(oldVersion) + Seq((oldVersion, newVersion)) // Handle projects that create conflicting aliases. case oldVersion @ Project(projectList, _) if findAliases(projectList).intersect(conflictingAttributes).nonEmpty => - Seq((oldVersion, oldVersion.copy(projectList = newAliases(projectList)))) + val newVersion = oldVersion.copy(projectList = newAliases(projectList)) + newVersion.copyTagsFrom(oldVersion) + Seq((oldVersion, newVersion)) // We don't need to search child plan recursively if the projectList of a Project // is only composed of Alias and doesn't contain any conflicting attributes. @@ -1379,8 +1400,9 @@ class Analyzer(override val catalogManager: CatalogManager) case oldVersion @ Aggregate(_, aggregateExpressions, _) if findAliases(aggregateExpressions).intersect(conflictingAttributes).nonEmpty => - Seq((oldVersion, oldVersion.copy( - aggregateExpressions = newAliases(aggregateExpressions)))) + val newVersion = oldVersion.copy(aggregateExpressions = newAliases(aggregateExpressions)) + newVersion.copyTagsFrom(oldVersion) + Seq((oldVersion, newVersion)) // We don't search the child plan recursively for the same reason as the above Project. case _ @ Aggregate(_, aggregateExpressions, _) @@ -1389,20 +1411,28 @@ class Analyzer(override val catalogManager: CatalogManager) case oldVersion @ FlatMapGroupsInPandas(_, _, output, _) if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => - Seq((oldVersion, oldVersion.copy(output = output.map(_.newInstance())))) + val newVersion = oldVersion.copy(output = output.map(_.newInstance())) + newVersion.copyTagsFrom(oldVersion) + Seq((oldVersion, newVersion)) case oldVersion @ FlatMapCoGroupsInPandas(_, _, _, output, _, _) if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => - Seq((oldVersion, oldVersion.copy(output = output.map(_.newInstance())))) + val newVersion = oldVersion.copy(output = output.map(_.newInstance())) + newVersion.copyTagsFrom(oldVersion) + Seq((oldVersion, newVersion)) case oldVersion @ MapInPandas(_, output, _) if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => - Seq((oldVersion, oldVersion.copy(output = output.map(_.newInstance())))) + val newVersion = oldVersion.copy(output = output.map(_.newInstance())) + newVersion.copyTagsFrom(oldVersion) + Seq((oldVersion, newVersion)) case oldVersion: Generate if oldVersion.producedAttributes.intersect(conflictingAttributes).nonEmpty => val newOutput = oldVersion.generatorOutput.map(_.newInstance()) - Seq((oldVersion, oldVersion.copy(generatorOutput = newOutput))) + val newVersion = oldVersion.copy(generatorOutput = newOutput) + newVersion.copyTagsFrom(oldVersion) + Seq((oldVersion, newVersion)) case oldVersion: Expand if oldVersion.producedAttributes.intersect(conflictingAttributes).nonEmpty => @@ -1414,12 +1444,22 @@ class Analyzer(override val catalogManager: CatalogManager) attr } } - Seq((oldVersion, oldVersion.copy(output = newOutput))) + val newVersion = oldVersion.copy(output = newOutput) + newVersion.copyTagsFrom(oldVersion) + Seq((oldVersion, newVersion)) case oldVersion @ Window(windowExpressions, _, _, child) if AttributeSet(windowExpressions.map(_.toAttribute)).intersect(conflictingAttributes) .nonEmpty => - Seq((oldVersion, oldVersion.copy(windowExpressions = newAliases(windowExpressions)))) + val newVersion = oldVersion.copy(windowExpressions = newAliases(windowExpressions)) + newVersion.copyTagsFrom(oldVersion) + Seq((oldVersion, newVersion)) + + case oldVersion @ ScriptTransformation(_, _, output, _, _) + if AttributeSet(output).intersect(conflictingAttributes).nonEmpty => + val newVersion = oldVersion.copy(output = output.map(_.newInstance())) + newVersion.copyTagsFrom(oldVersion) + Seq((oldVersion, newVersion)) case _ => plan.children.flatMap(collectConflictPlans) } @@ -3816,10 +3856,18 @@ object TimeWindowing extends Rule[LogicalPlan] { } /** - * Resolve a [[CreateNamedStruct]] if it contains [[NamePlaceholder]]s. + * Resolve expressions if they contains [[NamePlaceholder]]s. */ -object ResolveCreateNamedStruct extends Rule[LogicalPlan] { +object ResolveExpressionsWithNamePlaceholders extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveExpressions { + case e: ArraysZip if !e.resolved => + val names = e.children.zip(e.names).map { + case (e: NamedExpression, NamePlaceholder) if e.resolved => + Literal(e.name) + case (_, other) => other + } + ArraysZip(e.children, names) + case e: CreateNamedStruct if !e.resolved => val children = e.children.grouped(2).flatMap { case Seq(NamePlaceholder, e: NamedExpression) if e.resolved => @@ -3903,22 +3951,32 @@ object UpdateOuterReferences extends Rule[LogicalPlan] { */ object ApplyCharTypePadding extends Rule[LogicalPlan] { + object AttrOrOuterRef { + def unapply(e: Expression): Option[Attribute] = e match { + case a: Attribute => Some(a) + case OuterReference(a: Attribute) => Some(a) + case _ => None + } + } + override def apply(plan: LogicalPlan): LogicalPlan = { plan.resolveOperatorsUp { - case operator if operator.resolved => operator.transformExpressionsUp { + case operator => operator.transformExpressionsUp { + case e if !e.childrenResolved => e + // String literal is treated as char type when it's compared to a char type column. // We should pad the shorter one to the longer length. - case b @ BinaryComparison(attr: Attribute, lit) if lit.foldable => - padAttrLitCmp(attr, lit).map { newChildren => + case b @ BinaryComparison(e @ AttrOrOuterRef(attr), lit) if lit.foldable => + padAttrLitCmp(e, attr.metadata, lit).map { newChildren => b.withNewChildren(newChildren) }.getOrElse(b) - case b @ BinaryComparison(lit, attr: Attribute) if lit.foldable => - padAttrLitCmp(attr, lit).map { newChildren => + case b @ BinaryComparison(lit, e @ AttrOrOuterRef(attr)) if lit.foldable => + padAttrLitCmp(e, attr.metadata, lit).map { newChildren => b.withNewChildren(newChildren.reverse) }.getOrElse(b) - case i @ In(attr: Attribute, list) + case i @ In(e @ AttrOrOuterRef(attr), list) if attr.dataType == StringType && list.forall(_.foldable) => CharVarcharUtils.getRawType(attr.metadata).flatMap { case CharType(length) => @@ -3927,7 +3985,7 @@ object ApplyCharTypePadding extends Rule[LogicalPlan] { val literalCharLengths = literalChars.map(_.numChars()) val targetLen = (length +: literalCharLengths).max Some(i.copy( - value = addPadding(attr, length, targetLen), + value = addPadding(e, length, targetLen), list = list.zip(literalCharLengths).map { case (lit, charLength) => addPadding(lit, charLength, targetLen) } ++ nulls.map(Literal.create(_, StringType)))) @@ -3935,20 +3993,46 @@ object ApplyCharTypePadding extends Rule[LogicalPlan] { }.getOrElse(i) // For char type column or inner field comparison, pad the shorter one to the longer length. - case b @ BinaryComparison(left: Attribute, right: Attribute) => - b.withNewChildren(CharVarcharUtils.addPaddingInStringComparison(Seq(left, right))) + case b @ BinaryComparison(e1 @ AttrOrOuterRef(left), e2 @ AttrOrOuterRef(right)) + // For the same attribute, they must be the same length and no padding is needed. + if !left.semanticEquals(right) => + val outerRefs = (e1, e2) match { + case (_: OuterReference, _: OuterReference) => Seq(left, right) + case (_: OuterReference, _) => Seq(left) + case (_, _: OuterReference) => Seq(right) + case _ => Nil + } + val newChildren = CharVarcharUtils.addPaddingInStringComparison(Seq(left, right)) + if (outerRefs.nonEmpty) { + b.withNewChildren(newChildren.map(_.transform { + case a: Attribute if outerRefs.exists(_.semanticEquals(a)) => OuterReference(a) + })) + } else { + b.withNewChildren(newChildren) + } - case i @ In(attr: Attribute, list) if list.forall(_.isInstanceOf[Attribute]) => + case i @ In(e @ AttrOrOuterRef(attr), list) if list.forall(_.isInstanceOf[Attribute]) => val newChildren = CharVarcharUtils.addPaddingInStringComparison( attr +: list.map(_.asInstanceOf[Attribute])) - i.copy(value = newChildren.head, list = newChildren.tail) + if (e.isInstanceOf[OuterReference]) { + i.copy( + value = newChildren.head.transform { + case a: Attribute if a.semanticEquals(attr) => OuterReference(a) + }, + list = newChildren.tail) + } else { + i.copy(value = newChildren.head, list = newChildren.tail) + } } } } - private def padAttrLitCmp(attr: Attribute, lit: Expression): Option[Seq[Expression]] = { - if (attr.dataType == StringType) { - CharVarcharUtils.getRawType(attr.metadata).flatMap { + private def padAttrLitCmp( + expr: Expression, + metadata: Metadata, + lit: Expression): Option[Seq[Expression]] = { + if (expr.dataType == StringType) { + CharVarcharUtils.getRawType(metadata).flatMap { case CharType(length) => val str = lit.eval().asInstanceOf[UTF8String] if (str == null) { @@ -3956,9 +4040,9 @@ object ApplyCharTypePadding extends Rule[LogicalPlan] { } else { val stringLitLen = str.numChars() if (length < stringLitLen) { - Some(Seq(StringRPad(attr, Literal(stringLitLen)), lit)) + Some(Seq(StringRPad(expr, Literal(stringLitLen)), lit)) } else if (length > stringLitLen) { - Some(Seq(attr, StringRPad(lit, Literal(length)))) + Some(Seq(expr, StringRPad(lit, Literal(length)))) } else { None } @@ -3970,6 +4054,14 @@ object ApplyCharTypePadding extends Rule[LogicalPlan] { } } + private def padOuterRefAttrCmp(outerAttr: Attribute, attr: Attribute): Seq[Expression] = { + val Seq(r, newAttr) = CharVarcharUtils.addPaddingInStringComparison(Seq(outerAttr, attr)) + val newOuterRef = r.transform { + case ar: Attribute if ar.semanticEquals(outerAttr) => OuterReference(ar) + } + Seq(newOuterRef, newAttr) + } + private def addPadding(expr: Expression, charLength: Int, targetLength: Int): Expression = { if (targetLength > charLength) StringRPad(expr, Literal(targetLength)) else expr } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 442b22c0bbc05..6cd6f240b5023 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -112,6 +112,9 @@ trait CheckAnalysis extends PredicateHelper { case u: UnresolvedRelation => u.failAnalysis(s"Table or view not found: ${u.multipartIdentifier.quoted}") + case u: UnresolvedHint => + u.failAnalysis(s"Hint not found: ${u.name}") + case InsertIntoStatement(u: UnresolvedRelation, _, _, _, _, _) => failAnalysis(s"Table not found: ${u.multipartIdentifier.quoted}") @@ -927,14 +930,72 @@ trait CheckAnalysis extends PredicateHelper { // +- SubqueryAlias t1, `t1` // +- Project [_1#73 AS c1#76, _2#74 AS c2#77] // +- LocalRelation [_1#73, _2#74] - def failOnNonEqualCorrelatedPredicate(found: Boolean, p: LogicalPlan): Unit = { - if (found) { + // SPARK-35080: The same issue can happen to correlated equality predicates when + // they do not guarantee one-to-one mapping between inner and outer attributes. + // For example: + // Table: + // t1(a, b): [(0, 6), (1, 5), (2, 4)] + // t2(c): [(6)] + // + // Query: + // SELECT c, (SELECT COUNT(*) FROM t1 WHERE a + b = c) FROM t2 + // + // Original subquery plan: + // Aggregate [count(1)] + // +- Filter ((a + b) = outer(c)) + // +- LocalRelation [a, b] + // + // Plan after pulling up correlated predicates: + // Aggregate [a, b] [count(1), a, b] + // +- LocalRelation [a, b] + // + // Plan after rewrite: + // Project [c1, count(1)] + // +- Join LeftOuter ((a + b) = c) + // :- LocalRelation [c] + // +- Aggregate [a, b] [count(1), a, b] + // +- LocalRelation [a, b] + // + // The right hand side of the join transformed from the subquery will output + // count(1) | a | b + // 1 | 0 | 6 + // 1 | 1 | 5 + // 1 | 2 | 4 + // and the plan after rewrite will give the original query incorrect results. + def failOnUnsupportedCorrelatedPredicate(predicates: Seq[Expression], p: LogicalPlan): Unit = { + if (predicates.nonEmpty) { // Report a non-supported case as an exception - failAnalysis(s"Correlated column is not allowed in a non-equality predicate:\n$p") + failAnalysis("Correlated column is not allowed in predicate " + + s"${predicates.map(_.sql).mkString}:\n$p") } } - var foundNonEqualCorrelatedPred: Boolean = false + def containsAttribute(e: Expression): Boolean = { + e.find(_.isInstanceOf[Attribute]).isDefined + } + + // Given a correlated predicate, check if it is either a non-equality predicate or + // equality predicate that does not guarantee one-on-one mapping between inner and + // outer attributes. When the correlated predicate does not contain any attribute + // (i.e. only has outer references), it is supported and should return false. E.G.: + // (a = outer(c)) -> false + // (outer(c) = outer(d)) -> false + // (a > outer(c)) -> true + // (a + b = outer(c)) -> true + // The last one is true because there can be multiple combinations of (a, b) that + // satisfy the equality condition. For example, if outer(c) = 0, then both (0, 0) + // and (-1, 1) can make the predicate evaluate to true. + def isUnsupportedPredicate(condition: Expression): Boolean = condition match { + // Only allow equality condition with one side being an attribute and another + // side being an expression without attributes from the inner query. Note + // OuterReference is a leaf node and will not be found here. + case Equality(_: Attribute, b) => containsAttribute(b) + case Equality(a, _: Attribute) => containsAttribute(a) + case e @ Equality(_, _) => containsAttribute(e) + case _ => true + } + + val unsupportedPredicates = mutable.ArrayBuffer.empty[Expression] // Simplify the predicates before validating any unsupported correlation patterns in the plan. AnalysisHelper.allowInvokingTransformsInAnalyzer { BooleanSimplification(sub).foreachUp { @@ -977,22 +1038,17 @@ trait CheckAnalysis extends PredicateHelper { // The other operator is Join. Filter can be anywhere in a correlated subquery. case f: Filter => val (correlated, _) = splitConjunctivePredicates(f.condition).partition(containsOuter) - - // Find any non-equality correlated predicates - foundNonEqualCorrelatedPred = foundNonEqualCorrelatedPred || correlated.exists { - case _: EqualTo | _: EqualNullSafe => false - case _ => true - } + unsupportedPredicates ++= correlated.filter(isUnsupportedPredicate) failOnInvalidOuterReference(f) // Aggregate cannot host any correlated expressions // It can be on a correlation path if the correlation contains - // only equality correlated predicates. + // only supported correlated equality predicates. // It cannot be on a correlation path if the correlation has // non-equality correlated predicates. case a: Aggregate => failOnInvalidOuterReference(a) - failOnNonEqualCorrelatedPredicate(foundNonEqualCorrelatedPred, a) + failOnUnsupportedCorrelatedPredicate(unsupportedPredicates.toSeq, a) // Join can host correlated expressions. case j @ Join(left, right, joinType, _, _) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala index d5c407b47c5be..32bdb82b3b68e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala @@ -100,7 +100,11 @@ object TableOutputResolver { case _ => Cast(queryExpr, tableAttr.dataType, Option(conf.sessionLocalTimeZone)) } - val exprWithStrLenCheck = CharVarcharUtils.stringLengthCheck(casted, tableAttr) + val exprWithStrLenCheck = if (conf.charVarcharAsString) { + casted + } else { + CharVarcharUtils.stringLengthCheck(casted, tableAttr) + } // Renaming is needed for handling the following cases like // 1) Column names/types do not match, e.g., INSERT INTO TABLE tab1 SELECT 1, 2 // 2) Target tables have column metadata diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala index 4b132d8ab6c7d..c2b74604715b2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala @@ -119,12 +119,16 @@ object ExternalCatalogUtils { } } - def getPartitionPathString(col: String, value: String): String = { - val partitionString = if (value == null || value.isEmpty) { + def getPartitionValueString(value: String): String = { + if (value == null || value.isEmpty) { DEFAULT_PARTITION_NAME } else { escapePathName(value) } + } + + def getPartitionPathString(col: String, value: String): String = { + val partitionString = getPartitionValueString(value) escapePathName(col) + "=" + partitionString } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index 90e69469eef69..5809751967cb3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -342,8 +342,7 @@ class InMemoryCatalog( } override def tableExists(db: String, table: String): Boolean = synchronized { - requireDbExists(db) - catalog(db).tables.contains(table) + catalog.contains(db) && catalog(db).tables.contains(table) } override def listTables(db: String): Seq[String] = synchronized { @@ -500,7 +499,11 @@ class InMemoryCatalog( newSpec, partitionColumnNames, tablePath) try { val fs = tablePath.getFileSystem(hadoopConfig) - fs.rename(oldPartPath, newPartPath) + fs.mkdirs(newPartPath) + if(!fs.rename(oldPartPath, newPartPath)) { + throw new IOException(s"Renaming partition path from $oldPartPath to " + + s"$newPartPath returned false") + } } catch { case e: IOException => throw new SparkException(s"Unable to rename partition path $oldPartPath", e) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 0766cac1e7390..d08af6c7bae2d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -172,6 +172,13 @@ class SessionCatalog( tableRelationCache.invalidate(key) } + /** This method discards any cached table relation plans for the given table identifier. */ + def invalidateCachedTable(name: TableIdentifier): Unit = { + val dbName = formatDatabaseName(name.database.getOrElse(currentDb)) + val tableName = formatTableName(name.table) + invalidateCachedTable(QualifiedTableName(dbName, tableName)) + } + /** This method provides a way to invalidate all the cached plans. */ def invalidateAllCachedTables(): Unit = { tableRelationCache.invalidateAll() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala index 665b2cd1274fd..2d8f02860dd7d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala @@ -48,7 +48,7 @@ object OuterScopes { * useful for inner class defined in REPL. */ def getOuterScope(innerCls: Class[_]): () => AnyRef = { - assert(innerCls.isMemberClass) + assert(Utils.isMemberClass(innerCls)) val outerClassName = innerCls.getDeclaringClass.getName val outer = outerScopes.get(outerClassName) if (outer == null) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AliasHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AliasHelper.scala index 7eb95e6a84f15..4c94fa448cefe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AliasHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AliasHelper.scala @@ -72,7 +72,7 @@ trait AliasHelper { // Use transformUp to prevent infinite recursion when the replacement expression // redefines the same ExprId, trimNonTopLevelAliases(expr.transformUp { - case a: Attribute => aliasMap.getOrElse(a, a) + case a: Attribute => aliasMap.get(a).map(_.withName(a.name)).getOrElse(a) }).asInstanceOf[NamedExpression] } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 1d215cf492a67..9a53eebe6c482 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -1301,7 +1301,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit if (Float.isNaN($c) || Float.isInfinite($c)) { $evNull = true; } else { - $evPrim = (long)($c * $MICROS_PER_SECOND); + $evPrim = (long)((double)$c * $MICROS_PER_SECOND); } """ } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala index 1dfff412d9a8e..a26c19903186a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala @@ -120,6 +120,7 @@ class EquivalentExpressions { // For some special expressions we cannot just recurse into all of its children, but we can // recursively add the common expressions shared between all of its children. private def commonChildrenToRecurse(expr: Expression): Seq[Seq[Expression]] = expr match { + case _: CodegenFallback => Nil case i: If => Seq(Seq(i.trueValue, i.falseValue)) case c: CaseWhen => // We look at subexpressions in conditions and values of `CaseWhen` separately. It is @@ -128,7 +129,13 @@ class EquivalentExpressions { // a subexpression among values doesn't need to be in conditions because no matter which // condition is true, it will be evaluated. val conditions = c.branches.tail.map(_._1) - val values = c.branches.map(_._2) ++ c.elseValue + // For an expression to be in all branch values of a CaseWhen statement, it must also be in + // the elseValue. + val values = if (c.elseValue.nonEmpty) { + c.branches.map(_._2) ++ c.elseValue + } else { + Nil + } Seq(conditions, values) case c: Coalesce => Seq(c.children.tail) case _ => Nil diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ProjectionOverSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ProjectionOverSchema.scala index 241c761624b76..03b5517f6df05 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ProjectionOverSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ProjectionOverSchema.scala @@ -41,9 +41,14 @@ case class ProjectionOverSchema(schema: StructType) { case a: GetArrayStructFields => getProjection(a.child).map(p => (p, p.dataType)).map { case (projection, ArrayType(projSchema @ StructType(_), _)) => + // For case-sensitivity aware field resolution, we should take `ordinal` which + // points to correct struct field. + val selectedField = a.child.dataType.asInstanceOf[ArrayType] + .elementType.asInstanceOf[StructType](a.ordinal) + val prunedField = projSchema(selectedField.name) GetArrayStructFields(projection, - projSchema(a.field.name), - projSchema.fieldIndex(a.field.name), + prunedField.copy(name = a.field.name), + projSchema.fieldIndex(selectedField.name), projSchema.size, a.containsNull) case (_, projSchema) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruning.scala index 6213267c41c64..30093ef085913 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruning.scala @@ -17,26 +17,30 @@ package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.SQLConfHelper import org.apache.spark.sql.types._ -object SchemaPruning { +object SchemaPruning extends SQLConfHelper { /** - * Filters the schema by the requested fields. For example, if the schema is struct, - * and given requested field are "a", the field "b" is pruned in the returned schema. - * Note that schema field ordering at original schema is still preserved in pruned schema. + * Prunes the nested schema by the requested fields. For example, if the schema is: + * `id int, s struct`, and given requested field "s.a", the inner field "b" + * is pruned in the returned schema: `id int, s struct`. + * Note that: + * 1. The schema field ordering at original schema is still preserved in pruned schema. + * 2. The top-level fields are not pruned here. */ def pruneDataSchema( dataSchema: StructType, requestedRootFields: Seq[RootField]): StructType = { + val resolver = conf.resolver // Merge the requested root fields into a single schema. Note the ordering of the fields // in the resulting schema may differ from their ordering in the logical relation's // original schema val mergedSchema = requestedRootFields - .map { case root: RootField => StructType(Array(root.field)) } + .map { root: RootField => StructType(Array(root.field)) } .reduceLeft(_ merge _) - val dataSchemaFieldNames = dataSchema.fieldNames.toSet val mergedDataSchema = - StructType(mergedSchema.filter(f => dataSchemaFieldNames.contains(f.name))) + StructType(dataSchema.map(d => mergedSchema.find(m => resolver(m.name, d.name)).getOrElse(d))) // Sort the fields of mergedDataSchema according to their order in dataSchema, // recursively. This makes mergedDataSchema a pruned schema of dataSchema sortLeftFieldsByRight(mergedDataSchema, dataSchema).asInstanceOf[StructType] @@ -61,12 +65,15 @@ object SchemaPruning { sortLeftFieldsByRight(leftValueType, rightValueType), containsNull) case (leftStruct: StructType, rightStruct: StructType) => - val filteredRightFieldNames = rightStruct.fieldNames.filter(leftStruct.fieldNames.contains) + val resolver = conf.resolver + val filteredRightFieldNames = rightStruct.fieldNames + .filter(name => leftStruct.fieldNames.exists(resolver(_, name))) val sortedLeftFields = filteredRightFieldNames.map { fieldName => - val leftFieldType = leftStruct(fieldName).dataType + val resolvedLeftStruct = leftStruct.find(p => resolver(p.name, fieldName)).get + val leftFieldType = resolvedLeftStruct.dataType val rightFieldType = rightStruct(fieldName).dataType val sortedLeftFieldType = sortLeftFieldsByRight(leftFieldType, rightFieldType) - StructField(fieldName, sortedLeftFieldType, nullable = leftStruct(fieldName).nullable) + StructField(fieldName, sortedLeftFieldType, nullable = resolvedLeftStruct.nullable) } StructType(sortedLeftFields) case _ => left diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SelectedField.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SelectedField.scala index f2acb75ea6ac4..39dfdf99a9a2b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SelectedField.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SelectedField.scala @@ -75,7 +75,11 @@ object SelectedField { val field = c.childSchema(c.ordinal) val newField = field.copy(dataType = dataTypeOpt.getOrElse(field.dataType)) selectField(c.child, Option(struct(newField))) - case GetArrayStructFields(child, field, _, _, containsNull) => + case GetArrayStructFields(child, _, ordinal, _, containsNull) => + // For case-sensitivity aware field resolution, we should take `ordinal` which + // points to correct struct field. + val field = child.dataType.asInstanceOf[ArrayType] + .elementType.asInstanceOf[StructType](ordinal) val newFieldDataType = dataTypeOpt match { case None => // GetArrayStructFields is the top level extractor. This means its result is diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala index 2a5275e75d4f9..6db41b4082013 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala @@ -274,7 +274,7 @@ object ApproximatePercentile { } /** - * Serializer for class [[PercentileDigest]] + * Serializer for class [[PercentileDigest]] * * This class is thread safe. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountMinSketchAgg.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountMinSketchAgg.scala index 8b51e0a908f42..45d55a085a717 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountMinSketchAgg.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountMinSketchAgg.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions.aggregate import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} -import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription} +import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.sketch.CountMinSketch @@ -142,6 +142,9 @@ case class CountMinSketchAgg( override def dataType: DataType = BinaryType + override def defaultResult: Option[Literal] = + Option(Literal.create(eval(createAggregationBuffer()), dataType)) + override def children: Seq[Expression] = Seq(child, epsExpression, confidenceExpression, seedExpression) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala index b3cc9a31fec2f..b6a4d116b16b7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala @@ -90,6 +90,8 @@ case class HyperLogLogPlusPlus( override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes) + override def defaultResult: Option[Literal] = Option(Literal.create(0L, dataType)) + val hllppHelper = new HyperLogLogPlusPlusHelper(relativeSD) /** Allocate enough words to store all registers. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala index f95f44c808092..f1b9630312d55 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala @@ -47,6 +47,8 @@ abstract class Collect[T <: Growable[Any] with Iterable[Any]] extends TypedImper // actual order of input rows. override lazy val deterministic: Boolean = false + override def defaultResult: Option[Literal] = Option(Literal.create(Array(), dataType)) + protected def convertToBufferElement(value: Any): Any override def update(buffer: T, input: InternalRow): T = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala index 421b8ee2a25b2..9dd370445ec46 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala @@ -202,8 +202,7 @@ abstract class AggregateFunction extends Expression { def inputAggBufferAttributes: Seq[AttributeReference] /** - * Result of the aggregate function when the input is empty. This is currently only used for the - * proper rewriting of distinct aggregate functions. + * Result of the aggregate function when the input is empty. */ def defaultResult: Option[Literal] = None diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 1ff4a93cf0acd..4092436933413 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -334,7 +334,7 @@ class CodegenContext extends Logging { */ def addBufferedState(dataType: DataType, variableName: String, initCode: String): ExprCode = { val value = addMutableState(javaType(dataType), variableName) - val code = dataType match { + val code = UserDefinedType.sqlType(dataType) match { case StringType => code"$value = $initCode.clone();" case _: StructType | _: ArrayType | _: MapType => code"$value = $initCode.copy();" case _ => code"$value = $initCode;" @@ -1089,7 +1089,8 @@ class CodegenContext extends Logging { // Generate the code for this expression tree and wrap it in a function. val fnName = freshName("subExpr") val inputVars = inputVarsForAllFuncs(i) - val argList = inputVars.map(v => s"${v.javaType.getName} ${v.variableName}") + val argList = + inputVars.map(v => s"${CodeGenerator.typeName(v.javaType)} ${v.variableName}") val returnType = javaType(expr.dataType) val fn = s""" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index 0765bfdd78fa6..94291e66c9ce7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -23,7 +23,7 @@ import scala.collection.mutable import scala.reflect.ClassTag import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion} +import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion, UnresolvedAttribute} import org.apache.spark.sql.catalyst.expressions.ArraySortLike.NullOrder import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ @@ -32,11 +32,11 @@ import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.SQLOpenHashSet import org.apache.spark.unsafe.UTF8StringBuilder import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.unsafe.array.ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH import org.apache.spark.unsafe.types.{ByteArray, CalendarInterval, UTF8String} -import org.apache.spark.util.collection.OpenHashSet /** * Base trait for [[BinaryExpression]]s with two arrays of the same element type and implicit @@ -174,16 +174,36 @@ case class MapKeys(child: Expression) """, group = "array_funcs", since = "2.4.0") -case class ArraysZip(children: Seq[Expression]) extends Expression with ExpectsInputTypes { +case class ArraysZip(children: Seq[Expression], names: Seq[Expression]) + extends Expression with ExpectsInputTypes { + + def this(children: Seq[Expression]) = { + this( + children, + children.zipWithIndex.map { + case (u: UnresolvedAttribute, _) => Literal(u.nameParts.last) + case (e: NamedExpression, _) if e.resolved => Literal(e.name) + case (e: NamedExpression, _) => NamePlaceholder + case (_, idx) => Literal(idx.toString) + }) + } + + if (children.size != names.size) { + throw new IllegalArgumentException( + "The numbers of zipped arrays and field names should be the same") + } + + override lazy val resolved: Boolean = + childrenResolved && checkInputDataTypes().isSuccess && names.forall(_.resolved) override def inputTypes: Seq[AbstractDataType] = Seq.fill(children.length)(ArrayType) @transient override lazy val dataType: DataType = { - val fields = children.zip(arrayElementTypes).zipWithIndex.map { - case ((expr: NamedExpression, elementType), _) => - StructField(expr.name, elementType, nullable = true) - case ((_, elementType), idx) => - StructField(idx.toString, elementType, nullable = true) + val fields = arrayElementTypes.zip(names).map { + case (elementType, Literal(name, StringType)) => + StructField(name.toString, elementType, nullable = true) + case _ => + throw new IllegalStateException("Schema name of arrays_zip should be string literal.") } ArrayType(StructType(fields), containsNull = false) } @@ -322,6 +342,12 @@ case class ArraysZip(children: Seq[Expression]) extends Expression with ExpectsI override def prettyName: String = "arrays_zip" } +object ArraysZip { + def apply(children: Seq[Expression]): ArraysZip = { + new ArraysZip(children) + } +} + /** * Returns an unordered array containing the values of the map. */ @@ -1235,12 +1261,12 @@ case class ArraysOverlap(left: Expression, right: Expression) (arr2, arr1) } if (smaller.numElements() > 0) { - val smallestSet = new mutable.HashSet[Any] + val smallestSet = new java.util.HashSet[Any]() smaller.foreach(elementType, (_, v) => if (v == null) { hasNull = true } else { - smallestSet += v + smallestSet.add(v) }) bigger.foreach(elementType, (_, v1) => if (v1 == null) { @@ -2685,7 +2711,7 @@ object Sequence { val maxEstimatedArrayLength = getSequenceLength(startMicros, stopMicros, intervalStepInMicros) - val stepSign = if (stopMicros >= startMicros) +1 else -1 + val stepSign = if (intervalStepInMicros > 0) +1 else -1 val exclusiveItem = stopMicros + stepSign val arr = new Array[T](maxEstimatedArrayLength) var t = startMicros @@ -2760,7 +2786,7 @@ object Sequence { | | $sequenceLengthCode | - | final int $stepSign = $stopMicros >= $startMicros ? +1 : -1; + | final int $stepSign = $intervalInMicros > 0 ? +1 : -1; | final long $exclusiveItem = $stopMicros + $stepSign; | | $arr = new $elemType[$arrLength]; @@ -3100,7 +3126,9 @@ trait ArraySetLike { @transient protected lazy val nullValueHolder = et match { case ByteType => "(byte) 0" case ShortType => "(short) 0" - case LongType => "(long) 0" + case LongType => "0L" + case FloatType => "0.0f" + case DoubleType => "0.0" case _ => "0" } @@ -3176,32 +3204,59 @@ case class ArrayDistinct(child: Expression) } override def nullSafeEval(array: Any): Any = { - val data = array.asInstanceOf[ArrayData].toArray[AnyRef](elementType) + val data = array.asInstanceOf[ArrayData] doEvaluation(data) } @transient private lazy val doEvaluation = if (TypeUtils.typeWithProperEquals(elementType)) { - (data: Array[AnyRef]) => new GenericArrayData(data.distinct.asInstanceOf[Array[Any]]) + (array: ArrayData) => + val arrayBuffer = new scala.collection.mutable.ArrayBuffer[Any] + val hs = new SQLOpenHashSet[Any]() + val withNaNCheckFunc = SQLOpenHashSet.withNaNCheckFunc(elementType, hs, + (value: Any) => + if (!hs.contains(value)) { + if (arrayBuffer.size > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { + ArrayBinaryLike.throwUnionLengthOverflowException(arrayBuffer.size) + } + arrayBuffer += value + hs.add(value) + }, + (valueNaN: Any) => arrayBuffer += valueNaN) + var i = 0 + while (i < array.numElements()) { + if (array.isNullAt(i)) { + if (!hs.containsNull) { + hs.addNull + arrayBuffer += null + } + } else { + val elem = array.get(i, elementType) + withNaNCheckFunc(elem) + } + i += 1 + } + new GenericArrayData(arrayBuffer.toSeq) } else { - (data: Array[AnyRef]) => { + (data: ArrayData) => { + val array = data.toArray[AnyRef](elementType) val arrayBuffer = new scala.collection.mutable.ArrayBuffer[AnyRef] var alreadyStoredNull = false - for (i <- 0 until data.length) { - if (data(i) != null) { + for (i <- 0 until array.length) { + if (array(i) != null) { var found = false var j = 0 while (!found && j < arrayBuffer.size) { val va = arrayBuffer(j) - found = (va != null) && ordering.equiv(va, data(i)) + found = (va != null) && ordering.equiv(va, array(i)) j += 1 } if (!found) { - arrayBuffer += data(i) + arrayBuffer += array(i) } } else { // De-duplicate the null values. if (!alreadyStoredNull) { - arrayBuffer += data(i) + arrayBuffer += array(i) alreadyStoredNull = true } } @@ -3220,10 +3275,9 @@ case class ArrayDistinct(child: Expression) val ptName = CodeGenerator.primitiveTypeName(jt) nullSafeCodeGen(ctx, ev, (array) => { - val foundNullElement = ctx.freshName("foundNullElement") val nullElementIndex = ctx.freshName("nullElementIndex") val builder = ctx.freshName("builder") - val openHashSet = classOf[OpenHashSet[_]].getName + val openHashSet = classOf[SQLOpenHashSet[_]].getName val classTag = s"scala.reflect.ClassTag$$.MODULE$$.$hsTypeName()" val hashSet = ctx.freshName("hashSet") val arrayBuilder = classOf[mutable.ArrayBuilder[_]].getName @@ -3232,7 +3286,6 @@ case class ArrayDistinct(child: Expression) // Only need to track null element index when array's element is nullable. val declareNullTrackVariables = if (dataType.asInstanceOf[ArrayType].containsNull) { s""" - |boolean $foundNullElement = false; |int $nullElementIndex = -1; """.stripMargin } else { @@ -3243,9 +3296,9 @@ case class ArrayDistinct(child: Expression) if (dataType.asInstanceOf[ArrayType].containsNull) { s""" |if ($array.isNullAt($i)) { - | if (!$foundNullElement) { + | if (!$hashSet.containsNull()) { | $nullElementIndex = $size; - | $foundNullElement = true; + | $hashSet.addNull(); | $size++; | $builder.$$plus$$eq($nullValueHolder); | } @@ -3257,9 +3310,8 @@ case class ArrayDistinct(child: Expression) body } - val processArray = withArrayNullAssignment( + val body = s""" - |$jt $value = ${genGetValue(array, i)}; |if (!$hashSet.contains($hsValueCast$value)) { | if (++$size > ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}) { | break; @@ -3267,7 +3319,16 @@ case class ArrayDistinct(child: Expression) | $hashSet.add$hsPostFix($hsValueCast$value); | $builder.$$plus$$eq($value); |} - """.stripMargin) + """.stripMargin + + val processArray = withArrayNullAssignment( + s"$jt $value = ${genGetValue(array, i)};" + + SQLOpenHashSet.withNaNCheckCode(elementType, value, hashSet, body, + (valueNaN: String) => + s""" + |$size++; + |$builder.$$plus$$eq($valueNaN); + |""".stripMargin)) s""" |$openHashSet $hashSet = new $openHashSet$hsPostFix($classTag); @@ -3341,25 +3402,28 @@ case class ArrayUnion(left: Expression, right: Expression) extends ArrayBinaryLi if (TypeUtils.typeWithProperEquals(elementType)) { (array1, array2) => val arrayBuffer = new scala.collection.mutable.ArrayBuffer[Any] - val hs = new OpenHashSet[Any] - var foundNullElement = false + val hs = new SQLOpenHashSet[Any]() + val withNaNCheckFunc = SQLOpenHashSet.withNaNCheckFunc(elementType, hs, + (value: Any) => + if (!hs.contains(value)) { + if (arrayBuffer.size > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { + ArrayBinaryLike.throwUnionLengthOverflowException(arrayBuffer.size) + } + arrayBuffer += value + hs.add(value) + }, + (valueNaN: Any) => arrayBuffer += valueNaN) Seq(array1, array2).foreach { array => var i = 0 while (i < array.numElements()) { if (array.isNullAt(i)) { - if (!foundNullElement) { + if (!hs.containsNull) { + hs.addNull arrayBuffer += null - foundNullElement = true } } else { val elem = array.get(i, elementType) - if (!hs.contains(elem)) { - if (arrayBuffer.size > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { - ArrayBinaryLike.throwUnionLengthOverflowException(arrayBuffer.size) - } - arrayBuffer += elem - hs.add(elem) - } + withNaNCheckFunc(elem) } i += 1 } @@ -3415,13 +3479,12 @@ case class ArrayUnion(left: Expression, right: Expression) extends ArrayBinaryLi val ptName = CodeGenerator.primitiveTypeName(jt) nullSafeCodeGen(ctx, ev, (array1, array2) => { - val foundNullElement = ctx.freshName("foundNullElement") val nullElementIndex = ctx.freshName("nullElementIndex") val builder = ctx.freshName("builder") val array = ctx.freshName("array") val arrays = ctx.freshName("arrays") val arrayDataIdx = ctx.freshName("arrayDataIdx") - val openHashSet = classOf[OpenHashSet[_]].getName + val openHashSet = classOf[SQLOpenHashSet[_]].getName val classTag = s"scala.reflect.ClassTag$$.MODULE$$.$hsTypeName()" val hashSet = ctx.freshName("hashSet") val arrayBuilder = classOf[mutable.ArrayBuilder[_]].getName @@ -3431,9 +3494,9 @@ case class ArrayUnion(left: Expression, right: Expression) extends ArrayBinaryLi if (dataType.asInstanceOf[ArrayType].containsNull) { s""" |if ($array.isNullAt($i)) { - | if (!$foundNullElement) { + | if (!$hashSet.containsNull()) { | $nullElementIndex = $size; - | $foundNullElement = true; + | $hashSet.addNull(); | $size++; | $builder.$$plus$$eq($nullValueHolder); | } @@ -3445,9 +3508,8 @@ case class ArrayUnion(left: Expression, right: Expression) extends ArrayBinaryLi body } - val processArray = withArrayNullAssignment( + val body = s""" - |$jt $value = ${genGetValue(array, i)}; |if (!$hashSet.contains($hsValueCast$value)) { | if (++$size > ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}) { | break; @@ -3455,12 +3517,19 @@ case class ArrayUnion(left: Expression, right: Expression) extends ArrayBinaryLi | $hashSet.add$hsPostFix($hsValueCast$value); | $builder.$$plus$$eq($value); |} - """.stripMargin) + """.stripMargin + val processArray = withArrayNullAssignment( + s"$jt $value = ${genGetValue(array, i)};" + + SQLOpenHashSet.withNaNCheckCode(elementType, value, hashSet, body, + (valueNaN: String) => + s""" + |$size++; + |$builder.$$plus$$eq($valueNaN); + """.stripMargin)) // Only need to track null element index when result array's element is nullable. val declareNullTrackVariables = if (dataType.asInstanceOf[ArrayType].containsNull) { s""" - |boolean $foundNullElement = false; |int $nullElementIndex = -1; """.stripMargin } else { @@ -3562,33 +3631,42 @@ case class ArrayIntersect(left: Expression, right: Expression) extends ArrayBina if (TypeUtils.typeWithProperEquals(elementType)) { (array1, array2) => if (array1.numElements() != 0 && array2.numElements() != 0) { - val hs = new OpenHashSet[Any] - val hsResult = new OpenHashSet[Any] - var foundNullElement = false + val hs = new SQLOpenHashSet[Any] + val hsResult = new SQLOpenHashSet[Any] + val arrayBuffer = new scala.collection.mutable.ArrayBuffer[Any] + val withArray2NaNCheckFunc = SQLOpenHashSet.withNaNCheckFunc(elementType, hs, + (value: Any) => hs.add(value), + (valueNaN: Any) => {} ) + val withArray1NaNCheckFunc = SQLOpenHashSet.withNaNCheckFunc(elementType, hsResult, + (value: Any) => + if (hs.contains(value) && !hsResult.contains(value)) { + arrayBuffer += value + hsResult.add(value) + }, + (valueNaN: Any) => + if (hs.containsNaN()) { + arrayBuffer += valueNaN + }) var i = 0 while (i < array2.numElements()) { if (array2.isNullAt(i)) { - foundNullElement = true + hs.addNull() } else { val elem = array2.get(i, elementType) - hs.add(elem) + withArray2NaNCheckFunc(elem) } i += 1 } - val arrayBuffer = new scala.collection.mutable.ArrayBuffer[Any] i = 0 while (i < array1.numElements()) { if (array1.isNullAt(i)) { - if (foundNullElement) { + if (hs.containsNull() && !hsResult.containsNull()) { arrayBuffer += null - foundNullElement = false + hsResult.addNull() } } else { val elem = array1.get(i, elementType) - if (hs.contains(elem) && !hsResult.contains(elem)) { - arrayBuffer += elem - hsResult.add(elem) - } + withArray1NaNCheckFunc(elem) } i += 1 } @@ -3663,10 +3741,9 @@ case class ArrayIntersect(left: Expression, right: Expression) extends ArrayBina val ptName = CodeGenerator.primitiveTypeName(jt) nullSafeCodeGen(ctx, ev, (array1, array2) => { - val foundNullElement = ctx.freshName("foundNullElement") val nullElementIndex = ctx.freshName("nullElementIndex") val builder = ctx.freshName("builder") - val openHashSet = classOf[OpenHashSet[_]].getName + val openHashSet = classOf[SQLOpenHashSet[_]].getName val classTag = s"scala.reflect.ClassTag$$.MODULE$$.$hsTypeName()" val hashSet = ctx.freshName("hashSet") val hashSetResult = ctx.freshName("hashSetResult") @@ -3678,7 +3755,7 @@ case class ArrayIntersect(left: Expression, right: Expression) extends ArrayBina if (left.dataType.asInstanceOf[ArrayType].containsNull) { s""" |if ($array2.isNullAt($i)) { - | $foundNullElement = true; + | $hashSet.addNull(); |} else { | $body |} @@ -3696,19 +3773,18 @@ case class ArrayIntersect(left: Expression, right: Expression) extends ArrayBina } val writeArray2ToHashSet = withArray2NullCheck( - s""" - |$jt $value = ${genGetValue(array2, i)}; - |$hashSet.add$hsPostFix($hsValueCast$value); - """.stripMargin) + s"$jt $value = ${genGetValue(array2, i)};" + + SQLOpenHashSet.withNaNCheckCode(elementType, value, hashSet, + s"$hashSet.add$hsPostFix($hsValueCast$value);", (valueNaN: String) => "")) def withArray1NullAssignment(body: String) = if (left.dataType.asInstanceOf[ArrayType].containsNull) { if (right.dataType.asInstanceOf[ArrayType].containsNull) { s""" |if ($array1.isNullAt($i)) { - | if ($foundNullElement) { + | if ($hashSet.containsNull() && !$hashSetResult.containsNull()) { | $nullElementIndex = $size; - | $foundNullElement = false; + | $hashSetResult.addNull(); | $size++; | $builder.$$plus$$eq($nullValueHolder); | } @@ -3727,9 +3803,8 @@ case class ArrayIntersect(left: Expression, right: Expression) extends ArrayBina body } - val processArray1 = withArray1NullAssignment( + val body = s""" - |$jt $value = ${genGetValue(array1, i)}; |if ($hashSet.contains($hsValueCast$value) && | !$hashSetResult.contains($hsValueCast$value)) { | if (++$size > ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}) { @@ -3738,12 +3813,22 @@ case class ArrayIntersect(left: Expression, right: Expression) extends ArrayBina | $hashSetResult.add$hsPostFix($hsValueCast$value); | $builder.$$plus$$eq($value); |} - """.stripMargin) + """.stripMargin + + val processArray1 = withArray1NullAssignment( + s"$jt $value = ${genGetValue(array1, i)};" + + SQLOpenHashSet.withNaNCheckCode(elementType, value, hashSetResult, body, + (valueNaN: Any) => + s""" + |if ($hashSet.containsNaN()) { + | ++$size; + | $builder.$$plus$$eq($valueNaN); + |} + """.stripMargin)) // Only need to track null element index when result array's element is nullable. val declareNullTrackVariables = if (dataType.asInstanceOf[ArrayType].containsNull) { s""" - |boolean $foundNullElement = false; |int $nullElementIndex = -1; """.stripMargin } else { @@ -3804,32 +3889,38 @@ case class ArrayExcept(left: Expression, right: Expression) extends ArrayBinaryL @transient lazy val evalExcept: (ArrayData, ArrayData) => ArrayData = { if (TypeUtils.typeWithProperEquals(elementType)) { (array1, array2) => - val hs = new OpenHashSet[Any] - var notFoundNullElement = true + val hs = new SQLOpenHashSet[Any] + val arrayBuffer = new scala.collection.mutable.ArrayBuffer[Any] + val withArray2NaNCheckFunc = SQLOpenHashSet.withNaNCheckFunc(elementType, hs, + (value: Any) => hs.add(value), + (valueNaN: Any) => {}) + val withArray1NaNCheckFunc = SQLOpenHashSet.withNaNCheckFunc(elementType, hs, + (value: Any) => + if (!hs.contains(value)) { + arrayBuffer += value + hs.add(value) + }, + (valueNaN: Any) => arrayBuffer += valueNaN) var i = 0 while (i < array2.numElements()) { if (array2.isNullAt(i)) { - notFoundNullElement = false + hs.addNull() } else { val elem = array2.get(i, elementType) - hs.add(elem) + withArray2NaNCheckFunc(elem) } i += 1 } - val arrayBuffer = new scala.collection.mutable.ArrayBuffer[Any] i = 0 while (i < array1.numElements()) { if (array1.isNullAt(i)) { - if (notFoundNullElement) { + if (!hs.containsNull()) { arrayBuffer += null - notFoundNullElement = false + hs.addNull() } } else { val elem = array1.get(i, elementType) - if (!hs.contains(elem)) { - arrayBuffer += elem - hs.add(elem) - } + withArray1NaNCheckFunc(elem) } i += 1 } @@ -3898,10 +3989,9 @@ case class ArrayExcept(left: Expression, right: Expression) extends ArrayBinaryL val ptName = CodeGenerator.primitiveTypeName(jt) nullSafeCodeGen(ctx, ev, (array1, array2) => { - val notFoundNullElement = ctx.freshName("notFoundNullElement") val nullElementIndex = ctx.freshName("nullElementIndex") val builder = ctx.freshName("builder") - val openHashSet = classOf[OpenHashSet[_]].getName + val openHashSet = classOf[SQLOpenHashSet[_]].getName val classTag = s"scala.reflect.ClassTag$$.MODULE$$.$hsTypeName()" val hashSet = ctx.freshName("hashSet") val arrayBuilder = classOf[mutable.ArrayBuilder[_]].getName @@ -3912,7 +4002,7 @@ case class ArrayExcept(left: Expression, right: Expression) extends ArrayBinaryL if (left.dataType.asInstanceOf[ArrayType].containsNull) { s""" |if ($array2.isNullAt($i)) { - | $notFoundNullElement = false; + | $hashSet.addNull(); |} else { | $body |} @@ -3930,18 +4020,18 @@ case class ArrayExcept(left: Expression, right: Expression) extends ArrayBinaryL } val writeArray2ToHashSet = withArray2NullCheck( - s""" - |$jt $value = ${genGetValue(array2, i)}; - |$hashSet.add$hsPostFix($hsValueCast$value); - """.stripMargin) + s"$jt $value = ${genGetValue(array2, i)};" + + SQLOpenHashSet.withNaNCheckCode(elementType, value, hashSet, + s"$hashSet.add$hsPostFix($hsValueCast$value);", + (valueNaN: Any) => "")) def withArray1NullAssignment(body: String) = if (left.dataType.asInstanceOf[ArrayType].containsNull) { s""" |if ($array1.isNullAt($i)) { - | if ($notFoundNullElement) { + | if (!$hashSet.containsNull()) { + | $hashSet.addNull(); | $nullElementIndex = $size; - | $notFoundNullElement = false; | $size++; | $builder.$$plus$$eq($nullValueHolder); | } @@ -3953,9 +4043,8 @@ case class ArrayExcept(left: Expression, right: Expression) extends ArrayBinaryL body } - val processArray1 = withArray1NullAssignment( + val body = s""" - |$jt $value = ${genGetValue(array1, i)}; |if (!$hashSet.contains($hsValueCast$value)) { | if (++$size > ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}) { | break; @@ -3963,12 +4052,20 @@ case class ArrayExcept(left: Expression, right: Expression) extends ArrayBinaryL | $hashSet.add$hsPostFix($hsValueCast$value); | $builder.$$plus$$eq($value); |} - """.stripMargin) + """.stripMargin + + val processArray1 = withArray1NullAssignment( + s"$jt $value = ${genGetValue(array1, i)};" + + SQLOpenHashSet.withNaNCheckCode(elementType, value, hashSet, body, + (valueNaN: String) => + s""" + |$size++; + |$builder.$$plus$$eq($valueNaN); + """.stripMargin)) // Only need to track null element index when array1's element is nullable. val declareNullTrackVariables = if (left.dataType.asInstanceOf[ArrayType].containsNull) { s""" - |boolean $notFoundNullElement = true; |int $nullElementIndex = -1; """.stripMargin } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index cb59fbda2b3b9..1779d413e025d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.{Resolver, TypeCheckResult, TypeCoercion, UnresolvedExtractValue} +import org.apache.spark.sql.catalyst.analysis.{Resolver, TypeCheckResult, TypeCoercion, UnresolvedAttribute, UnresolvedExtractValue} import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.{FUNC_ALIAS, FunctionBuilder} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ @@ -336,6 +336,14 @@ object CreateStruct { */ def apply(children: Seq[Expression]): CreateNamedStruct = { CreateNamedStruct(children.zipWithIndex.flatMap { + // For multi-part column name like `struct(a.b.c)`, it may be resolved into: + // 1. Attribute if `a.b.c` is simply a qualified column name. + // 2. GetStructField if `a.b` refers to a struct-type column. + // 3. GetArrayStructFields if `a.b` refers to a array-of-struct-type column. + // 4. GetMapValue if `a.b` refers to a map-type column. + // We should always use the last part of the column name (`c` in the above example) as the + // alias name inside CreateNamedStruct. + case (u: UnresolvedAttribute, _) => Seq(Literal(u.nameParts.last), u) case (e: NamedExpression, _) if e.resolved => Seq(Literal(e.name), e) case (e: NamedExpression, _) => Seq(NamePlaceholder, e) case (e, index) => Seq(Literal(s"col${index + 1}"), e) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index c20dd6148be3e..4a27b2a482728 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -502,7 +502,7 @@ case class SecondsToTimestamp(child: Expression) extends UnaryExpression input.asInstanceOf[Decimal].toJavaBigDecimal.multiply(operand).longValueExact() case _: FloatType => input => val f = input.asInstanceOf[Float] - if (f.isNaN || f.isInfinite) null else (f * MICROS_PER_SECOND).toLong + if (f.isNaN || f.isInfinite) null else (f.toDouble * MICROS_PER_SECOND).toLong case _: DoubleType => input => val d = input.asInstanceOf[Double] if (d.isNaN || d.isInfinite) null else (d * MICROS_PER_SECOND).toLong @@ -517,13 +517,14 @@ case class SecondsToTimestamp(child: Expression) extends UnaryExpression val operand = s"new java.math.BigDecimal($MICROS_PER_SECOND)" defineCodeGen(ctx, ev, c => s"$c.toJavaBigDecimal().multiply($operand).longValueExact()") case other => + val castToDouble = if (other.isInstanceOf[FloatType]) "(double)" else "" nullSafeCodeGen(ctx, ev, c => { val typeStr = CodeGenerator.boxedType(other) s""" |if ($typeStr.isNaN($c) || $typeStr.isInfinite($c)) { | ${ev.isNull} = true; |} else { - | ${ev.value} = (long)($c * $MICROS_PER_SECOND); + | ${ev.value} = (long)($castToDouble$c * $MICROS_PER_SECOND); |} |""".stripMargin }) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala index 7e4560ab8161b..2de5bbbe9893e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, EmptyBlock, ExprCode} +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -103,10 +103,8 @@ object MakeDecimal { case class PromotePrecision(child: Expression) extends UnaryExpression { override def dataType: DataType = child.dataType override def eval(input: InternalRow): Any = child.eval(input) - /** Just a simple pass-through for code generation. */ - override def genCode(ctx: CodegenContext): ExprCode = child.genCode(ctx) override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = - ev.copy(EmptyBlock) + child.genCode(ctx) override def prettyName: String = "promote_precision" override def sql: String = child.sql override lazy val canonicalized: Expression = child.canonicalized diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala index 4454afb6c099b..a4e069d652b43 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import java.util.Comparator -import java.util.concurrent.atomic.AtomicReference +import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} import scala.collection.mutable @@ -52,6 +52,16 @@ case class UnresolvedNamedLambdaVariable(nameParts: Seq[String]) override def sql: String = name } +object UnresolvedNamedLambdaVariable { + + // Counter to ensure lambda variable names are unique + private val nextVarNameId = new AtomicInteger(0) + + def freshVarName(name: String): String = { + s"${name}_${nextVarNameId.getAndIncrement()}" + } +} + /** * A named lambda variable. */ @@ -277,7 +287,8 @@ case class ArrayTransform( if (indexVar.isDefined) { indexVar.get.value.set(i) } - result.update(i, f.eval(inputRow)) + val v = InternalRow.copyValue(f.eval(inputRow)) + result.update(i, v) i += 1 } result @@ -796,7 +807,7 @@ case class TransformKeys( while (i < map.numElements) { keyVar.value.set(map.keyArray().get(i, keyVar.dataType)) valueVar.value.set(map.valueArray().get(i, valueVar.dataType)) - val result = functionForEval.eval(inputRow) + val result = InternalRow.copyValue(functionForEval.eval(inputRow)) resultKeys.update(i, result) i += 1 } @@ -843,7 +854,8 @@ case class TransformValues( while (i < map.numElements) { keyVar.value.set(map.keyArray().get(i, keyVar.dataType)) valueVar.value.set(map.valueArray().get(i, valueVar.dataType)) - resultValues.update(i, functionForEval.eval(inputRow)) + val v = InternalRow.copyValue(functionForEval.eval(inputRow)) + resultValues.update(i, v) i += 1 } new ArrayBasedMapData(map.keyArray(), resultValues) @@ -1026,7 +1038,8 @@ case class MapZipWith(left: Expression, right: Expression, function: Expression) value1Var.value.set(v1) value2Var.value.set(v2) keys.update(i, key) - values.update(i, functionForEval.eval(inputRow)) + val v = InternalRow.copyValue(functionForEval.eval(inputRow)) + values.update(i, v) i += 1 } new ArrayBasedMapData(keys, values) @@ -1098,7 +1111,8 @@ case class ZipWith(left: Expression, right: Expression, function: Expression) } else { rightElemVar.value.set(null) } - result.update(i, f.eval(input)) + val v = InternalRow.copyValue(f.eval(input)) + result.update(i, v) i += 1 } result diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala index 931365fb25a1e..5b079889fc30f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala @@ -1367,14 +1367,27 @@ case class BRound(child: Expression, scale: Expression) } object WidthBucket { - def computeBucketNumber(value: Double, min: Double, max: Double, numBucket: Long): jl.Long = { - if (numBucket <= 0 || numBucket == Long.MaxValue || jl.Double.isNaN(value) || min == max || - jl.Double.isNaN(min) || jl.Double.isInfinite(min) || - jl.Double.isNaN(max) || jl.Double.isInfinite(max)) { - return null + if (isNull(value, min, max, numBucket)) { + null + } else { + computeBucketNumberNotNull(value, min, max, numBucket) } + } + /** This function is called by generated Java code, so it needs to be public. */ + def isNull(value: Double, min: Double, max: Double, numBucket: Long): Boolean = { + numBucket <= 0 || + numBucket == Long.MaxValue || + jl.Double.isNaN(value) || + min == max || + jl.Double.isNaN(min) || jl.Double.isInfinite(min) || + jl.Double.isNaN(max) || jl.Double.isInfinite(max) + } + + /** This function is called by generated Java code, so it needs to be public. */ + def computeBucketNumberNotNull( + value: Double, min: Double, max: Double, numBucket: Long): jl.Long = { val lower = Math.min(min, max) val upper = Math.max(min, max) @@ -1457,8 +1470,13 @@ case class WidthBucket( } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - defineCodeGen(ctx, ev, (input, min, max, numBucket) => - "org.apache.spark.sql.catalyst.expressions.WidthBucket" + - s".computeBucketNumber($input, $min, $max, $numBucket)") + nullSafeCodeGen(ctx, ev, (input, min, max, numBucket) => { + s"""${ev.isNull} = org.apache.spark.sql.catalyst.expressions.WidthBucket + | .isNull($input, $min, $max, $numBucket); + |if (!${ev.isNull}) { + | ${ev.value} = org.apache.spark.sql.catalyst.expressions.WidthBucket + | .computeBucketNumberNotNull($input, $min, $max, $numBucket); + |}""".stripMargin + }) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index badc2ecc9cb28..cabc09052bab6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -180,6 +180,14 @@ case class Alias(child: Expression, name: String)( } } + def withName(newName: String): NamedExpression = { + Alias(child, newName)( + exprId = exprId, + qualifier = qualifier, + explicitMetadata = explicitMetadata, + nonInheritableMetadataKeys = nonInheritableMetadataKeys) + } + def newInstance(): NamedExpression = Alias(child, name)( qualifier = qualifier, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index f391b3128cf41..6fcca6b784490 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -27,7 +27,7 @@ import scala.util.{Properties, Try} import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.serializer._ import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, ScalaReflection} +import org.apache.spark.sql.catalyst.{InternalRow, ScalaReflection} import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ @@ -127,7 +127,12 @@ trait InvokeLike extends Expression with NonSQLExpression { // return null if one of arguments is null null } else { - val ret = method.invoke(obj, args: _*) + val ret = try { + method.invoke(obj, args: _*) + } catch { + // Re-throw the original exception. + case e: java.lang.reflect.InvocationTargetException => throw e.getCause + } val boxedClass = ScalaReflection.typeBoxedJavaMapping.get(dataType) if (boxedClass.isDefined) { boxedClass.get.cast(ret) @@ -136,6 +141,34 @@ trait InvokeLike extends Expression with NonSQLExpression { } } } + + final def findMethod(cls: Class[_], functionName: String, argClasses: Seq[Class[_]]): Method = { + // Looking with function name + argument classes first. + try { + cls.getMethod(functionName, argClasses: _*) + } catch { + case _: NoSuchMethodException => + // For some cases, e.g. arg class is Object, `getMethod` cannot find the method. + // We look at function name + argument length + val m = cls.getMethods.filter { m => + m.getName == functionName && m.getParameterCount == arguments.length + } + if (m.isEmpty) { + sys.error(s"Couldn't find $functionName on $cls") + } else if (m.length > 1) { + // More than one matched method signature. Exclude synthetic one, e.g. generic one. + val realMethods = m.filter(!_.isSynthetic) + if (realMethods.length > 1) { + // Ambiguous case, we don't know which method to choose, just fail it. + sys.error(s"Found ${realMethods.length} $functionName on $cls") + } else { + realMethods.head + } + } else { + m.head + } + } + } } /** @@ -227,7 +260,7 @@ case class StaticInvoke( override def children: Seq[Expression] = arguments lazy val argClasses = ScalaReflection.expressionJavaClasses(arguments) - @transient lazy val method = cls.getDeclaredMethod(functionName, argClasses : _*) + @transient lazy val method = findMethod(cls, functionName, argClasses) override def eval(input: InternalRow): Any = { invoke(null, method, arguments, input, dataType) @@ -314,12 +347,7 @@ case class Invoke( @transient lazy val method = targetObject.dataType match { case ObjectType(cls) => - val m = cls.getMethods.find(_.getName == encodedFunctionName) - if (m.isEmpty) { - sys.error(s"Couldn't find $encodedFunctionName on $cls") - } else { - m - } + Some(findMethod(cls, encodedFunctionName, argClasses)) case _ => None } @@ -332,7 +360,7 @@ case class Invoke( val invokeMethod = if (method.isDefined) { method.get } else { - obj.getClass.getDeclaredMethod(functionName, argClasses: _*) + obj.getClass.getMethod(functionName, argClasses: _*) } invoke(obj, invokeMethod, arguments, input, dataType) } @@ -443,7 +471,7 @@ case class NewInstance( // Note that static inner classes (e.g., inner classes within Scala objects) don't need // outer pointer registration. val needOuterPointer = - outerPointer.isEmpty && cls.isMemberClass && !Modifier.isStatic(cls.getModifiers) + outerPointer.isEmpty && Utils.isMemberClass(cls) && !Modifier.isStatic(cls.getModifiers) childrenResolved && !needOuterPointer } @@ -489,7 +517,7 @@ case class NewInstance( // that might be defined on the companion object. case 0 => s"$className$$.MODULE$$.apply($argString)" case _ => outer.map { gen => - s"${gen.value}.new ${cls.getSimpleName}($argString)" + s"${gen.value}.new ${Utils.getSimpleName(cls)}($argString)" }.getOrElse { s"new $className($argString)" } @@ -1090,11 +1118,6 @@ case class CatalystToExternalMap private( private lazy val inputMapType = inputData.dataType.asInstanceOf[MapType] - private lazy val keyConverter = - CatalystTypeConverters.createToScalaConverter(inputMapType.keyType) - private lazy val valueConverter = - CatalystTypeConverters.createToScalaConverter(inputMapType.valueType) - private lazy val (newMapBuilderMethod, moduleField) = { val clazz = Utils.classForName(collClass.getCanonicalName + "$") (clazz.getMethod("newBuilder"), clazz.getField("MODULE$").get(null)) @@ -1111,10 +1134,13 @@ case class CatalystToExternalMap private( builder.sizeHint(result.numElements()) val keyArray = result.keyArray() val valueArray = result.valueArray() + val row = new GenericInternalRow(1) var i = 0 while (i < result.numElements()) { - val key = keyConverter(keyArray.get(i, inputMapType.keyType)) - val value = valueConverter(valueArray.get(i, inputMapType.valueType)) + row.update(0, keyArray.get(i, inputMapType.keyType)) + val key = keyLambdaFunction.eval(row) + row.update(0, valueArray.get(i, inputMapType.valueType)) + val value = valueLambdaFunction.eval(row) builder += Tuple2(key, value) i += 1 } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 53ac3560bc3b3..f2d91b5d8d4bf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -487,12 +487,24 @@ case class InSet(child: Expression, hset: Set[Any]) extends UnaryExpression with override def toString: String = s"$child INSET ${hset.mkString("(", ",", ")")}" @transient private[this] lazy val hasNull: Boolean = hset.contains(null) + @transient private[this] lazy val isNaN: Any => Boolean = child.dataType match { + case DoubleType => (value: Any) => java.lang.Double.isNaN(value.asInstanceOf[java.lang.Double]) + case FloatType => (value: Any) => java.lang.Float.isNaN(value.asInstanceOf[java.lang.Float]) + case _ => (_: Any) => false + } + @transient private[this] lazy val hasNaN = child.dataType match { + case DoubleType | FloatType => set.exists(isNaN) + case _ => false + } + override def nullable: Boolean = child.nullable || hasNull protected override def nullSafeEval(value: Any): Any = { if (set.contains(value)) { true + } else if (isNaN(value)) { + hasNaN } else if (hasNull) { null } else { @@ -524,15 +536,33 @@ case class InSet(child: Expression, hset: Set[Any]) extends UnaryExpression with private def genCodeWithSet(ctx: CodegenContext, ev: ExprCode): ExprCode = { nullSafeCodeGen(ctx, ev, c => { val setTerm = ctx.addReferenceObj("set", set) + val setIsNull = if (hasNull) { s"${ev.isNull} = !${ev.value};" } else { "" } - s""" - |${ev.value} = $setTerm.contains($c); - |$setIsNull - """.stripMargin + + val ret = child.dataType match { + case DoubleType => Some((v: Any) => s"java.lang.Double.isNaN($v)") + case FloatType => Some((v: Any) => s"java.lang.Float.isNaN($v)") + case _ => None + } + + ret.map { isNaN => + s""" + |if ($setTerm.contains($c)) { + | ${ev.value} = true; + |} else if (${isNaN(c)}) { + | ${ev.value} = $hasNaN; + |} + |$setIsNull + |""".stripMargin + }.getOrElse( + s""" + |${ev.value} = $setTerm.contains($c); + |$setIsNull + """.stripMargin) }) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala index c41686da79487..828f768f17701 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala @@ -348,12 +348,30 @@ object JoinReorderDP extends PredicateHelper with Logging { } } + /** + * To identify the plan with smaller computational cost, + * we use the weighted geometric mean of ratio of rows and the ratio of sizes in bytes. + * + * There are other ways to combine these values as a cost comparison function. + * Some of these, that we have experimented with, but have gotten worse result, + * than with the current one: + * 1) Weighted arithmetic mean of these two ratios - adding up fractions puts + * less emphasis on ratios between 0 and 1. Ratios 10 and 0.1 should be considered + * to be just as strong evidences in opposite directions. The arithmetic mean of these + * would be heavily biased towards the 10. + * 2) Absolute cost (cost = weight * rowCount + (1 - weight) * size) - when adding up + * two numeric measurements that have different units we can easily end up with one + * overwhelming the other. + */ def betterThan(other: JoinPlan, conf: SQLConf): Boolean = { - val thisCost = BigDecimal(this.planCost.card) * conf.joinReorderCardWeight + - BigDecimal(this.planCost.size) * (1 - conf.joinReorderCardWeight) - val otherCost = BigDecimal(other.planCost.card) * conf.joinReorderCardWeight + - BigDecimal(other.planCost.size) * (1 - conf.joinReorderCardWeight) - thisCost < otherCost + if (other.planCost.card == 0 || other.planCost.size == 0) { + false + } else { + val relativeRows = BigDecimal(this.planCost.card) / BigDecimal(other.planCost.card) + val relativeSize = BigDecimal(this.planCost.size) / BigDecimal(other.planCost.size) + Math.pow(relativeRows.doubleValue, conf.joinReorderCardWeight) * + Math.pow(relativeSize.doubleValue, 1 - conf.joinReorderCardWeight) < 1 + } } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala index b053bf6d61e6b..028e112213451 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala @@ -68,7 +68,7 @@ object NestedColumnAliasing { */ private def replaceToAliases( plan: LogicalPlan, - nestedFieldToAlias: Map[ExtractValue, Alias], + nestedFieldToAlias: Map[Expression, Alias], attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = plan match { case Project(projectList, child) => Project( @@ -85,10 +85,10 @@ object NestedColumnAliasing { */ def getNewProjectList( projectList: Seq[NamedExpression], - nestedFieldToAlias: Map[ExtractValue, Alias]): Seq[NamedExpression] = { + nestedFieldToAlias: Map[Expression, Alias]): Seq[NamedExpression] = { projectList.map(_.transform { - case f: ExtractValue if nestedFieldToAlias.contains(f) => - nestedFieldToAlias(f).toAttribute + case f: ExtractValue if nestedFieldToAlias.contains(f.canonicalized) => + nestedFieldToAlias(f.canonicalized).toAttribute }.asInstanceOf[NamedExpression]) } @@ -98,13 +98,13 @@ object NestedColumnAliasing { */ def replaceWithAliases( plan: LogicalPlan, - nestedFieldToAlias: Map[ExtractValue, Alias], + nestedFieldToAlias: Map[Expression, Alias], attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = { plan.withNewChildren(plan.children.map { plan => Project(plan.output.flatMap(a => attrToAliases.getOrElse(a.exprId, Seq(a))), plan) }).transformExpressions { - case f: ExtractValue if nestedFieldToAlias.contains(f) => - nestedFieldToAlias(f).toAttribute + case f: ExtractValue if nestedFieldToAlias.contains(f.canonicalized) => + nestedFieldToAlias(f.canonicalized).toAttribute } } @@ -158,7 +158,7 @@ object NestedColumnAliasing { * 2. ExprId -> Seq[Alias]: A reference attribute has multiple aliases pointing it. */ def getAliasSubMap(exprList: Seq[Expression], exclusiveAttrs: Seq[Attribute] = Seq.empty) - : Option[(Map[ExtractValue, Alias], Map[ExprId, Seq[Alias]])] = { + : Option[(Map[Expression, Alias], Map[ExprId, Seq[Alias]])] = { val (nestedFieldReferences, otherRootReferences) = exprList.flatMap(collectRootReferenceAndExtractValue).partition { case _: ExtractValue => true @@ -208,7 +208,9 @@ object NestedColumnAliasing { if (aliasSub.isEmpty) { None } else { - Some((aliasSub.values.flatten.toMap, aliasSub.map(x => (x._1, x._2.map(_._2))))) + Some((aliasSub.values.flatten.map { + case (field, alias) => field.canonicalized -> alias + }.toMap, aliasSub.map(x => (x._1, x._2.map(_._2))))) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index d8e6c04f7e270..ef9f19914cec7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.SchemaUtils._ import org.apache.spark.util.Utils /** @@ -44,10 +45,14 @@ abstract class Optimizer(catalogManager: CatalogManager) // - is still resolved // - only host special expressions in supported operators // - has globally-unique attribute IDs - override protected def isPlanIntegral(plan: LogicalPlan): Boolean = { - !Utils.isTesting || (plan.resolved && - plan.find(PlanHelper.specialExpressionsInUnsupportedOperator(_).nonEmpty).isEmpty && - LogicalPlanIntegrity.checkIfExprIdsAreGloballyUnique(plan)) + // - optimized plan have same schema with previous plan. + override protected def isPlanIntegral( + previousPlan: LogicalPlan, + currentPlan: LogicalPlan): Boolean = { + !Utils.isTesting || (currentPlan.resolved && + currentPlan.find(PlanHelper.specialExpressionsInUnsupportedOperator(_).nonEmpty).isEmpty && + LogicalPlanIntegrity.checkIfExprIdsAreGloballyUnique(currentPlan) && + DataType.equalsIgnoreNullability(previousPlan.schema, currentPlan.schema)) } override protected val excludedOnceBatches: Set[String] = @@ -446,7 +451,7 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] { createAttributeMapping(left, newLeft) ++ createAttributeMapping(right, newRight)) val newCondition = condition.map(_.transform { - case a: Attribute => mapping.getOrElse(a, a) + case a: Attribute => mapping.get(a).map(_.withName(a.name)).getOrElse(a) }) Join(newLeft, newRight, joinType, newCondition, hint) @@ -476,7 +481,7 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] { // Transform the expressions. newNode.mapExpressions { expr => clean(expr.transform { - case a: Attribute => mapping.getOrElse(a, a) + case a: Attribute => mapping.get(a).map(_.withName(a.name)).getOrElse(a) }) } } @@ -491,7 +496,21 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] { object RemoveNoopOperators extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { // Eliminate no-op Projects - case p @ Project(_, child) if child.sameOutput(p) => child + case p @ Project(projectList, child) if child.sameOutput(p) => + val newChild = child match { + case p: Project => + p.copy(projectList = restoreOriginalOutputNames(p.projectList, projectList.map(_.name))) + case agg: Aggregate => + agg.copy(aggregateExpressions = + restoreOriginalOutputNames(agg.aggregateExpressions, projectList.map(_.name))) + case _ => + child + } + if (newChild.output.zip(projectList).forall { case (a1, a2) => a1.name == a2.name }) { + newChild + } else { + p + } // Eliminate no-op Window case w: Window if w.windowExpressions.isEmpty => w.child @@ -875,24 +894,30 @@ object TransposeWindow extends Rule[LogicalPlan] { * by this [[Generate]] can be removed earlier - before joins and in data sources. */ object InferFiltersFromGenerate extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - // This rule does not infer filters from foldable expressions to avoid constant filters - // like 'size([1, 2, 3]) > 0'. These do not show up in child's constraints and - // then the idempotence will break. - case generate @ Generate(e, _, _, _, _, _) - if !e.deterministic || e.children.forall(_.foldable) => generate - + def apply(plan: LogicalPlan): LogicalPlan = plan.transformUp { case generate @ Generate(g, _, false, _, _, _) if canInferFilters(g) => - // Exclude child's constraints to guarantee idempotency - val inferredFilters = ExpressionSet( - Seq( - GreaterThan(Size(g.children.head), Literal(0)), - IsNotNull(g.children.head) - ) - ) -- generate.child.constraints - - if (inferredFilters.nonEmpty) { - generate.copy(child = Filter(inferredFilters.reduce(And), generate.child)) + assert(g.children.length == 1) + val input = g.children.head + // Generating extra predicates here has overheads/risks: + // - We may evaluate expensive input expressions multiple times. + // - We may infer too many constraints later. + // - The input expression may fail to be evaluated under ANSI mode. If we reorder the + // predicates and evaluate the input expression first, we may fail the query unexpectedly. + // To be safe, here we only generate extra predicates if the input is an attribute. + // Note that, foldable input is also excluded here, to avoid constant filters like + // 'size([1, 2, 3]) > 0'. These do not show up in child's constraints and then the + // idempotence will break. + if (input.isInstanceOf[Attribute]) { + // Exclude child's constraints to guarantee idempotency + val inferredFilters = ExpressionSet( + Seq(GreaterThan(Size(input), Literal(0)), IsNotNull(input)) + ) -- generate.child.constraints + + if (inferredFilters.nonEmpty) { + generate.copy(child = Filter(inferredFilters.reduce(And), generate.child)) + } else { + generate + } } else { generate } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/UpdateFields.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/UpdateFields.scala index 465d2efe2775c..d3127b5560178 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/UpdateFields.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/UpdateFields.scala @@ -50,28 +50,22 @@ object OptimizeUpdateFields extends Rule[LogicalPlan] { val values = withFields.map(_.valExpr) val newNames = mutable.ArrayBuffer.empty[String] - val newValues = mutable.ArrayBuffer.empty[Expression] + val newValues = mutable.HashMap.empty[String, Expression] + // Used to remember the casing of the last instance + val nameMap = mutable.HashMap.empty[String, String] - if (caseSensitive) { - names.zip(values).reverse.foreach { case (name, value) => - if (!newNames.contains(name)) { - newNames += name - newValues += value - } - } - } else { - val nameSet = mutable.HashSet.empty[String] - names.zip(values).reverse.foreach { case (name, value) => - val lowercaseName = name.toLowerCase(Locale.ROOT) - if (!nameSet.contains(lowercaseName)) { - newNames += name - newValues += value - nameSet += lowercaseName - } + names.zip(values).foreach { case (name, value) => + val normalizedName = if (caseSensitive) name else name.toLowerCase(Locale.ROOT) + if (nameMap.contains(normalizedName)) { + newValues += normalizedName -> value + } else { + newNames += normalizedName + newValues += normalizedName -> value } + nameMap += normalizedName -> name } - val newWithFields = newNames.reverse.zip(newValues.reverse).map(p => WithField(p._1, p._2)) + val newWithFields = newNames.map(n => WithField(nameMap(n), newValues(n))) UpdateFields(structExpr, newWithFields.toSeq) case UpdateFields(UpdateFields(struct, fieldOps1), fieldOps2) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index 78d7cccc09dfb..78098c41ec736 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -46,6 +46,7 @@ object ConstantFolding extends Rule[LogicalPlan] { private def hasNoSideEffect(e: Expression): Boolean = e match { case _: Attribute => true case _: Literal => true + case c: Cast if !conf.ansiEnabled => hasNoSideEffect(c.child) case _: NoThrow if e.deterministic => e.children.forall(hasNoSideEffect) case _ => false } @@ -576,7 +577,9 @@ object LikeSimplification extends Rule[LogicalPlan] { private def simplifyMultiLike( child: Expression, patterns: Seq[UTF8String], multi: MultiLikeBase): Expression = { val (remainPatternMap, replacementMap) = - patterns.map { p => p -> simplifyLike(child, p.toString)}.partition(_._2.isEmpty) + patterns.map { p => + p -> Option(p).flatMap(p => simplifyLike(child, p.toString)) + }.partition(_._2.isEmpty) val remainPatterns = remainPatternMap.map(_._1) val replacements = replacementMap.map(_._2.get) if (replacements.isEmpty) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index c3d593c47f74e..63906e11cebc7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -491,7 +491,11 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg // Before calling `toMap`, we check duplicated keys to avoid silently ignore partition values // in partition spec like PARTITION(a='1', b='2', a='3'). The real semantical check for // partition columns will be done in analyzer. - checkDuplicateKeys(parts.toSeq, ctx) + if (conf.caseSensitiveAnalysis) { + checkDuplicateKeys(parts.toSeq, ctx) + } else { + checkDuplicateKeys(parts.map(kv => kv._1.toLowerCase(Locale.ROOT) -> kv._2).toSeq, ctx) + } parts.toMap } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index 2880e87ab1566..fac4d2a980ca2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -144,13 +144,15 @@ object ScanOperation extends OperationHelper with PredicateHelper { case Filter(condition, child) => collectProjectsAndFilters(child) match { case Some((fields, filters, other, aliases)) => - // Follow CombineFilters and only keep going if 1) the collected Filters - // and this filter are all deterministic or 2) if this filter is the first - // collected filter and doesn't have common non-deterministic expressions - // with lower Project. + // When collecting projects and filters, we effectively push down filters through + // projects. We need to meet the following conditions to do so: + // 1) no Project collected so far or the collected Projects are all deterministic + // 2) the collected filters and this filter are all deterministic, or this is the + // first collected filter. + val canCombineFilters = fields.forall(_.forall(_.deterministic)) && { + filters.isEmpty || (filters.forall(_.deterministic) && condition.deterministic) + } val substitutedCondition = substitute(aliases)(condition) - val canCombineFilters = (filters.nonEmpty && filters.forall(_.deterministic) && - substitutedCondition.deterministic) || filters.isEmpty if (canCombineFilters && !hasCommonNonDeterministic(Seq(condition), aliases)) { Some((fields, filters ++ splitConjunctivePredicates(substitutedCondition), other, aliases)) @@ -333,7 +335,7 @@ object PhysicalAggregation { case ue: PythonUDF if PythonUDF.isGroupedAggPandasUDF(ue) => equivalentAggregateExpressions.getEquivalentExprs(ue).headOption .getOrElse(ue).asInstanceOf[PythonUDF].resultAttribute - case expression => + case expression if !expression.foldable => // Since we're using `namedGroupingAttributes` to extract the grouping key // columns, we need to replace grouping key expressions with their corresponding // attributes. We do not rely on the equality check at here since attributes may diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AnalysisHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AnalysisHelper.scala index 54b01416381c6..b31b3e6d17054 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AnalysisHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AnalysisHelper.scala @@ -91,7 +91,9 @@ trait AnalysisHelper extends QueryPlan[LogicalPlan] { self: LogicalPlan => } } else { CurrentOrigin.withOrigin(origin) { - rule.applyOrElse(afterRuleOnChildren, identity[LogicalPlan]) + val afterRule = rule.applyOrElse(afterRuleOnChildren, identity[LogicalPlan]) + afterRule.copyTagsFrom(self) + afterRule } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index ad5c3fd74e9b5..3ea79b35e8c52 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -33,7 +33,10 @@ abstract class LogicalPlan with QueryPlanConstraints with Logging { - /** Metadata fields that can be projected from this node */ + /** + * Metadata fields that can be projected from this node. + * Should be overridden if the plan does not propagate its children's output. + */ def metadataOutput: Seq[Attribute] = children.flatMap(_.metadataOutput) /** Returns true if this subtree has data from a streaming data source. */ @@ -89,11 +92,14 @@ abstract class LogicalPlan } } - private[this] lazy val childAttributes = - AttributeSeq(children.flatMap(c => c.output ++ c.metadataOutput)) + private[this] lazy val childAttributes = AttributeSeq(children.flatMap(_.output)) + + private[this] lazy val childMetadataAttributes = AttributeSeq(children.flatMap(_.metadataOutput)) private[this] lazy val outputAttributes = AttributeSeq(output) + private[this] lazy val outputMetadataAttributes = AttributeSeq(metadataOutput) + /** * Optionally resolves the given strings to a [[NamedExpression]] using the input from all child * nodes of this LogicalPlan. The attribute is expressed as @@ -103,6 +109,7 @@ abstract class LogicalPlan nameParts: Seq[String], resolver: Resolver): Option[NamedExpression] = childAttributes.resolve(nameParts, resolver) + .orElse(childMetadataAttributes.resolve(nameParts, resolver)) /** * Optionally resolves the given strings to a [[NamedExpression]] based on the output of this @@ -113,6 +120,7 @@ abstract class LogicalPlan nameParts: Seq[String], resolver: Resolver): Option[NamedExpression] = outputAttributes.resolve(nameParts, resolver) + .orElse(outputMetadataAttributes.resolve(nameParts, resolver)) /** * Given an attribute name, split it to name parts by dot, but @@ -122,7 +130,7 @@ abstract class LogicalPlan def resolveQuoted( name: String, resolver: Resolver): Option[NamedExpression] = { - outputAttributes.resolve(UnresolvedAttribute.parseAttributeName(name), resolver) + resolve(UnresolvedAttribute.parseAttributeName(name), resolver) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala index 1346f80247a1f..e80eae64d1451 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala @@ -24,6 +24,7 @@ import net.jpountz.lz4.{LZ4BlockInputStream, LZ4BlockOutputStream} import org.apache.spark.sql.catalyst.catalog.CatalogColumnStat import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -116,6 +117,18 @@ case class ColumnStat( maxLen = maxLen, histogram = histogram, version = version) + + def updateCountStats( + oldNumRows: BigInt, + newNumRows: BigInt, + updatedColumnStatOpt: Option[ColumnStat] = None): ColumnStat = { + val updatedColumnStat = updatedColumnStatOpt.getOrElse(this) + val newDistinctCount = EstimationUtils.updateStat(oldNumRows, newNumRows, + distinctCount, updatedColumnStat.distinctCount) + val newNullCount = EstimationUtils.updateStat(oldNumRows, newNumRows, + nullCount, updatedColumnStat.nullCount) + updatedColumnStat.copy(distinctCount = newDistinctCount, nullCount = newNullCount) + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index eee99f9130f8c..4aefb66f72ee2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -59,6 +59,7 @@ object Subquery { case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends OrderPreservingUnaryNode { override def output: Seq[Attribute] = projectList.map(_.toAttribute) + override def metadataOutput: Seq[Attribute] = Nil override def maxRows: Option[Long] = child.maxRows override lazy val resolved: Boolean = { @@ -185,6 +186,8 @@ case class Intersect( leftAttr.withNullability(leftAttr.nullable && rightAttr.nullable) } + override def metadataOutput: Seq[Attribute] = Nil + override protected lazy val validConstraints: ExpressionSet = leftConstraints.union(rightConstraints) @@ -205,6 +208,8 @@ case class Except( /** We don't use right.output because those rows get excluded from the set. */ override def output: Seq[Attribute] = left.output + override def metadataOutput: Seq[Attribute] = Nil + override protected lazy val validConstraints: ExpressionSet = leftConstraints } @@ -268,6 +273,8 @@ case class Union( } } + override def metadataOutput: Seq[Attribute] = Nil + override lazy val resolved: Boolean = { // allChildrenCompatible needs to be evaluated after childrenResolved def allChildrenCompatible: Boolean = @@ -343,6 +350,17 @@ case class Join( } } + override def metadataOutput: Seq[Attribute] = { + joinType match { + case ExistenceJoin(_) => + left.metadataOutput + case LeftExistence(_) => + left.metadataOutput + case _ => + children.flatMap(_.metadataOutput) + } + } + override protected lazy val validConstraints: ExpressionSet = { joinType match { case _: InnerLike if condition.isDefined => @@ -419,6 +437,7 @@ case class InsertIntoDir( extends UnaryNode { override def output: Seq[Attribute] = Seq.empty + override def metadataOutput: Seq[Attribute] = Nil override lazy val resolved: Boolean = false } @@ -449,6 +468,8 @@ case class View( override def newInstance(): LogicalPlan = copy(output = output.map(_.newInstance())) + override def metadataOutput: Seq[Attribute] = Nil + override def simpleString(maxFields: Int): String = { s"View (${desc.identifier}, ${output.mkString("[", ",", "]")})" } @@ -479,7 +500,18 @@ object View { if (activeConf.useCurrentSQLConfigsForView && !isTempView) return activeConf val sqlConf = new SQLConf() - for ((k, v) <- configs) { + // We retain below configs from current session because they are not captured by view + // as optimization configs but they are still needed during the view resolution. + // TODO: remove this `retainedConfigs` after the `RelationConversions` is moved to + // optimization phase. + val retainedConfigs = activeConf.getAllConfs.filterKeys(key => + Seq( + "spark.sql.hive.convertMetastoreParquet", + "spark.sql.hive.convertMetastoreOrc", + "spark.sql.hive.convertInsertingPartitionedTable", + "spark.sql.hive.convertMetastoreCtas" + ).contains(key)) + for ((k, v) <- configs ++ retainedConfigs) { sqlConf.settings.put(k, v) } sqlConf @@ -616,6 +648,7 @@ case class Aggregate( } override def output: Seq[Attribute] = aggregateExpressions.map(_.toAttribute) + override def metadataOutput: Seq[Attribute] = Nil override def maxRows: Option[Long] = { if (groupingExpressions.isEmpty) { Some(1L) @@ -639,6 +672,8 @@ case class Window( override def output: Seq[Attribute] = child.output ++ windowExpressions.map(_.toAttribute) + override def producedAttributes: AttributeSet = windowOutputSet + def windowOutputSet: AttributeSet = AttributeSet(windowExpressions.map(_.toAttribute)) } @@ -749,6 +784,8 @@ case class Expand( override lazy val references: AttributeSet = AttributeSet(projections.flatten.flatMap(_.references)) + override def metadataOutput: Seq[Attribute] = Nil + override def producedAttributes: AttributeSet = AttributeSet(output diff child.output) // This operator can reuse attributes (for example making them null when doing a roll up) so @@ -843,6 +880,7 @@ case class Pivot( } groupByExprsOpt.getOrElse(Seq.empty).map(_.toAttribute) ++ pivotAgg } + override def metadataOutput: Seq[Attribute] = Nil } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/hints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/hints.scala index 4b5e278fccdfb..1b72d21f532af 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/hints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/hints.scala @@ -29,7 +29,10 @@ import org.apache.spark.sql.catalyst.expressions.Attribute case class UnresolvedHint(name: String, parameters: Seq[Any], child: LogicalPlan) extends UnaryNode { - override lazy val resolved: Boolean = false + // we need it to be resolved so that the analyzer can continue to analyze the rest of the query + // plan. + override lazy val resolved: Boolean = child.resolved + override def output: Seq[Attribute] = child.output } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala index 967ccedeeeacb..dafb979767ad1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala @@ -52,14 +52,20 @@ object EstimationUtils { } /** - * Updates (scales down) the number of distinct values if the number of rows decreases after - * some operation (such as filter, join). Otherwise keep it unchanged. + * Updates (scales down) a statistic (eg. number of distinct values) if the number of rows + * decreases after some operation (such as filter, join). Otherwise keep it unchanged. */ - def updateNdv(oldNumRows: BigInt, newNumRows: BigInt, oldNdv: BigInt): BigInt = { - if (newNumRows < oldNumRows) { - ceil(BigDecimal(oldNdv) * BigDecimal(newNumRows) / BigDecimal(oldNumRows)) + def updateStat( + oldNumRows: BigInt, + newNumRows: BigInt, + oldStatOpt: Option[BigInt], + updatedStatOpt: Option[BigInt]): Option[BigInt] = { + if (oldStatOpt.isDefined && updatedStatOpt.isDefined && updatedStatOpt.get > 1 && + newNumRows < oldNumRows) { + // no need to scale down since it is already down to 1 + Some(ceil(BigDecimal(oldStatOpt.get) * BigDecimal(newNumRows) / BigDecimal(oldNumRows))) } else { - oldNdv + updatedStatOpt } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala index 2c5beef43f52a..bc341b9fd563a 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala @@ -106,7 +106,7 @@ case class FilterEstimation(plan: Filter) extends Logging { // The foldable Not has been processed in the ConstantFolding rule // This is a top-down traversal. The Not could be pushed down by the above two cases. case Not(l @ Literal(null, _)) => - calculateSingleCondition(l, update = false) + calculateSingleCondition(l, update = false).map(boundProbability(_)) case Not(cond) => calculateFilterSelectivity(cond, update = false) match { @@ -115,7 +115,7 @@ case class FilterEstimation(plan: Filter) extends Logging { } case _ => - calculateSingleCondition(condition, update) + calculateSingleCondition(condition, update).map(boundProbability(_)) } } @@ -233,6 +233,8 @@ case class FilterEstimation(plan: Filter) extends Logging { val rowCountValue = childStats.rowCount.get val nullPercent: Double = if (rowCountValue == 0) { 0 + } else if (colStat.nullCount.get > rowCountValue) { + 1 } else { (BigDecimal(colStat.nullCount.get) / BigDecimal(rowCountValue)).toDouble } @@ -854,6 +856,10 @@ case class FilterEstimation(plan: Filter) extends Logging { Some(percent) } + // Bound result in [0, 1] + private def boundProbability(p: Double): Double = { + Math.max(0.0, Math.min(1.0, p)) + } } /** @@ -907,26 +913,14 @@ case class ColumnStatsMap(originalMap: AttributeMap[ColumnStat]) { def update(a: Attribute, stats: ColumnStat): Unit = updatedMap.update(a.exprId, a -> stats) /** - * Collects updated column stats, and scales down ndv for other column stats if the number of rows - * decreases after this Filter operator. + * Collects updated column stats; scales down column count stats if the + * number of rows decreases after this Filter operator. */ def outputColumnStats(rowsBeforeFilter: BigInt, rowsAfterFilter: BigInt) : AttributeMap[ColumnStat] = { val newColumnStats = originalMap.map { case (attr, oriColStat) => - val colStat = updatedMap.get(attr.exprId).map(_._2).getOrElse(oriColStat) - val newNdv = if (colStat.distinctCount.isEmpty) { - // No NDV in the original stats. - None - } else if (colStat.distinctCount.get > 1) { - // Update ndv based on the overall filter selectivity: scale down ndv if the number of rows - // decreases; otherwise keep it unchanged. - Some(EstimationUtils.updateNdv(oldNumRows = rowsBeforeFilter, - newNumRows = rowsAfterFilter, oldNdv = oriColStat.distinctCount.get)) - } else { - // no need to scale down since it is already down to 1 (for skewed distribution case) - colStat.distinctCount - } - attr -> colStat.copy(distinctCount = newNdv) + attr -> oriColStat.updateCountStats( + rowsBeforeFilter, rowsAfterFilter, updatedMap.get(attr.exprId).map(_._2)) } AttributeMap(newColumnStats.toSeq) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala index 777a4c8291223..c9661173b1bcc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala @@ -308,17 +308,12 @@ case class JoinEstimation(join: Join) extends Logging { outputAttrStats += a -> keyStatsAfterJoin(a) } else { val oldColStat = oldAttrStats(a) - val oldNdv = oldColStat.distinctCount - val newNdv = if (oldNdv.isDefined) { - Some(if (join.left.outputSet.contains(a)) { - updateNdv(oldNumRows = leftRows, newNumRows = outputRows, oldNdv = oldNdv.get) - } else { - updateNdv(oldNumRows = rightRows, newNumRows = outputRows, oldNdv = oldNdv.get) - }) + val oldNumRows = if (join.left.outputSet.contains(a)) { + leftRows } else { - None + rightRows } - val newColStat = oldColStat.copy(distinctCount = newNdv) + val newColStat = oldColStat.updateCountStats(oldNumRows, outputRows) // TODO: support nullCount updates for specific outer joins outputAttrStats += a -> newColStat } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala index d5b0884f6ff13..45910f2e839b3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala @@ -156,7 +156,7 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { * `Optimizer`, so we can catch rules that return invalid plans. The check function returns * `false` if the given plan doesn't pass the structural integrity check. */ - protected def isPlanIntegral(plan: TreeType): Boolean = true + protected def isPlanIntegral(previousPlan: TreeType, currentPlan: TreeType): Boolean = true /** * Util method for checking whether a plan remains the same if re-optimized. @@ -196,7 +196,7 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { val beforeMetrics = RuleExecutor.getCurrentMetrics() // Run the structural integrity checker against the initial input - if (!isPlanIntegral(plan)) { + if (!isPlanIntegral(plan, plan)) { val message = "The structural integrity of the input plan is broken in " + s"${this.getClass.getName.stripSuffix("$")}." throw new TreeNodeException(plan, message, null) @@ -229,7 +229,7 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { tracker.foreach(_.recordRuleInvocation(rule.ruleName, runTime, effective)) // Run the structural integrity checker against the plan after each rule. - if (effective && !isPlanIntegral(result)) { + if (effective && !isPlanIntegral(plan, result)) { val message = s"After applying rule ${rule.ruleName} in batch ${batch.name}, " + "the structural integrity of the plan is broken." throw new TreeNodeException(result, message, null) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 75a1f712671ee..7ebaba8031c54 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -787,9 +787,10 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { ("deserialized" -> s.deserialized) ~ ("replication" -> s.replication) case n: TreeNode[_] => n.jsonValue case o: Option[_] => o.map(parseToJson) - // Recursive scan Seq[TreeNode], Seq[Partitioning], Seq[DataType] - case t: Seq[_] if t.forall(_.isInstanceOf[TreeNode[_]]) || - t.forall(_.isInstanceOf[Partitioning]) || t.forall(_.isInstanceOf[DataType]) => + // Recursive scan Seq[Partitioning], Seq[DataType], Seq[Product] + case t: Seq[_] if t.forall(_.isInstanceOf[Partitioning]) || + t.forall(_.isInstanceOf[DataType]) || + t.forall(_.isInstanceOf[Product]) => JArray(t.map(parseToJson).toList) case t: Seq[_] if t.length > 0 && t.head.isInstanceOf[String] => JString(truncatedString(t, "[", ", ", "]", SQLConf.get.maxToStringFields)) @@ -802,7 +803,20 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { case p: Product if shouldConvertToJson(p) => try { val fieldNames = getConstructorParameterNames(p.getClass) - val fieldValues = p.productIterator.toSeq + val fieldValues = { + if (p.productArity == fieldNames.length) { + p.productIterator.toSeq + } else { + val clazz = p.getClass + // Fallback to use reflection if length of product elements do not match + // constructor params. + fieldNames.map { fieldName => + val field = clazz.getDeclaredField(fieldName) + field.setAccessible(true) + field.get(p) + } + } + } assert(fieldNames.length == fieldValues.length, s"$simpleClassName fields: " + fieldNames.mkString(", ") + s", values: " + fieldValues.mkString(", ")) ("product-class" -> JString(p.getClass.getName)) :: fieldNames.zip(fieldValues).map { @@ -810,6 +824,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { }.toList } catch { case _: RuntimeException => null + case _: ReflectiveOperationException => null } case _ => JNull } @@ -827,6 +842,9 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { case broadcast: BroadcastMode => true case table: CatalogTableType => true case storage: CatalogStorageFormat => true + // Write out product that contains TreeNode, since there are some Tuples such as cteRelations + // in With, branches in CaseWhen which are essential to understand the plan. + case p if p.productIterator.exists(_.isInstanceOf[TreeNode[_]]) => true case _ => false } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 8d7cf81d85dac..21b3adbc1c44f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -242,7 +242,7 @@ object DateTimeUtils { * - +|-hhmmss * - Region-based zone IDs in the form `area/city`, such as `Europe/Paris` */ - def stringToTimestamp(s: UTF8String, timeZoneId: ZoneId): Option[Long] = { + def stringToTimestamp(s: UTF8String, timeZoneId: ZoneId): Option[Long] = try { if (s == null) { return None } @@ -350,27 +350,25 @@ object DateTimeUtils { segments(6) /= 10 digitsMilli -= 1 } - try { - val zoneId = tz match { - case None => timeZoneId - case Some("+") => ZoneOffset.ofHoursMinutes(segments(7), segments(8)) - case Some("-") => ZoneOffset.ofHoursMinutes(-segments(7), -segments(8)) - case Some(zoneName: String) => getZoneId(zoneName.trim) - } - val nanoseconds = MICROSECONDS.toNanos(segments(6)) - val localTime = LocalTime.of(segments(3), segments(4), segments(5), nanoseconds.toInt) - val localDate = if (justTime) { - LocalDate.now(zoneId) - } else { - LocalDate.of(segments(0), segments(1), segments(2)) - } - val localDateTime = LocalDateTime.of(localDate, localTime) - val zonedDateTime = ZonedDateTime.of(localDateTime, zoneId) - val instant = Instant.from(zonedDateTime) - Some(instantToMicros(instant)) - } catch { - case NonFatal(_) => None + val zoneId = tz match { + case None => timeZoneId + case Some("+") => ZoneOffset.ofHoursMinutes(segments(7), segments(8)) + case Some("-") => ZoneOffset.ofHoursMinutes(-segments(7), -segments(8)) + case Some(zoneName: String) => getZoneId(zoneName.trim) + } + val nanoseconds = MICROSECONDS.toNanos(segments(6)) + val localTime = LocalTime.of(segments(3), segments(4), segments(5), nanoseconds.toInt) + val localDate = if (justTime) { + LocalDate.now(zoneId) + } else { + LocalDate.of(segments(0), segments(1), segments(2)) } + val localDateTime = LocalDateTime.of(localDate, localTime) + val zonedDateTime = ZonedDateTime.of(localDateTime, zoneId) + val instant = Instant.from(zonedDateTime) + Some(instantToMicros(instant)) + } catch { + case NonFatal(_) => None } def stringToTimestampAnsi(s: UTF8String, timeZoneId: ZoneId): Long = { @@ -381,6 +379,9 @@ object DateTimeUtils { timestamp.get } } + // See issue SPARK-35679 + // min second cause overflow in instant to micro + private val MIN_SECONDS = Math.floorDiv(Long.MinValue, MICROS_PER_SECOND) @tailrec def stringToTime(s: String): java.util.Date = { @@ -408,9 +409,14 @@ object DateTimeUtils { * microseconds where microsecond 0 is 1970-01-01 00:00:00Z. */ def instantToMicros(instant: Instant): Long = { - val us = Math.multiplyExact(instant.getEpochSecond, MICROS_PER_SECOND) - val result = Math.addExact(us, NANOSECONDS.toMicros(instant.getNano)) - result + val secs = instant.getEpochSecond + if (secs == MIN_SECONDS) { + val us = Math.multiplyExact(secs + 1, MICROS_PER_SECOND) + Math.addExact(us, NANOSECONDS.toMicros(instant.getNano) - MICROS_PER_SECOND) + } else { + val us = Math.multiplyExact(secs, MICROS_PER_SECOND) + Math.addExact(us, NANOSECONDS.toMicros(instant.getNano)) + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala index 7dbdd1ef1cdc5..dca75e5083331 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala @@ -21,26 +21,6 @@ import org.apache.spark.unsafe.types.UTF8String object NumberConverter { - /** - * Divide x by m as if x is an unsigned 64-bit integer. Examples: - * unsignedLongDiv(-1, 2) == Long.MAX_VALUE unsignedLongDiv(6, 3) == 2 - * unsignedLongDiv(0, 5) == 0 - * - * @param x is treated as unsigned - * @param m is treated as signed - */ - private def unsignedLongDiv(x: Long, m: Int): Long = { - if (x >= 0) { - x / m - } else { - // Let uval be the value of the unsigned long with the same bits as x - // Two's complement => x = uval - 2*MAX - 2 - // => uval = x + 2*MAX + 2 - // Now, use the fact: (a+b)/c = a/c + b/c + (a%c+b%c)/c - x / m + 2 * (Long.MaxValue / m) + 2 / m + (x % m + 2 * (Long.MaxValue % m) + 2 % m) / m - } - } - /** * Decode v into value[]. * @@ -52,7 +32,7 @@ object NumberConverter { java.util.Arrays.fill(value, 0.asInstanceOf[Byte]) var i = value.length - 1 while (tmpV != 0) { - val q = unsignedLongDiv(tmpV, radix) + val q = java.lang.Long.divideUnsigned(tmpV, radix) value(i) = (tmpV - q * radix).asInstanceOf[Byte] tmpV = q i -= 1 @@ -69,12 +49,12 @@ object NumberConverter { */ private def encode(radix: Int, fromPos: Int, value: Array[Byte]): Long = { var v: Long = 0L - val bound = unsignedLongDiv(-1 - radix, radix) // Possible overflow once + val bound = java.lang.Long.divideUnsigned(-1 - radix, radix) // Possible overflow once var i = fromPos while (i < value.length && value(i) >= 0) { if (v >= bound) { // Check for overflow - if (unsignedLongDiv(-1 - value(i), radix) < v) { + if (java.lang.Long.divideUnsigned(-1 - value(i), radix) < v) { return -1 } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 536c9ccb7e7ba..19215d51d4ef8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -491,8 +491,8 @@ object SQLConf { "reduce IO and improve performance. Note, multiple contiguous blocks exist in single " + s"fetch request only happen when '${ADAPTIVE_EXECUTION_ENABLED.key}' and " + s"'${COALESCE_PARTITIONS_ENABLED.key}' are both true. This feature also depends " + - "on a relocatable serializer, the concatenation support codec in use and the new version " + - "shuffle fetch protocol.") + "on a relocatable serializer, the concatenation support codec in use, the new version " + + "shuffle fetch protocol and io encryption is disabled.") .version("3.0.0") .booleanConf .createWithDefault(true) @@ -1882,8 +1882,10 @@ object SQLConf { val JOIN_REORDER_CARD_WEIGHT = buildConf("spark.sql.cbo.joinReorder.card.weight") .internal() - .doc("The weight of cardinality (number of rows) for plan cost comparison in join reorder: " + - "rows * weight + size * (1 - weight).") + .doc("The weight of the ratio of cardinalities (number of rows) " + + "in the cost comparison function. The ratio of sizes in bytes has weight " + + "1 - this value. The weighted geometric mean of these ratios is used to decide " + + "which of the candidate plans will be chosen by the CBO.") .version("2.2.0") .doubleConf .checkValue(weight => weight >= 0 && weight <= 1, "The weight value must be in [0, 1].") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala index 9e820f0796a96..016d0bc7e9030 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala @@ -276,7 +276,7 @@ object DataType { /** * Compares two types, ignoring nullability of ArrayType, MapType, StructType. */ - private[types] def equalsIgnoreNullability(left: DataType, right: DataType): Boolean = { + private[sql] def equalsIgnoreNullability(left: DataType, right: DataType): Boolean = { (left, right) match { case (ArrayType(leftElementType, _), ArrayType(rightElementType, _)) => equalsIgnoreNullability(leftElementType, rightElementType) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index 960e174f9c368..53eb26b6fad21 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -586,12 +586,8 @@ object Decimal { } } - private def calculatePrecision(bigDecimal: JavaBigDecimal): Int = { - if (bigDecimal.scale < 0) { + private def numDigitsInIntegralPart(bigDecimal: JavaBigDecimal): Int = { bigDecimal.precision - bigDecimal.scale - } else { - bigDecimal.precision - } } private def stringToJavaBigDecimal(str: UTF8String): JavaBigDecimal = { @@ -605,7 +601,8 @@ object Decimal { val bigDecimal = stringToJavaBigDecimal(str) // We fast fail because constructing a very large JavaBigDecimal to Decimal is very slow. // For example: Decimal("6.0790316E+25569151") - if (calculatePrecision(bigDecimal) > DecimalType.MAX_PRECISION) { + if (numDigitsInIntegralPart(bigDecimal) > DecimalType.MAX_PRECISION && + !SQLConf.get.allowNegativeScaleOfDecimalEnabled) { null } else { Decimal(bigDecimal) @@ -621,7 +618,8 @@ object Decimal { val bigDecimal = stringToJavaBigDecimal(str) // We fast fail because constructing a very large JavaBigDecimal to Decimal is very slow. // For example: Decimal("6.0790316E+25569151") - if (calculatePrecision(bigDecimal) > DecimalType.MAX_PRECISION) { + if (numDigitsInIntegralPart(bigDecimal) > DecimalType.MAX_PRECISION && + !SQLConf.get.allowNegativeScaleOfDecimalEnabled) { throw new ArithmeticException(s"out of decimal type range: $str") } else { Decimal(bigDecimal) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/PartitioningUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/PartitioningUtils.scala index 004725487911a..cf30c713d8d0c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/PartitioningUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/PartitioningUtils.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.DEFAULT_PARTITION_NAME import org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils import org.apache.spark.sql.catalyst.util.CharVarcharUtils +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{CharType, StructType, VarcharType} import org.apache.spark.unsafe.types.UTF8String @@ -44,23 +45,24 @@ private[sql] object PartitioningUtils { throw new AnalysisException(s"$key is not a valid partition column in table $tblName.") } - val normalizedVal = normalizedFiled.dataType match { - case CharType(len) if value != null && value != DEFAULT_PARTITION_NAME => - val v = value match { - case Some(str: String) => Some(charTypeWriteSideCheck(str, len)) - case str: String => charTypeWriteSideCheck(str, len) - case other => other - } - v.asInstanceOf[T] - case VarcharType(len) if value != null && value != DEFAULT_PARTITION_NAME => - val v = value match { - case Some(str: String) => Some(varcharTypeWriteSideCheck(str, len)) - case str: String => varcharTypeWriteSideCheck(str, len) - case other => other - } - v.asInstanceOf[T] - case _ => value - } + val normalizedVal = + if (SQLConf.get.charVarcharAsString) value else normalizedFiled.dataType match { + case CharType(len) if value != null && value != DEFAULT_PARTITION_NAME => + val v = value match { + case Some(str: String) => Some(charTypeWriteSideCheck(str, len)) + case str: String => charTypeWriteSideCheck(str, len) + case other => other + } + v.asInstanceOf[T] + case VarcharType(len) if value != null && value != DEFAULT_PARTITION_NAME => + val v = value match { + case Some(str: String) => Some(varcharTypeWriteSideCheck(str, len)) + case str: String => varcharTypeWriteSideCheck(str, len) + case other => other + } + v.asInstanceOf[T] + case _ => value + } normalizedFiled.name -> normalizedVal } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SQLOpenHashSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SQLOpenHashSet.scala new file mode 100644 index 0000000000000..e09cd95db50bd --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SQLOpenHashSet.scala @@ -0,0 +1,114 @@ +/* + * 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.util + +import scala.reflect._ + +import org.apache.spark.annotation.Private +import org.apache.spark.sql.types.{DataType, DoubleType, FloatType} +import org.apache.spark.util.collection.OpenHashSet + +// A wrap of OpenHashSet that can handle null, Double.NaN and Float.NaN w.r.t. the SQL semantic. +@Private +class SQLOpenHashSet[@specialized(Long, Int, Double, Float) T: ClassTag]( + initialCapacity: Int, + loadFactor: Double) { + + def this(initialCapacity: Int) = this(initialCapacity, 0.7) + + def this() = this(64) + + private val hashSet = new OpenHashSet[T](initialCapacity, loadFactor) + + private var containNull = false + private var containNaN = false + + def addNull(): Unit = { + containNull = true + } + + def addNaN(): Unit = { + containNaN = true + } + + def add(k: T): Unit = { + hashSet.add(k) + } + + def contains(k: T): Boolean = { + hashSet.contains(k) + } + + def containsNull(): Boolean = containNull + + def containsNaN(): Boolean = containNaN +} + +object SQLOpenHashSet { + def withNaNCheckFunc( + dataType: DataType, + hashSet: SQLOpenHashSet[Any], + handleNotNaN: Any => Unit, + handleNaN: Any => Unit): Any => Unit = { + val (isNaN, valueNaN) = dataType match { + case DoubleType => + ((value: Any) => java.lang.Double.isNaN(value.asInstanceOf[java.lang.Double]), + java.lang.Double.NaN) + case FloatType => + ((value: Any) => java.lang.Float.isNaN(value.asInstanceOf[java.lang.Float]), + java.lang.Float.NaN) + case _ => ((_: Any) => false, null) + } + (value: Any) => + if (isNaN(value)) { + if (!hashSet.containsNaN) { + hashSet.addNaN + handleNaN(valueNaN) + } + } else { + handleNotNaN(value) + } + } + + def withNaNCheckCode( + dataType: DataType, + valueName: String, + hashSet: String, + handleNotNaN: String, + handleNaN: String => String): String = { + val ret = dataType match { + case DoubleType => + Some((s"java.lang.Double.isNaN((double)$valueName)", "java.lang.Double.NaN")) + case FloatType => + Some((s"java.lang.Float.isNaN((float)$valueName)", "java.lang.Float.NaN")) + case _ => None + } + ret.map { case (isNaN, valueNaN) => + s""" + |if ($isNaN) { + | if (!$hashSet.containsNaN()) { + | $hashSet.addNaN(); + | ${handleNaN(valueNaN)} + | } + |} else { + | $handleNotNaN + |} + """.stripMargin + }.getOrElse(handleNotNaN) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala index 4aae7bd7dc408..ccbe6984bbf83 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala @@ -21,6 +21,7 @@ import java.util.Locale import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, NamedExpression} import org.apache.spark.sql.connector.expressions.{BucketTransform, FieldReference, NamedTransform, Transform} import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, StructType} @@ -201,6 +202,16 @@ private[spark] object SchemaUtils { } } + def restoreOriginalOutputNames( + projectList: Seq[NamedExpression], + originalNames: Seq[String]): Seq[NamedExpression] = { + projectList.zip(originalNames).map { + case (attr: Attribute, name) => attr.withName(name) + case (alias: Alias, name) => alias.withName(name) + case (other, _) => other + } + } + /** * Returns the given column's ordinal within the given `schema`. The length of the returned * position will be as long as how nested the column is. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 47e97743941d7..79dc2ea0f9a51 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -733,4 +733,44 @@ class AnalysisErrorSuite extends AnalysisTest { UnresolvedRelation(TableIdentifier("t", Option("nonexist"))))))) assertAnalysisError(plan, "Table or view not found:" :: Nil) } + + test("SPARK-35080: Unsupported correlated equality predicates in subquery") { + val a = AttributeReference("a", IntegerType)() + val b = AttributeReference("b", IntegerType)() + val c = AttributeReference("c", IntegerType)() + val t1 = LocalRelation(a, b) + val t2 = LocalRelation(c) + val conditions = Seq( + (abs($"a") === $"c", "abs(`a`) = outer(`c`)"), + (abs($"a") <=> $"c", "abs(`a`) <=> outer(`c`)"), + ($"a" + 1 === $"c", "(`a` + 1) = outer(`c`)"), + ($"a" + $"b" === $"c", "(`a` + `b`) = outer(`c`)"), + ($"a" + $"c" === $"b", "(`a` + outer(`c`)) = `b`"), + (And($"a" === $"c", Cast($"a", IntegerType) === $"c"), "CAST(`a` AS INT) = outer(`c`)")) + conditions.foreach { case (cond, msg) => + val plan = Project( + ScalarSubquery( + Aggregate(Nil, count(Literal(1)).as("cnt") :: Nil, + Filter(cond, t1)) + ).as("sub") :: Nil, + t2) + assertAnalysisError(plan, s"Correlated column is not allowed in predicate ($msg)" :: Nil) + } + } + + test("SPARK-35673: fail if the plan still contains UnresolvedHint after analysis") { + val hintName = "some_random_hint_that_does_not_exist" + val plan = UnresolvedHint(hintName, Seq.empty, + Project(Alias(Literal(1), "x")() :: Nil, OneRowRelation()) + ) + assert(plan.resolved) + + val error = intercept[AnalysisException] { + SimpleAnalyzer.checkAnalysis(plan) + } + assert(error.message.contains(s"Hint not found: ${hintName}")) + + // UnresolvedHint be removed by batch `Remove Unresolved Hints` + assertAnalysisSuccess(plan, true) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala index 3e9a8b71a8fb6..ec9480514ba2d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.catalyst.analysis import java.io.File +import scala.collection.JavaConverters._ + import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock @@ -27,8 +29,8 @@ import org.scalatest.matchers.must.Matchers import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogStorageFormat, CatalogTable, CatalogTableType, ExternalCatalog, InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.connector.InMemoryTableCatalog -import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Identifier, Table, V1Table} +import org.apache.spark.sql.connector.{InMemoryTable, InMemoryTableCatalog} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Identifier, Table} import org.apache.spark.sql.types._ class TableLookupCacheSuite extends AnalysisTest with Matchers { @@ -46,7 +48,12 @@ class TableLookupCacheSuite extends AnalysisTest with Matchers { ignoreIfExists = false) val v2Catalog = new InMemoryTableCatalog { override def loadTable(ident: Identifier): Table = { - V1Table(externalCatalog.getTable("default", ident.name)) + val catalogTable = externalCatalog.getTable("default", ident.name) + new InMemoryTable( + catalogTable.identifier.table, + catalogTable.schema, + Array.empty, + Map.empty[String, String].asJava) } override def name: String = CatalogManager.SESSION_CATALOG_NAME } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index 98f9ce6fe9dbb..ad996dbc509be 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -691,6 +691,9 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { catalog.createTempView("tbl3", tempTable, overrideIfExists = false) // tableExists should not check temp view. assert(!catalog.tableExists(TableIdentifier("tbl3"))) + + // If database doesn't exist, return false instead of failing. + assert(!catalog.tableExists(TableIdentifier("tbl1", Some("non-exist")))) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index f2598a925e08e..7cd659fc9ceb3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -114,6 +114,7 @@ case class ReferenceValueClass(wrapped: ReferenceValueClass.Container) extends A object ReferenceValueClass { case class Container(data: Int) } +case class IntAndString(i: Int, s: String) class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTest { OuterScopes.addOuterScope(this) @@ -174,6 +175,10 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes encodeDecodeTest(Map(1 -> "a", 2 -> "b"), "map") encodeDecodeTest(Map(1 -> "a", 2 -> null), "map with null") encodeDecodeTest(Map(1 -> Map("a" -> 1), 2 -> Map("b" -> 2)), "map of map") + encodeDecodeTest(Map(1 -> IntAndString(1, "a")), "map with case class as value") + encodeDecodeTest(Map(IntAndString(1, "a") -> 1), "map with case class as key") + encodeDecodeTest(Map(IntAndString(1, "a") -> IntAndString(2, "b")), + "map with case class as key and value") encodeDecodeTest(Tuple1[Seq[Int]](null), "null seq in tuple") encodeDecodeTest(Tuple1[Map[String, String]](null), "null map in tuple") @@ -205,6 +210,88 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes encodeDecodeTest(Array(Option(InnerClass(1))), "array of optional inner class") + // holder class to trigger Class.getSimpleName issue + object MalformedClassObject extends Serializable { + case class MalformedNameExample(x: Int) + } + + { + OuterScopes.addOuterScope(MalformedClassObject) + encodeDecodeTest( + MalformedClassObject.MalformedNameExample(42), + "nested Scala class should work") + } + + object OuterLevelWithVeryVeryVeryLongClassName1 { + object OuterLevelWithVeryVeryVeryLongClassName2 { + object OuterLevelWithVeryVeryVeryLongClassName3 { + object OuterLevelWithVeryVeryVeryLongClassName4 { + object OuterLevelWithVeryVeryVeryLongClassName5 { + object OuterLevelWithVeryVeryVeryLongClassName6 { + object OuterLevelWithVeryVeryVeryLongClassName7 { + object OuterLevelWithVeryVeryVeryLongClassName8 { + object OuterLevelWithVeryVeryVeryLongClassName9 { + object OuterLevelWithVeryVeryVeryLongClassName10 { + object OuterLevelWithVeryVeryVeryLongClassName11 { + object OuterLevelWithVeryVeryVeryLongClassName12 { + object OuterLevelWithVeryVeryVeryLongClassName13 { + object OuterLevelWithVeryVeryVeryLongClassName14 { + object OuterLevelWithVeryVeryVeryLongClassName15 { + object OuterLevelWithVeryVeryVeryLongClassName16 { + object OuterLevelWithVeryVeryVeryLongClassName17 { + object OuterLevelWithVeryVeryVeryLongClassName18 { + object OuterLevelWithVeryVeryVeryLongClassName19 { + object OuterLevelWithVeryVeryVeryLongClassName20 { + case class MalformedNameExample(x: Int) + }}}}}}}}}}}}}}}}}}}} + + { + OuterScopes.addOuterScope( + OuterLevelWithVeryVeryVeryLongClassName1 + .OuterLevelWithVeryVeryVeryLongClassName2 + .OuterLevelWithVeryVeryVeryLongClassName3 + .OuterLevelWithVeryVeryVeryLongClassName4 + .OuterLevelWithVeryVeryVeryLongClassName5 + .OuterLevelWithVeryVeryVeryLongClassName6 + .OuterLevelWithVeryVeryVeryLongClassName7 + .OuterLevelWithVeryVeryVeryLongClassName8 + .OuterLevelWithVeryVeryVeryLongClassName9 + .OuterLevelWithVeryVeryVeryLongClassName10 + .OuterLevelWithVeryVeryVeryLongClassName11 + .OuterLevelWithVeryVeryVeryLongClassName12 + .OuterLevelWithVeryVeryVeryLongClassName13 + .OuterLevelWithVeryVeryVeryLongClassName14 + .OuterLevelWithVeryVeryVeryLongClassName15 + .OuterLevelWithVeryVeryVeryLongClassName16 + .OuterLevelWithVeryVeryVeryLongClassName17 + .OuterLevelWithVeryVeryVeryLongClassName18 + .OuterLevelWithVeryVeryVeryLongClassName19 + .OuterLevelWithVeryVeryVeryLongClassName20) + encodeDecodeTest( + OuterLevelWithVeryVeryVeryLongClassName1 + .OuterLevelWithVeryVeryVeryLongClassName2 + .OuterLevelWithVeryVeryVeryLongClassName3 + .OuterLevelWithVeryVeryVeryLongClassName4 + .OuterLevelWithVeryVeryVeryLongClassName5 + .OuterLevelWithVeryVeryVeryLongClassName6 + .OuterLevelWithVeryVeryVeryLongClassName7 + .OuterLevelWithVeryVeryVeryLongClassName8 + .OuterLevelWithVeryVeryVeryLongClassName9 + .OuterLevelWithVeryVeryVeryLongClassName10 + .OuterLevelWithVeryVeryVeryLongClassName11 + .OuterLevelWithVeryVeryVeryLongClassName12 + .OuterLevelWithVeryVeryVeryLongClassName13 + .OuterLevelWithVeryVeryVeryLongClassName14 + .OuterLevelWithVeryVeryVeryLongClassName15 + .OuterLevelWithVeryVeryVeryLongClassName16 + .OuterLevelWithVeryVeryVeryLongClassName17 + .OuterLevelWithVeryVeryVeryLongClassName18 + .OuterLevelWithVeryVeryVeryLongClassName19 + .OuterLevelWithVeryVeryVeryLongClassName20 + .MalformedNameExample(42), + "deeply nested Scala class should work") + } + productTest(PrimitiveData(1, 1, 1, 1, 1, 1, true)) productTest( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index e46599dc19a8b..20c7928b94209 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -1510,6 +1510,10 @@ class CastSuite extends CastSuiteBase { test("Cast from double II") { checkEvaluation(cast(cast(1.toDouble, TimestampType), DoubleType), 1.toDouble) } + + test("SPARK-34727: cast from float II") { + checkCast(16777215.0f, java.time.Instant.ofEpochSecond(16777215)) + } } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index 095894b9fffac..67665343d8f81 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -1888,6 +1888,24 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper Seq(Date.valueOf("2018-01-01"))) } + test("SPARK-36639: Start and end equal in month range with a negative step") { + checkEvaluation(new Sequence( + Literal(Date.valueOf("2018-01-01")), + Literal(Date.valueOf("2018-01-01")), + Literal(stringToInterval("interval -1 day"))), + Seq(Date.valueOf("2018-01-01"))) + checkEvaluation(new Sequence( + Literal(Date.valueOf("2018-01-01")), + Literal(Date.valueOf("2018-01-01")), + Literal(stringToInterval("interval -1 month"))), + Seq(Date.valueOf("2018-01-01"))) + checkEvaluation(new Sequence( + Literal(Date.valueOf("2018-01-01")), + Literal(Date.valueOf("2018-01-01")), + Literal(stringToInterval("interval -1 year"))), + Seq(Date.valueOf("2018-01-01"))) + } + test("SPARK-33386: element_at ArrayIndexOutOfBoundsException") { Seq(true, false).foreach { ansiEnabled => withSQLConf(SQLConf.ANSI_ENABLED.key -> ansiEnabled.toString) { @@ -1930,4 +1948,77 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper } } } + + test("SPARK-36702: ArrayUnion should handle duplicated Double.NaN and Float.Nan") { + checkEvaluation(ArrayUnion( + Literal.apply(Array(Double.NaN, Double.NaN)), Literal.apply(Array(1d))), + Seq(Double.NaN, 1d)) + checkEvaluation(ArrayUnion( + Literal.create(Seq(Double.NaN, null), ArrayType(DoubleType)), + Literal.create(Seq(Double.NaN, null, 1d), ArrayType(DoubleType))), + Seq(Double.NaN, null, 1d)) + checkEvaluation(ArrayUnion( + Literal.apply(Array(Float.NaN, Float.NaN)), Literal.apply(Array(1f))), + Seq(Float.NaN, 1f)) + checkEvaluation(ArrayUnion( + Literal.create(Seq(Float.NaN, null), ArrayType(FloatType)), + Literal.create(Seq(Float.NaN, null, 1f), ArrayType(FloatType))), + Seq(Float.NaN, null, 1f)) + } + + test("SPARK-36753: ArrayExcept should handle duplicated Double.NaN and Float.Nan") { + checkEvaluation(ArrayExcept( + Literal.apply(Array(Double.NaN, 1d)), Literal.apply(Array(Double.NaN))), + Seq(1d)) + checkEvaluation(ArrayExcept( + Literal.create(Seq(null, Double.NaN, null, 1d), ArrayType(DoubleType)), + Literal.create(Seq(Double.NaN, null), ArrayType(DoubleType))), + Seq(1d)) + checkEvaluation(ArrayExcept( + Literal.apply(Array(Float.NaN, 1f)), Literal.apply(Array(Float.NaN))), + Seq(1f)) + checkEvaluation(ArrayExcept( + Literal.create(Seq(null, Float.NaN, null, 1f), ArrayType(FloatType)), + Literal.create(Seq(Float.NaN, null), ArrayType(FloatType))), + Seq(1f)) + } + + test("SPARK-36754: ArrayIntersect should handle duplicated Double.NaN and Float.Nan") { + checkEvaluation(ArrayIntersect( + Literal.apply(Array(Double.NaN, 1d)), Literal.apply(Array(Double.NaN, 1d, 2d))), + Seq(Double.NaN, 1d)) + checkEvaluation(ArrayIntersect( + Literal.create(Seq(null, Double.NaN, null, 1d), ArrayType(DoubleType)), + Literal.create(Seq(null, Double.NaN, null), ArrayType(DoubleType))), + Seq(null, Double.NaN)) + checkEvaluation(ArrayIntersect( + Literal.apply(Array(Float.NaN, 1f)), Literal.apply(Array(Float.NaN, 1f, 2f))), + Seq(Float.NaN, 1f)) + checkEvaluation(ArrayIntersect( + Literal.create(Seq(null, Float.NaN, null, 1f), ArrayType(FloatType)), + Literal.create(Seq(null, Float.NaN, null), ArrayType(FloatType))), + Seq(null, Float.NaN)) + } + + test("SPARK-36741: ArrayDistinct should handle duplicated Double.NaN and Float.Nan") { + checkEvaluation(ArrayDistinct( + Literal.create(Seq(Double.NaN, Double.NaN, null, null, 1d, 1d), ArrayType(DoubleType))), + Seq(Double.NaN, null, 1d)) + checkEvaluation(ArrayDistinct( + Literal.create(Seq(Float.NaN, Float.NaN, null, null, 1f, 1f), ArrayType(FloatType))), + Seq(Float.NaN, null, 1f)) + } + + test("SPARK-36755: ArraysOverlap hould handle duplicated Double.NaN and Float.Nan") { + checkEvaluation(ArraysOverlap( + Literal.apply(Array(Double.NaN, 1d)), Literal.apply(Array(Double.NaN))), true) + checkEvaluation(ArraysOverlap( + Literal.create(Seq(Double.NaN, null), ArrayType(DoubleType)), + Literal.create(Seq(Double.NaN, null, 1d), ArrayType(DoubleType))), true) + checkEvaluation(ArraysOverlap( + Literal.apply(Array(Float.NaN)), Literal.apply(Array(Float.NaN, 1f))), true) + checkEvaluation(ArraysOverlap( + Literal.create(Seq(Float.NaN, null), ArrayType(FloatType)), + Literal.create(Seq(Float.NaN, null, 1f), ArrayType(FloatType))), true) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 79770505ec35d..763ecba2f3fcb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -1365,6 +1365,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation( SecondsToTimestamp(Literal(123.456789123)), Instant.ofEpochSecond(123, 456789000)) + checkEvaluation(SecondsToTimestamp(Literal(16777215.0f)), Instant.ofEpochSecond(16777215)) } test("TIMESTAMP_MILLIS") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathExpressionsSuite.scala index 6d09e28362e11..28c19d1274ad2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathExpressionsSuite.scala @@ -676,4 +676,32 @@ class MathExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(BRound(-0.35, 1), -0.4) checkEvaluation(BRound(-35, -1), -40) } + + test("SPARK-37388: width_bucket") { + val nullDouble = Literal.create(null, DoubleType) + val nullLong = Literal.create(null, LongType) + + checkEvaluation(WidthBucket(5.35, 0.024, 10.06, 5L), 3L) + checkEvaluation(WidthBucket(-2.1, 1.3, 3.4, 3L), 0L) + checkEvaluation(WidthBucket(8.1, 0.0, 5.7, 4L), 5L) + checkEvaluation(WidthBucket(-0.9, 5.2, 0.5, 2L), 3L) + checkEvaluation(WidthBucket(nullDouble, 0.024, 10.06, 5L), null) + checkEvaluation(WidthBucket(5.35, nullDouble, 10.06, 5L), null) + checkEvaluation(WidthBucket(5.35, 0.024, nullDouble, 5L), null) + checkEvaluation(WidthBucket(5.35, nullDouble, nullDouble, 5L), null) + checkEvaluation(WidthBucket(5.35, 0.024, 10.06, nullLong), null) + checkEvaluation(WidthBucket(nullDouble, nullDouble, nullDouble, nullLong), null) + checkEvaluation(WidthBucket(5.35, 0.024, 10.06, -5L), null) + checkEvaluation(WidthBucket(5.35, 0.024, 10.06, Long.MaxValue), null) + checkEvaluation(WidthBucket(Double.NaN, 0.024, 10.06, 5L), null) + checkEvaluation(WidthBucket(Double.NegativeInfinity, 0.024, 10.06, 5L), 0L) + checkEvaluation(WidthBucket(Double.PositiveInfinity, 0.024, 10.06, 5L), 6L) + checkEvaluation(WidthBucket(5.35, 0.024, 0.024, 5L), null) + checkEvaluation(WidthBucket(5.35, Double.NaN, 10.06, 5L), null) + checkEvaluation(WidthBucket(5.35, Double.NegativeInfinity, 10.06, 5L), null) + checkEvaluation(WidthBucket(5.35, Double.PositiveInfinity, 10.06, 5L), null) + checkEvaluation(WidthBucket(5.35, 0.024, Double.NaN, 5L), null) + checkEvaluation(WidthBucket(5.35, 0.024, Double.NegativeInfinity, 5L), null) + checkEvaluation(WidthBucket(5.35, 0.024, Double.PositiveInfinity, 5L), null) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala index bc2b93e5390da..35ceff6931d7e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala @@ -608,6 +608,53 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkExceptionInExpression[RuntimeException]( serializer4, EmptyRow, "Cannot use null as map key!") } + + test("SPARK-35244: invoke should throw the original exception") { + val strClsType = ObjectType(classOf[String]) + checkExceptionInExpression[StringIndexOutOfBoundsException]( + Invoke(Literal("a", strClsType), "substring", strClsType, Seq(Literal(3))), "") + + val mathCls = classOf[Math] + checkExceptionInExpression[ArithmeticException]( + StaticInvoke(mathCls, IntegerType, "addExact", Seq(Literal(Int.MaxValue), Literal(1))), "") + } + + test("SPARK-35278: invoke should find method with correct number of parameters") { + val strClsType = ObjectType(classOf[String]) + checkExceptionInExpression[StringIndexOutOfBoundsException]( + Invoke(Literal("a", strClsType), "substring", strClsType, Seq(Literal(3))), "") + + checkObjectExprEvaluation( + Invoke(Literal("a", strClsType), "substring", strClsType, Seq(Literal(0))), "a") + + checkExceptionInExpression[StringIndexOutOfBoundsException]( + Invoke(Literal("a", strClsType), "substring", strClsType, Seq(Literal(0), Literal(3))), "") + + checkObjectExprEvaluation( + Invoke(Literal("a", strClsType), "substring", strClsType, Seq(Literal(0), Literal(1))), "a") + } + + test("SPARK-35278: invoke should correctly invoke override method") { + val clsType = ObjectType(classOf[ConcreteClass]) + val obj = new ConcreteClass + + val input = (1, 2) + checkObjectExprEvaluation( + Invoke(Literal(obj, clsType), "testFunc", IntegerType, + Seq(Literal(input, ObjectType(input.getClass)))), 2) + } + + test("SPARK-35288: static invoke should find method without exact param type match") { + val input = (1, 2) + + checkObjectExprEvaluation( + StaticInvoke(TestStaticInvoke.getClass, IntegerType, "func", + Seq(Literal(input, ObjectType(input.getClass)))), 3) + + checkObjectExprEvaluation( + StaticInvoke(TestStaticInvoke.getClass, IntegerType, "func", + Seq(Literal(1, IntegerType))), -1) + } } class TestBean extends Serializable { @@ -618,3 +665,23 @@ class TestBean extends Serializable { def setNonPrimitive(i: AnyRef): Unit = assert(i != null, "this setter should not be called with null.") } + +object TestStaticInvoke { + def func(param: Any): Int = param match { + case pair: Tuple2[_, _] => + pair.asInstanceOf[Tuple2[Int, Int]]._1 + pair.asInstanceOf[Tuple2[Int, Int]]._2 + case _ => -1 + } +} + +abstract class BaseClass[T] { + def testFunc(param: T): Int +} + +class ConcreteClass extends BaseClass[Product] with Serializable { + override def testFunc(param: Product): Int = param match { + case _: Tuple2[_, _] => 2 + case _: Tuple3[_, _, _] => 3 + case _ => 4 + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala index 6f75623dc59ae..c34b37d7c6c88 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala @@ -644,4 +644,18 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { checkExpr(GreaterThan, Double.NaN, Double.NaN, false) checkExpr(GreaterThan, 0.0, -0.0, false) } + + test("SPARK-36792: InSet should handle Double.NaN and Float.NaN") { + checkInAndInSet(In(Literal(Double.NaN), Seq(Literal(Double.NaN), Literal(2d))), true) + checkInAndInSet(In(Literal.create(null, DoubleType), + Seq(Literal(Double.NaN), Literal(2d), Literal.create(null, DoubleType))), null) + checkInAndInSet(In(Literal.create(null, DoubleType), + Seq(Literal(Double.NaN), Literal(2d))), null) + checkInAndInSet(In(Literal(3d), + Seq(Literal(Double.NaN), Literal(2d))), false) + checkInAndInSet(In(Literal(3d), + Seq(Literal(Double.NaN), Literal(2d), Literal.create(null, DoubleType))), null) + checkInAndInSet(In(Literal(Double.NaN), + Seq(Literal(Double.NaN), Literal(2d), Literal.create(null, DoubleType))), true) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala index 8d7501d952ecb..019857580d077 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala @@ -22,6 +22,8 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.catalyst.optimizer.ConstantFolding +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StringType /** @@ -470,4 +472,12 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { cache.setAccessible(true) assert(cache.get(expr).asInstanceOf[java.util.regex.Pattern].pattern().contains("a")) } + + test("SPARK-34814: LikeSimplification should handle NULL") { + withSQLConf(SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> + ConstantFolding.getClass.getName.stripSuffix("$")) { + checkEvaluation(Literal.create("foo", StringType) + .likeAll("%foo%", Literal.create(null, StringType)), null) + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruningSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruningSuite.scala index c04f59ebb1b1b..2fab553183492 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruningSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruningSuite.scala @@ -18,28 +18,36 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.internal.SQLConf.CASE_SENSITIVE import org.apache.spark.sql.types._ -class SchemaPruningSuite extends SparkFunSuite { - test("prune schema by the requested fields") { - def testPrunedSchema( - schema: StructType, - requestedFields: StructField*): Unit = { - val requestedRootFields = requestedFields.map { f => - // `derivedFromAtt` doesn't affect the result of pruned schema. - SchemaPruning.RootField(field = f, derivedFromAtt = true) - } - val expectedSchema = SchemaPruning.pruneDataSchema(schema, requestedRootFields) - assert(expectedSchema == StructType(requestedFields)) +class SchemaPruningSuite extends SparkFunSuite with SQLHelper { + private def testPrunedSchema( + schema: StructType, + requestedFields: Seq[StructField], + expectedSchema: StructType): Unit = { + val requestedRootFields = requestedFields.map { f => + // `derivedFromAtt` doesn't affect the result of pruned schema. + SchemaPruning.RootField(field = f, derivedFromAtt = true) } + val prunedSchema = SchemaPruning.pruneDataSchema(schema, requestedRootFields) + assert(prunedSchema === expectedSchema) + } - testPrunedSchema(StructType.fromDDL("a int, b int"), StructField("a", IntegerType)) - testPrunedSchema(StructType.fromDDL("a int, b int"), StructField("b", IntegerType)) + test("prune schema by the requested fields") { + testPrunedSchema( + StructType.fromDDL("a int, b int"), + Seq(StructField("a", IntegerType)), + StructType.fromDDL("a int, b int")) val structOfStruct = StructType.fromDDL("a struct, b int") - testPrunedSchema(structOfStruct, StructField("a", StructType.fromDDL("a int, b int"))) - testPrunedSchema(structOfStruct, StructField("b", IntegerType)) - testPrunedSchema(structOfStruct, StructField("a", StructType.fromDDL("b int"))) + testPrunedSchema(structOfStruct, + Seq(StructField("a", StructType.fromDDL("a int")), StructField("b", IntegerType)), + StructType.fromDDL("a struct, b int")) + testPrunedSchema(structOfStruct, + Seq(StructField("a", StructType.fromDDL("a int"))), + StructType.fromDDL("a struct, b int")) val arrayOfStruct = StructField("a", ArrayType(StructType.fromDDL("a int, b int, c string"))) val mapOfStruct = StructField("d", MapType(StructType.fromDDL("a int, b int, c string"), @@ -49,14 +57,76 @@ class SchemaPruningSuite extends SparkFunSuite { arrayOfStruct :: StructField("b", structOfStruct) :: StructField("c", IntegerType) :: mapOfStruct :: Nil) - testPrunedSchema(complexStruct, StructField("a", ArrayType(StructType.fromDDL("b int"))), - StructField("b", StructType.fromDDL("a int"))) testPrunedSchema(complexStruct, - StructField("a", ArrayType(StructType.fromDDL("b int, c string"))), - StructField("b", StructType.fromDDL("b int"))) + Seq(StructField("a", ArrayType(StructType.fromDDL("b int"))), + StructField("b", StructType.fromDDL("a int"))), + StructType( + StructField("a", ArrayType(StructType.fromDDL("b int"))) :: + StructField("b", StructType.fromDDL("a int")) :: + StructField("c", IntegerType) :: + mapOfStruct :: Nil)) + testPrunedSchema(complexStruct, + Seq(StructField("a", ArrayType(StructType.fromDDL("b int, c string"))), + StructField("b", StructType.fromDDL("b int"))), + StructType( + StructField("a", ArrayType(StructType.fromDDL("b int, c string"))) :: + StructField("b", StructType.fromDDL("b int")) :: + StructField("c", IntegerType) :: + mapOfStruct :: Nil)) val selectFieldInMap = StructField("d", MapType(StructType.fromDDL("a int, b int"), StructType.fromDDL("e int, f string"))) - testPrunedSchema(complexStruct, StructField("c", IntegerType), selectFieldInMap) + testPrunedSchema(complexStruct, + Seq(StructField("c", IntegerType), selectFieldInMap), + StructType( + arrayOfStruct :: + StructField("b", structOfStruct) :: + StructField("c", IntegerType) :: + selectFieldInMap :: Nil)) + } + + test("SPARK-35096: test case insensitivity of pruned schema") { + val upperCaseSchema = StructType.fromDDL("A struct, B int") + val lowerCaseSchema = StructType.fromDDL("a struct, b int") + val upperCaseRequestedFields = Seq(StructField("A", StructType.fromDDL("A int"))) + val lowerCaseRequestedFields = Seq(StructField("a", StructType.fromDDL("a int"))) + + Seq(true, false).foreach { isCaseSensitive => + withSQLConf(CASE_SENSITIVE.key -> isCaseSensitive.toString) { + if (isCaseSensitive) { + testPrunedSchema( + upperCaseSchema, + upperCaseRequestedFields, + StructType.fromDDL("A struct, B int")) + testPrunedSchema( + upperCaseSchema, + lowerCaseRequestedFields, + upperCaseSchema) + + testPrunedSchema( + lowerCaseSchema, + upperCaseRequestedFields, + lowerCaseSchema) + testPrunedSchema( + lowerCaseSchema, + lowerCaseRequestedFields, + StructType.fromDDL("a struct, b int")) + } else { + Seq(upperCaseRequestedFields, lowerCaseRequestedFields).foreach { requestedFields => + testPrunedSchema( + upperCaseSchema, + requestedFields, + StructType.fromDDL("A struct, B int")) + } + + Seq(upperCaseRequestedFields, lowerCaseRequestedFields).foreach { requestedFields => + testPrunedSchema( + lowerCaseSchema, + requestedFields, + StructType.fromDDL("a struct, b int")) + } + } + } + } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala index 0147c6c6a8260..689164798dc06 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala @@ -17,10 +17,11 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback -import org.apache.spark.sql.types.{DataType, IntegerType} +import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{BinaryType, DataType, Decimal, IntegerType} -class SubexpressionEliminationSuite extends SparkFunSuite { +class SubexpressionEliminationSuite extends SparkFunSuite with ExpressionEvalHelper { test("Semantic equals and hash") { val a: AttributeReference = AttributeReference("name", IntegerType)() val id = { @@ -208,7 +209,7 @@ class SubexpressionEliminationSuite extends SparkFunSuite { (GreaterThan(add2, Literal(4)), add1) :: (GreaterThan(add2, Literal(5)), add1) :: Nil - val caseWhenExpr2 = CaseWhen(conditions2, None) + val caseWhenExpr2 = CaseWhen(conditions2, add1) val equivalence2 = new EquivalentExpressions equivalence2.addExprTree(caseWhenExpr2) @@ -253,6 +254,90 @@ class SubexpressionEliminationSuite extends SparkFunSuite { assert(equivalence2.getAllEquivalentExprs.count(_.size == 2) == 0) } + + test("SPARK-34723: Correct parameter type for subexpression elimination under whole-stage") { + withSQLConf(SQLConf.CODEGEN_METHOD_SPLIT_THRESHOLD.key -> "1") { + val str = BoundReference(0, BinaryType, false) + val pos = BoundReference(1, IntegerType, false) + + val substr = new Substring(str, pos) + + val add = Add(Length(substr), Literal(1)) + val add2 = Add(Length(substr), Literal(2)) + + val ctx = new CodegenContext() + val exprs = Seq(add, add2) + + val oneVar = ctx.freshVariable("str", BinaryType) + val twoVar = ctx.freshVariable("pos", IntegerType) + ctx.addMutableState("byte[]", oneVar, forceInline = true, useFreshName = false) + ctx.addMutableState("int", twoVar, useFreshName = false) + + ctx.INPUT_ROW = null + ctx.currentVars = Seq( + ExprCode(TrueLiteral, oneVar), + ExprCode(TrueLiteral, twoVar)) + + val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(exprs) + ctx.withSubExprEliminationExprs(subExprs.states) { + exprs.map(_.genCode(ctx)) + } + val subExprsCode = subExprs.codes.mkString("\n") + + val codeBody = s""" + public java.lang.Object generate(Object[] references) { + return new TestCode(references); + } + + class TestCode { + ${ctx.declareMutableStates()} + + public TestCode(Object[] references) { + } + + public void initialize(int partitionIndex) { + ${subExprsCode} + } + + ${ctx.declareAddedFunctions()} + } + """ + + val code = CodeFormatter.stripOverlappingComments( + new CodeAndComment(codeBody, ctx.getPlaceHolderToComments())) + + CodeGenerator.compile(code) + } + } + + test("SPARK-35499: Subexpressions should only be extracted from CaseWhen values with an " + + "elseValue") { + val add1 = Add(Literal(1), Literal(2)) + val add2 = Add(Literal(2), Literal(3)) + val conditions = (GreaterThan(add1, Literal(3)), add1) :: + (GreaterThan(add2, Literal(4)), add1) :: + (GreaterThan(add2, Literal(5)), add1) :: Nil + + val caseWhenExpr = CaseWhen(conditions, None) + val equivalence = new EquivalentExpressions + equivalence.addExprTree(caseWhenExpr) + + // `add1` is not in the elseValue, so we can't extract it from the branches + assert(equivalence.getAllEquivalentExprs.count(_.size == 2) == 0) + } + + test("SPARK-35886: PromotePrecision should not overwrite genCode") { + val p = PromotePrecision(Literal(Decimal("10.1"))) + + val ctx = new CodegenContext() + val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(Seq(p, p)) + val code = ctx.withSubExprEliminationExprs(subExprs.states) { + Seq(p.genCode(ctx)) + }.head + // Decimal `Literal` will add the value by `addReferenceObj`. + // So if `p` is replaced by subexpression, the literal will be reused. + assert(code.value.toString == "((Decimal) references[0] /* literal */)") + } } case class CodegenFallbackExpression(child: Expression) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseProjectSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseProjectSuite.scala index 42bcd13ee378d..1e7f9b0edd91c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseProjectSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseProjectSuite.scala @@ -170,4 +170,13 @@ class CollapseProjectSuite extends PlanTest { val expected = Sample(0.0, 0.6, false, 11L, relation.select('a as 'c)).analyze comparePlans(optimized, expected) } + + test("SPARK-36086: CollapseProject should keep output schema name") { + val relation = LocalRelation('a.int, 'b.int) + val select = relation.select(('a + 'b).as('c)).analyze + val query = Project(Seq(select.output.head.withName("C")), select) + val optimized = Optimize.execute(query) + val expected = relation.select(('a + 'b).as('C)).analyze + comparePlans(optimized, expected) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromGenerateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromGenerateSuite.scala index 93a1d414ed403..61ab4f027ed22 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromGenerateSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromGenerateSuite.scala @@ -17,14 +17,14 @@ package org.apache.spark.sql.catalyst.optimizer -import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.RuleExecutor -import org.apache.spark.sql.types.{IntegerType, StructField, StructType} +import org.apache.spark.sql.types.{ArrayType, IntegerType, StringType, StructField, StructType} class InferFiltersFromGenerateSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { @@ -34,7 +34,7 @@ class InferFiltersFromGenerateSuite extends PlanTest { val testRelation = LocalRelation('a.array(StructType(Seq( StructField("x", IntegerType), StructField("y", IntegerType) - ))), 'c1.string, 'c2.string) + ))), 'c1.string, 'c2.string, 'c3.int) Seq(Explode(_), PosExplode(_), Inline(_)).foreach { f => val generator = f('a) @@ -72,43 +72,53 @@ class InferFiltersFromGenerateSuite extends PlanTest { val optimized = Optimize.execute(originalQuery) comparePlans(optimized, originalQuery) } - } - // setup rules to test inferFilters with ConstantFolding to make sure - // the Filter rule added in inferFilters is removed again when doing - // explode with CreateArray/CreateMap - object OptimizeInferAndConstantFold extends RuleExecutor[LogicalPlan] { - val batches = - Batch("AnalysisNodes", Once, - EliminateSubqueryAliases) :: - Batch("Infer Filters", Once, InferFiltersFromGenerate) :: - Batch("ConstantFolding after", FixedPoint(4), - ConstantFolding, - NullPropagation, - PruneFilters) :: Nil + val generatorWithFromJson = f(JsonToStructs( + ArrayType(new StructType().add("s", "string")), + Map.empty, + 'c1)) + test("SPARK-37392: Don't infer filters from " + generatorWithFromJson) { + val originalQuery = testRelation.generate(generatorWithFromJson).analyze + val optimized = Optimize.execute(originalQuery) + comparePlans(optimized, originalQuery) + } + + val returnSchema = ArrayType(StructType(Seq( + StructField("x", IntegerType), + StructField("y", StringType) + ))) + val fakeUDF = ScalaUDF( + (i: Int) => Array(Row.fromSeq(Seq(1, "a")), Row.fromSeq(Seq(2, "b"))), + returnSchema, 'c3 :: Nil, Nil) + val generatorWithUDF = f(fakeUDF) + test("SPARK-36715: Don't infer filters from " + generatorWithUDF) { + val originalQuery = testRelation.generate(generatorWithUDF).analyze + val optimized = Optimize.execute(originalQuery) + comparePlans(optimized, originalQuery) + } } Seq(Explode(_), PosExplode(_)).foreach { f => - val createArrayExplode = f(CreateArray(Seq('c1))) - test("SPARK-33544: Don't infer filters from CreateArray " + createArrayExplode) { - val originalQuery = testRelation.generate(createArrayExplode).analyze - val optimized = OptimizeInferAndConstantFold.execute(originalQuery) - comparePlans(optimized, originalQuery) - } - val createMapExplode = f(CreateMap(Seq('c1, 'c2))) - test("SPARK-33544: Don't infer filters from CreateMap " + createMapExplode) { - val originalQuery = testRelation.generate(createMapExplode).analyze - val optimized = OptimizeInferAndConstantFold.execute(originalQuery) - comparePlans(optimized, originalQuery) - } - } + val createArrayExplode = f(CreateArray(Seq('c1))) + test("SPARK-33544: Don't infer filters from " + createArrayExplode) { + val originalQuery = testRelation.generate(createArrayExplode).analyze + val optimized = Optimize.execute(originalQuery) + comparePlans(optimized, originalQuery) + } + val createMapExplode = f(CreateMap(Seq('c1, 'c2))) + test("SPARK-33544: Don't infer filters from " + createMapExplode) { + val originalQuery = testRelation.generate(createMapExplode).analyze + val optimized = Optimize.execute(originalQuery) + comparePlans(optimized, originalQuery) + } + } - Seq(Inline(_)).foreach { f => - val createArrayStructExplode = f(CreateArray(Seq(CreateStruct(Seq('c1))))) - test("SPARK-33544: Don't infer filters from CreateArray " + createArrayStructExplode) { - val originalQuery = testRelation.generate(createArrayStructExplode).analyze - val optimized = OptimizeInferAndConstantFold.execute(originalQuery) - comparePlans(optimized, originalQuery) - } - } + Seq(Inline(_)).foreach { f => + val createArrayStructExplode = f(CreateArray(Seq(CreateStruct(Seq('c1))))) + test("SPARK-33544: Don't infer filters from " + createArrayStructExplode) { + val originalQuery = testRelation.generate(createArrayStructExplode).analyze + val optimized = Optimize.execute(originalQuery) + comparePlans(optimized, originalQuery) + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala index c83ab375ee15a..7a667cc6991c6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala @@ -684,6 +684,33 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { ).analyze comparePlans(optimized2, expected2) } + + test("SPARK-35972: NestedColumnAliasing should consider semantic equality") { + val dataType = new StructType() + .add(StructField("itemid", StringType)) + .add(StructField("search_params", StructType(Seq( + StructField("col1", StringType), + StructField("col2", StringType) + )))) + val relation = LocalRelation('struct_data.struct(dataType)) + val plan = relation + .repartition(100) + .select( + GetStructField('struct_data, 1, None).as("value"), + $"struct_data.search_params.col1".as("col1"), + $"struct_data.search_params.col2".as("col2")).analyze + val query = Optimize.execute(plan) + val alias = collectGeneratedAliases(query) + + val optimized = relation + .select(GetStructField('struct_data, 1, None).as(alias(0))) + .repartition(100) + .select( + $"${alias(0)}".as("value"), + $"${alias(0)}.col1".as("col1"), + $"${alias(0)}.col2".as("col2")).analyze + comparePlans(optimized, query) + } } object NestedColumnAliasingSuite { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeWithFieldsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeWithFieldsSuite.scala index b093b39cc4b88..e63742ac0de56 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeWithFieldsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeWithFieldsSuite.scala @@ -126,4 +126,25 @@ class OptimizeWithFieldsSuite extends PlanTest { comparePlans(optimized, correctAnswer) } } + + test("SPARK-35213: ensure optimize WithFields maintains correct WithField ordering") { + val originalQuery = testRelation + .select( + Alias(UpdateFields('a, + WithField("a1", Literal(3)) :: + WithField("b1", Literal(4)) :: + WithField("a1", Literal(5)) :: + Nil), "out")()) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = testRelation + .select( + Alias(UpdateFields('a, + WithField("a1", Literal(5)) :: + WithField("b1", Literal(4)) :: + Nil), "out")()) + .analyze + + comparePlans(optimized, correctAnswer) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveNoopOperatorsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveNoopOperatorsSuite.scala new file mode 100644 index 0000000000000..0f70e6fa739dd --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveNoopOperatorsSuite.scala @@ -0,0 +1,47 @@ +/* + * 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.catalyst.optimizer + +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.RuleExecutor + +class RemoveNoopOperatorsSuite extends PlanTest { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("RemoveNoopOperators", Once, + RemoveNoopOperators) :: Nil + } + + val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + + test("SPARK-36353: RemoveNoopOperators should keep output schema") { + val query = testRelation + .select(('a + 'b).as("c")) + .analyze + val originalQuery = Project(Seq(query.output.head.withName("C")), query) + val optimized = Optimize.execute(originalQuery.analyze) + val result = testRelation + .select(('a + 'b).as("C")) + .analyze + comparePlans(optimized, result) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala index 2e1cf4a137e25..7b591766cb4b2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala @@ -369,9 +369,6 @@ class JoinReorderSuite extends JoinReorderPlanTestBase with StatsEstimationTestB val plan1 = JoinPlan(null, null, null, Cost(300, 80)) val plan2 = JoinPlan(null, null, null, Cost(500, 30)) - // cost1 = 300*0.7 + 80*0.3 = 234 - // cost2 = 500*0.7 + 30*0.3 = 359 - assert(!plan1.betterThan(plan1, conf)) assert(!plan2.betterThan(plan2, conf)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/StarJoinCostBasedReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/StarJoinCostBasedReorderSuite.scala index a42914765dcc8..c4a660532b207 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/StarJoinCostBasedReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/StarJoinCostBasedReorderSuite.scala @@ -294,12 +294,13 @@ class StarJoinCostBasedReorderSuite extends JoinReorderPlanTestBase with StatsEs (nameToAttr("f1_fk2") === nameToAttr("d2_pk"))) val expected = - t3.join(t4, Inner, Some(nameToAttr("t3_c1") === nameToAttr("t4_c1"))) + f1 + .join(d2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk"))) + .join(d1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("d1_pk"))) + .join(t3.join(t4, Inner, Some(nameToAttr("t3_c1") === nameToAttr("t4_c1"))), Inner, + Some(nameToAttr("d1_c2") === nameToAttr("t4_c1"))) .join(t1.join(t2, Inner, Some(nameToAttr("t1_c1") === nameToAttr("t2_c1"))), Inner, Some(nameToAttr("t1_c2") === nameToAttr("t4_c2"))) - .join(f1 - .join(d2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk"))) - .join(d1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("d1_pk")))) .select(outputsOf(d1, t1, t2, t3, t4, f1, d2): _*) assertEqualJoinPlans(Optimize, query, expected) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 21a8dafd6b760..7b5e83f4396f3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -429,7 +429,7 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual("(a + b).b", ('a + 'b).getField("b")) // This will fail analysis. assertEqual( "struct(a, b).b", - namedStruct(NamePlaceholder, 'a, NamePlaceholder, 'b).getField("b")) + namedStruct(Literal("a"), 'a, Literal("b"), 'b).getField("b")) } test("reference") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/ScanOperationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/ScanOperationSuite.scala index 1290f770349e7..b1baeccbe94b9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/ScanOperationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/ScanOperationSuite.scala @@ -71,31 +71,19 @@ class ScanOperationSuite extends SparkFunSuite { } } - test("Filter which has the same non-deterministic expression with its child Project") { - val filter1 = Filter(EqualTo(colR, Literal(1)), Project(Seq(colA, aliasR), relation)) + test("Filter with non-deterministic Project") { + val filter1 = Filter(EqualTo(colA, Literal(1)), Project(Seq(colA, aliasR), relation)) assert(ScanOperation.unapply(filter1).isEmpty) } - test("Deterministic filter with a child Project with a non-deterministic expression") { - val filter2 = Filter(EqualTo(colA, Literal(1)), Project(Seq(colA, aliasR), relation)) - filter2 match { - case ScanOperation(projects, filters, _: LocalRelation) => - assert(projects.size === 2) - assert(projects(0) === colA) - assert(projects(1) === aliasR) - assert(filters.size === 1) - case _ => assert(false) - } - } - - test("Filter which has different non-deterministic expressions with its child Project") { + test("Non-deterministic Filter with deterministic Project") { val filter3 = Filter(EqualTo(MonotonicallyIncreasingID(), Literal(1)), - Project(Seq(colA, aliasR), relation)) + Project(Seq(colA, colB), relation)) filter3 match { case ScanOperation(projects, filters, _: LocalRelation) => assert(projects.size === 2) assert(projects(0) === colA) - assert(projects(1) === aliasR) + assert(projects(1) === colB) assert(filters.size === 1) case _ => assert(false) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala index 878fae4c547b3..2ec247564caf3 100755 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala @@ -822,6 +822,41 @@ class FilterEstimationSuite extends StatsEstimationTestBase { expectedRowCount = 3) } + test("SPARK-36079: Null count should be no higher than row count after filter") { + val colStatNullableString = colStatString.copy(nullCount = Some(10)) + val condition = Filter(EqualTo(attrBool, Literal(true)), + childStatsTestPlan(Seq(attrBool, attrString), tableRowCount = 10L, + attributeMap = AttributeMap(Seq( + attrBool -> colStatBool, attrString -> colStatNullableString)))) + validateEstimatedStats( + condition, + Seq(attrBool -> colStatBool.copy(distinctCount = Some(1), min = Some(true)), + attrString -> colStatNullableString.copy(distinctCount = Some(5), nullCount = Some(5))), + expectedRowCount = 5) + } + + test("SPARK-36079: Null count higher than row count") { + val colStatNullableString = colStatString.copy(nullCount = Some(15)) + val condition = Filter(IsNotNull(attrString), + childStatsTestPlan(Seq(attrString), tableRowCount = 10L, + attributeMap = AttributeMap(Seq(attrString -> colStatNullableString)))) + validateEstimatedStats( + condition, + Seq(attrString -> colStatNullableString), + expectedRowCount = 0) + } + + test("SPARK-36079: Bound selectivity >= 0") { + val colStatNullableString = colStatString.copy(nullCount = Some(-1)) + val condition = Filter(IsNotNull(attrString), + childStatsTestPlan(Seq(attrString), tableRowCount = 10L, + attributeMap = AttributeMap(Seq(attrString -> colStatNullableString)))) + validateEstimatedStats( + condition, + Seq(attrString -> colStatString), + expectedRowCount = 10) + } + test("ColumnStatsMap tests") { val attrNoDistinct = AttributeReference("att_without_distinct", IntegerType)() val attrNoCount = AttributeReference("att_without_count", BooleanType)() @@ -848,7 +883,10 @@ class FilterEstimationSuite extends StatsEstimationTestBase { assert(!columnStatsMap.hasMinMaxStats(attrNoMinMax)) } - private def childStatsTestPlan(outList: Seq[Attribute], tableRowCount: BigInt): StatsTestPlan = { + private def childStatsTestPlan( + outList: Seq[Attribute], + tableRowCount: BigInt, + attributeMap: AttributeMap[ColumnStat] = attributeMap): StatsTestPlan = { StatsTestPlan( outputList = outList, rowCount = tableRowCount, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala index 60dd9a1fe8a12..5855289692d42 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala @@ -75,7 +75,9 @@ class RuleExecutorSuite extends SparkFunSuite { test("structural integrity checker - verify initial input") { object WithSIChecker extends RuleExecutor[Expression] { - override protected def isPlanIntegral(expr: Expression): Boolean = expr match { + override protected def isPlanIntegral( + previousPlan: Expression, + currentPlan: Expression): Boolean = currentPlan match { case IntegerLiteral(_) => true case _ => false } @@ -93,7 +95,9 @@ class RuleExecutorSuite extends SparkFunSuite { test("structural integrity checker - verify rule execution result") { object WithSICheckerForPositiveLiteral extends RuleExecutor[Expression] { - override protected def isPlanIntegral(expr: Expression): Boolean = expr match { + override protected def isPlanIntegral( + previousPlan: Expression, + currentPlan: Expression): Boolean = currentPlan match { case IntegerLiteral(i) if i > 0 => true case _ => false } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index 4ad8475a0113c..9afad22debdfe 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -88,6 +88,8 @@ case class FakeLeafPlan(child: LogicalPlan) override def output: Seq[Attribute] = child.output } +case class FakeCurryingProduct(x: Expression)(val y: Int) + class TreeNodeSuite extends SparkFunSuite with SQLHelper { test("top node changed") { val after = Literal(1) transform { case Literal(1, _) => Literal(2) } @@ -594,6 +596,42 @@ class TreeNodeSuite extends SparkFunSuite with SQLHelper { "class" -> classOf[JsonTestTreeNode].getName, "num-children" -> 0, "arg" -> "1"))) + + // Convert Seq of Product contains TreeNode to JSON. + assertJSON( + Seq(("a", JsonTestTreeNode("0")), ("b", JsonTestTreeNode("1"))), + List( + JObject( + "product-class" -> "scala.Tuple2", + "_1" -> "a", + "_2" -> List(JObject( + "class" -> classOf[JsonTestTreeNode].getName, + "num-children" -> 0, + "arg" -> "0" + ))), + JObject( + "product-class" -> "scala.Tuple2", + "_1" -> "b", + "_2" -> List(JObject( + "class" -> classOf[JsonTestTreeNode].getName, + "num-children" -> 0, + "arg" -> "1" + ))))) + + // Convert currying product contains TreeNode to JSON. + assertJSON( + FakeCurryingProduct(Literal(1))(1), + JObject( + "product-class" -> classOf[FakeCurryingProduct].getName, + "x" -> List( + JObject( + "class" -> JString(classOf[Literal].getName), + "num-children" -> 0, + "value" -> "1", + "dataType" -> "integer")), + "y" -> 1 + ) + ) } test("toJSON should not throws java.lang.StackOverflowError") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index 17d8f55691409..81fa5db946827 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -283,6 +283,10 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { } } + test("SPARK-36076: Cast string to timestamp throw ArrayIndexOutOfBounds") { + assert(toTimestamp(":8:434421+ 98:38", UTC) === None) + } + test("SPARK-15379: special invalid date string") { // Test stringToDate assert(toDate("2015-02-29 00:00:00").isEmpty) @@ -676,48 +680,53 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { } } - test("ceilTimestamp") { - def testCeil( - level: Int, - expected: String, - inputTS: SQLTimestamp, - zoneId: ZoneId = ZoneId.systemDefault()): Unit = { - val ceilTS = - DateTimeUtils.ceilTimestamp(inputTS, level, zoneId) - val expectedTS = - DateTimeUtils.stringToTimestamp(UTF8String.fromString(expected), zoneId) - assert(ceilTS === expectedTS.get) - } + test("SPARK-35679: instantToMicros should be able to return microseconds of Long.MinValue") { + assert(instantToMicros(microsToInstant(Long.MaxValue)) === Long.MaxValue) + assert(instantToMicros(microsToInstant(Long.MinValue)) === Long.MinValue) + } - val defaultInputTS = - DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-05T09:32:05.359"), - ZoneId.systemDefault()) - val defaultInputTS1 = - DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-31T20:32:05.359"), - ZoneId.systemDefault()) - val defaultInputTS2 = - DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-04-01T02:32:05.359"), - ZoneId.systemDefault()) - val defaultInputTS3 = - DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-30T02:32:05.359"), - ZoneId.systemDefault()) - val defaultInputTS4 = - DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-29T02:32:05.359"), - ZoneId.systemDefault()) - - testCeil(DateTimeUtils.TRUNC_TO_YEAR, "2016-01-01T00:00:00", defaultInputTS.get) - testCeil(DateTimeUtils.TRUNC_TO_MONTH, "2015-04-01T00:00:00", defaultInputTS.get) - testCeil(DateTimeUtils.TRUNC_TO_DAY, "2015-03-06T00:00:00", defaultInputTS.get) - testCeil(DateTimeUtils.TRUNC_TO_HOUR, "2015-03-05T10:00:00", defaultInputTS.get) - testCeil(DateTimeUtils.TRUNC_TO_MINUTE, "2015-03-05T09:33:00", defaultInputTS.get) - testCeil(DateTimeUtils.TRUNC_TO_SECOND, "2015-03-05T09:32:06", defaultInputTS.get) - testCeil(DateTimeUtils.TRUNC_TO_WEEK, "2015-03-09T00:00:00", defaultInputTS.get) - testCeil(DateTimeUtils.TRUNC_TO_WEEK, "2015-04-06T00:00:00", defaultInputTS1.get) - testCeil(DateTimeUtils.TRUNC_TO_WEEK, "2015-04-06T00:00:00", defaultInputTS2.get) - testCeil(DateTimeUtils.TRUNC_TO_WEEK, "2015-04-06T00:00:00", defaultInputTS3.get) - testCeil(DateTimeUtils.TRUNC_TO_WEEK, "2015-03-30T00:00:00", defaultInputTS4.get) - testCeil(DateTimeUtils.TRUNC_TO_QUARTER, "2015-04-01T00:00:00", defaultInputTS.get) - testCeil(DateTimeUtils.TRUNC_TO_QUARTER, "2015-04-01T00:00:00", defaultInputTS1.get) - testCeil(DateTimeUtils.TRUNC_TO_QUARTER, "2015-07-01T00:00:00", defaultInputTS2.get) + test("ceilTimestamp") { + def testCeil( + level: Int, + expected: String, + inputTS: SQLTimestamp, + zoneId: ZoneId = ZoneId.systemDefault()): Unit = { + val ceilTS = + DateTimeUtils.ceilTimestamp(inputTS, level, zoneId) + val expectedTS = + DateTimeUtils.stringToTimestamp(UTF8String.fromString(expected), zoneId) + assert(ceilTS === expectedTS.get) } + + val defaultInputTS = + DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-05T09:32:05.359"), + ZoneId.systemDefault()) + val defaultInputTS1 = + DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-31T20:32:05.359"), + ZoneId.systemDefault()) + val defaultInputTS2 = + DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-04-01T02:32:05.359"), + ZoneId.systemDefault()) + val defaultInputTS3 = + DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-30T02:32:05.359"), + ZoneId.systemDefault()) + val defaultInputTS4 = + DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-29T02:32:05.359"), + ZoneId.systemDefault()) + + testCeil(DateTimeUtils.TRUNC_TO_YEAR, "2016-01-01T00:00:00", defaultInputTS.get) + testCeil(DateTimeUtils.TRUNC_TO_MONTH, "2015-04-01T00:00:00", defaultInputTS.get) + testCeil(DateTimeUtils.TRUNC_TO_DAY, "2015-03-06T00:00:00", defaultInputTS.get) + testCeil(DateTimeUtils.TRUNC_TO_HOUR, "2015-03-05T10:00:00", defaultInputTS.get) + testCeil(DateTimeUtils.TRUNC_TO_MINUTE, "2015-03-05T09:33:00", defaultInputTS.get) + testCeil(DateTimeUtils.TRUNC_TO_SECOND, "2015-03-05T09:32:06", defaultInputTS.get) + testCeil(DateTimeUtils.TRUNC_TO_WEEK, "2015-03-09T00:00:00", defaultInputTS.get) + testCeil(DateTimeUtils.TRUNC_TO_WEEK, "2015-04-06T00:00:00", defaultInputTS1.get) + testCeil(DateTimeUtils.TRUNC_TO_WEEK, "2015-04-06T00:00:00", defaultInputTS2.get) + testCeil(DateTimeUtils.TRUNC_TO_WEEK, "2015-04-06T00:00:00", defaultInputTS3.get) + testCeil(DateTimeUtils.TRUNC_TO_WEEK, "2015-03-30T00:00:00", defaultInputTS4.get) + testCeil(DateTimeUtils.TRUNC_TO_QUARTER, "2015-04-01T00:00:00", defaultInputTS.get) + testCeil(DateTimeUtils.TRUNC_TO_QUARTER, "2015-04-01T00:00:00", defaultInputTS1.get) + testCeil(DateTimeUtils.TRUNC_TO_QUARTER, "2015-07-01T00:00:00", defaultInputTS2.get) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/NumberConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/NumberConverterSuite.scala index ec73f4518737d..eb257b7975622 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/NumberConverterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/NumberConverterSuite.scala @@ -40,6 +40,10 @@ class NumberConverterSuite extends SparkFunSuite { checkConv("11abc", 10, 16, "B") } + test("SPARK-34909: convert negative to unsigned") { + checkConv("-10", 11, 7, "45012021522523134134555") + } + test("byte to binary") { checkToBinary(0.toByte) checkToBinary(1.toByte) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/PhysicalAggregationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/PhysicalAggregationSuite.scala new file mode 100644 index 0000000000000..b8c60dfbf4f97 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/PhysicalAggregationSuite.scala @@ -0,0 +1,54 @@ +/* + * 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.catalyst.util + +import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.planning.PhysicalAggregation +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation + +class PhysicalAggregationSuite extends PlanTest { + val testRelation = LocalRelation('a.int, 'b.int) + + test("SPARK-35014: a foldable expression should not be replaced by an AttributeReference") { + val query = testRelation + .groupBy('a, Literal.create(1) as 'k)( + 'a, Round(Literal.create(1.2), Literal.create(1)) as 'r, count('b) as 'c) + val analyzedQuery = SimpleAnalyzer.execute(query) + + val PhysicalAggregation( + groupingExpressions, + aggregateExpressions, + resultExpressions, + _ /* child */ + ) = analyzedQuery + + assertResult(2)(groupingExpressions.length) + assertResult(1)(aggregateExpressions.length) + assertResult(3)(resultExpressions.length) + + // Verify that Round's scale parameter is a Literal. + resultExpressions(1) match { + case Alias(Round(_, _: Literal), _) => + case other => fail("unexpected result expression: " + other) + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala index 7ce451ed6d577..5433c561a0379 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala @@ -286,4 +286,32 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester with SQLHelper val e = intercept[NumberFormatException](Decimal.fromStringANSI(UTF8String.fromString("str"))) assert(e.getMessage.contains("invalid input syntax for type numeric")) } + + test("SPARK-35841: Casting string to decimal type doesn't work " + + "if the sum of the digits is greater than 38") { + val values = Array( + "28.9259999999999983799625624669715762138", + "28.925999999999998379962562466971576213", + "2.9259999999999983799625624669715762138" + ) + for (string <- values) { + assert(Decimal.fromString(UTF8String.fromString(string)) === Decimal(string)) + assert(Decimal.fromStringANSI(UTF8String.fromString(string)) === Decimal(string)) + } + } + + test("SPARK-37451: Performance improvement regressed String to Decimal cast") { + val values = Array("7.836725755512218E38") + for (string <- values) { + assert(Decimal.fromString(UTF8String.fromString(string)) === null) + intercept[ArithmeticException](Decimal.fromStringANSI(UTF8String.fromString(string))) + } + + withSQLConf(SQLConf.LEGACY_ALLOW_NEGATIVE_SCALE_OF_DECIMAL_ENABLED.key -> "true") { + for (string <- values) { + assert(Decimal.fromString(UTF8String.fromString(string)) === Decimal(string)) + assert(Decimal.fromStringANSI(UTF8String.fromString(string)) === Decimal(string)) + } + } + } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java index 8c0f1e1257503..97a685a25a801 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java @@ -611,6 +611,9 @@ public final int appendByteArray(byte[] value, int offset, int length) { public final int appendArray(int length) { reserve(elementsAppended + 1); + for (WritableColumnVector childColumn : childColumns) { + childColumn.reserve(childColumn.elementsAppended + length); + } putArray(elementsAppended, arrayData().elementsAppended, length); return elementsAppended++; } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala index bbf0ac1dd85e9..308bb96502b19 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -395,10 +395,13 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { private def fillMap(values: Seq[(String, Any)]): DataFrame = { // Error handling - values.foreach { case (colName, replaceValue) => + val attrToValue = AttributeMap(values.map { case (colName, replaceValue) => // Check column name exists - df.resolve(colName) - + val attr = df.resolve(colName) match { + case a: Attribute => a + case _ => throw new UnsupportedOperationException( + s"Nested field ${colName} is not supported.") + } // Check data type replaceValue match { case _: jl.Double | _: jl.Float | _: jl.Integer | _: jl.Long | _: jl.Boolean | _: String => @@ -406,31 +409,29 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { case _ => throw new IllegalArgumentException( s"Unsupported value type ${replaceValue.getClass.getName} ($replaceValue).") } - } - - val columnEquals = df.sparkSession.sessionState.analyzer.resolver - val projections = df.schema.fields.map { f => - values.find { case (k, _) => columnEquals(k, f.name) }.map { case (_, v) => - v match { - case v: jl.Float => fillCol[Float](f, v) - case v: jl.Double => fillCol[Double](f, v) - case v: jl.Long => fillCol[Long](f, v) - case v: jl.Integer => fillCol[Integer](f, v) - case v: jl.Boolean => fillCol[Boolean](f, v.booleanValue()) - case v: String => fillCol[String](f, v) - } - }.getOrElse(df.col(f.name)) + attr -> replaceValue + }) + + val output = df.queryExecution.analyzed.output + val projections = output.map { + attr => attrToValue.get(attr).map { + case v: jl.Float => fillCol[Float](attr, v) + case v: jl.Double => fillCol[Double](attr, v) + case v: jl.Long => fillCol[Long](attr, v) + case v: jl.Integer => fillCol[Integer](attr, v) + case v: jl.Boolean => fillCol[Boolean](attr, v.booleanValue()) + case v: String => fillCol[String](attr, v) + }.getOrElse(Column(attr)) } df.select(projections : _*) } /** - * Returns a [[Column]] expression that replaces null value in `col` with `replacement`. - * It selects a column based on its name. + * Returns a [[Column]] expression that replaces null value in column defined by `attr` + * with `replacement`. */ - private def fillCol[T](col: StructField, replacement: T): Column = { - val quotedColName = "`" + col.name + "`" - fillCol(col.dataType, col.name, df.col(quotedColName), replacement) + private def fillCol[T](attr: Attribute, replacement: T): Column = { + fillCol(attr.dataType, attr.name, Column(attr), replacement) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index b94c42a2c9544..e4da076035171 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -73,8 +73,10 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * @since 1.4.0 */ def schema(schema: StructType): DataFrameReader = { - val replaced = CharVarcharUtils.failIfHasCharVarchar(schema).asInstanceOf[StructType] - this.userSpecifiedSchema = Option(replaced) + if (schema != null) { + val replaced = CharVarcharUtils.failIfHasCharVarchar(schema).asInstanceOf[StructType] + this.userSpecifiedSchema = Option(replaced) + } this } @@ -90,10 +92,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * @since 2.3.0 */ def schema(schemaString: String): DataFrameReader = { - val rawSchema = StructType.fromDDL(schemaString) - val schema = CharVarcharUtils.failIfHasCharVarchar(rawSchema).asInstanceOf[StructType] - this.userSpecifiedSchema = Option(schema) - this + schema(StructType.fromDDL(schemaString)) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 91607dcd09cc9..dd58f50cbe077 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import java.io.{ByteArrayOutputStream, CharArrayWriter, DataOutputStream} import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{ArrayBuffer, HashSet} import scala.reflect.runtime.universe.TypeTag import scala.util.control.NonFatal @@ -69,7 +69,7 @@ private[sql] object Dataset { val curId = new java.util.concurrent.atomic.AtomicLong() val DATASET_ID_KEY = "__dataset_id" val COL_POS_KEY = "__col_position" - val DATASET_ID_TAG = TreeNodeTag[Long]("dataset_id") + val DATASET_ID_TAG = TreeNodeTag[HashSet[Long]]("dataset_id") def apply[T: Encoder](sparkSession: SparkSession, logicalPlan: LogicalPlan): Dataset[T] = { val dataset = new Dataset(sparkSession, logicalPlan, implicitly[Encoder[T]]) @@ -231,9 +231,10 @@ class Dataset[T] private[sql]( case _ => queryExecution.analyzed } - if (sparkSession.sessionState.conf.getConf(SQLConf.FAIL_AMBIGUOUS_SELF_JOIN_ENABLED) && - plan.getTagValue(Dataset.DATASET_ID_TAG).isEmpty) { - plan.setTagValue(Dataset.DATASET_ID_TAG, id) + if (sparkSession.sessionState.conf.getConf(SQLConf.FAIL_AMBIGUOUS_SELF_JOIN_ENABLED)) { + val dsIds = plan.getTagValue(Dataset.DATASET_ID_TAG).getOrElse(new HashSet[Long]) + dsIds.add(id) + plan.setTagValue(Dataset.DATASET_ID_TAG, dsIds) } plan } @@ -1040,6 +1041,30 @@ class Dataset[T] private[sql]( */ def join(right: Dataset[_], joinExprs: Column): DataFrame = join(right, joinExprs, "inner") + /** + * find the trivially true predicates and automatically resolves them to both sides. + */ + private def resolveSelfJoinCondition(plan: Join): Join = { + val resolver = sparkSession.sessionState.analyzer.resolver + val cond = plan.condition.map { _.transform { + case catalyst.expressions.EqualTo(a: AttributeReference, b: AttributeReference) + if a.sameRef(b) => + catalyst.expressions.EqualTo( + plan.left.resolveQuoted(a.name, resolver) + .getOrElse(throw resolveException(a.name, plan.left.schema.fieldNames)), + plan.right.resolveQuoted(b.name, resolver) + .getOrElse(throw resolveException(b.name, plan.right.schema.fieldNames))) + case catalyst.expressions.EqualNullSafe(a: AttributeReference, b: AttributeReference) + if a.sameRef(b) => + catalyst.expressions.EqualNullSafe( + plan.left.resolveQuoted(a.name, resolver) + .getOrElse(throw resolveException(a.name, plan.left.schema.fieldNames)), + plan.right.resolveQuoted(b.name, resolver) + .getOrElse(throw resolveException(b.name, plan.right.schema.fieldNames))) + }} + plan.copy(condition = cond) + } + /** * Join with another `DataFrame`, using the given join expression. The following performs * a full outer join between `df1` and `df2`. @@ -1094,26 +1119,9 @@ class Dataset[T] private[sql]( // Otherwise, find the trivially true predicates and automatically resolves them to both sides. // By the time we get here, since we have already run analysis, all attributes should've been // resolved and become AttributeReference. - val resolver = sparkSession.sessionState.analyzer.resolver - val cond = plan.condition.map { _.transform { - case catalyst.expressions.EqualTo(a: AttributeReference, b: AttributeReference) - if a.sameRef(b) => - catalyst.expressions.EqualTo( - plan.left.resolveQuoted(a.name, resolver) - .getOrElse(throw resolveException(a.name, plan.left.schema.fieldNames)), - plan.right.resolveQuoted(b.name, resolver) - .getOrElse(throw resolveException(b.name, plan.right.schema.fieldNames))) - case catalyst.expressions.EqualNullSafe(a: AttributeReference, b: AttributeReference) - if a.sameRef(b) => - catalyst.expressions.EqualNullSafe( - plan.left.resolveQuoted(a.name, resolver) - .getOrElse(throw resolveException(a.name, plan.left.schema.fieldNames)), - plan.right.resolveQuoted(b.name, resolver) - .getOrElse(throw resolveException(b.name, plan.right.schema.fieldNames))) - }} withPlan { - plan.copy(condition = cond) + resolveSelfJoinCondition(plan) } } @@ -1155,7 +1163,7 @@ class Dataset[T] private[sql]( def joinWith[U](other: Dataset[U], condition: Column, joinType: String): Dataset[(T, U)] = { // Creates a Join node and resolve it first, to get join condition resolved, self-join resolved, // etc. - val joined = sparkSession.sessionState.executePlan( + var joined = sparkSession.sessionState.executePlan( Join( this.logicalPlan, other.logicalPlan, @@ -1167,6 +1175,11 @@ class Dataset[T] private[sql]( throw new AnalysisException("Invalid join type in joinWith: " + joined.joinType.sql) } + // If auto self join alias is enable + if (sqlContext.conf.dataFrameSelfJoinAutoResolveAmbiguity) { + joined = resolveSelfJoinCondition(joined) + } + implicit val tuple2Encoder: Encoder[(T, U)] = ExpressionEncoder.tuple(this.exprEnc, other.exprEnc) @@ -2413,10 +2426,10 @@ class Dataset[T] private[sql]( val resolver = sparkSession.sessionState.analyzer.resolver val output = queryExecution.analyzed.output - val columnMap = colNames.zip(cols).toMap + val columnSeq = colNames.zip(cols) val replacedAndExistingColumns = output.map { field => - columnMap.find { case (colName, _) => + columnSeq.find { case (colName, _) => resolver(field.name, colName) } match { case Some((colName: String, col: Column)) => col.as(colName) @@ -2424,7 +2437,7 @@ class Dataset[T] private[sql]( } } - val newColumns = columnMap.filter { case (colName, col) => + val newColumns = columnSeq.filter { case (colName, col) => !output.exists(f => resolver(f.name, colName)) }.map { case (colName, col) => col.as(colName) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index c40ce0f4777c6..7e735eecbac3e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.annotation.Stable import org.apache.spark.api.python.PythonEvalType import org.apache.spark.broadcast.Broadcast -import org.apache.spark.sql.catalyst.analysis.{Star, UnresolvedAlias, UnresolvedAttribute, UnresolvedFunction} +import org.apache.spark.sql.catalyst.analysis.{Star, UnresolvedAlias, UnresolvedFunction} import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ @@ -80,11 +80,7 @@ class RelationalGroupedDataset protected[sql]( } } - // Wrap UnresolvedAttribute with UnresolvedAlias, as when we resolve UnresolvedAttribute, we - // will remove intermediate Alias for ExtractValue chain, and we need to alias it again to - // make it a NamedExpression. private[this] def alias(expr: Expression): NamedExpression = expr match { - case u: UnresolvedAttribute => UnresolvedAlias(u) case expr: NamedExpression => expr case a: AggregateExpression if a.aggregateFunction.isInstanceOf[TypedAggregateExpression] => UnresolvedAlias(a, Some(Column.generateAlias)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/AggregatingAccumulator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/AggregatingAccumulator.scala index 94e159c562e31..36de9d84cb930 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/AggregatingAccumulator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/AggregatingAccumulator.scala @@ -33,7 +33,7 @@ class AggregatingAccumulator private( bufferSchema: Seq[DataType], initialValues: Seq[Expression], updateExpressions: Seq[Expression], - @transient private val mergeExpressions: Seq[Expression], + mergeExpressions: Seq[Expression], @transient private val resultExpressions: Seq[Expression], imperatives: Array[ImperativeAggregate], typedImperatives: Array[TypedImperativeAggregate[_]], @@ -95,13 +95,14 @@ class AggregatingAccumulator private( /** * Driver side operations like `merge` and `value` are executed in the DAGScheduler thread. This - * thread does not have a SQL configuration so we attach our own here. Note that we can't (and - * shouldn't) call `merge` or `value` on an accumulator originating from an executor so we just - * return a default value here. + * thread does not have a SQL configuration so we attach our own here. */ - private[this] def withSQLConf[T](default: => T)(body: => T): T = { + private[this] def withSQLConf[T](canRunOnExecutor: Boolean, default: => T)(body: => T): T = { if (conf != null) { + // When we can reach here, we are on the driver side. SQLConf.withExistingConf(conf)(body) + } else if (canRunOnExecutor) { + body } else { default } @@ -147,7 +148,8 @@ class AggregatingAccumulator private( } } - override def merge(other: AccumulatorV2[InternalRow, InternalRow]): Unit = withSQLConf(()) { + override def merge( + other: AccumulatorV2[InternalRow, InternalRow]): Unit = withSQLConf(true, ()) { if (!other.isZero) { other match { case agg: AggregatingAccumulator => @@ -160,9 +162,18 @@ class AggregatingAccumulator private( i += 1 } i = 0 - while (i < typedImperatives.length) { - typedImperatives(i).mergeBuffersObjects(buffer, otherBuffer) - i += 1 + if (isAtDriverSide) { + while (i < typedImperatives.length) { + // The input buffer stores serialized data + typedImperatives(i).merge(buffer, otherBuffer) + i += 1 + } + } else { + while (i < typedImperatives.length) { + // The input buffer stores deserialized object + typedImperatives(i).mergeBuffersObjects(buffer, otherBuffer) + i += 1 + } } case _ => throw new UnsupportedOperationException( @@ -171,7 +182,7 @@ class AggregatingAccumulator private( } } - override def value: InternalRow = withSQLConf(InternalRow.empty) { + override def value: InternalRow = withSQLConf(false, InternalRow.empty) { // Either use the existing buffer or create a temporary one. val input = if (!isZero) { buffer @@ -185,6 +196,17 @@ class AggregatingAccumulator private( resultProjection(input) } + override def withBufferSerialized(): AggregatingAccumulator = { + assert(!isAtDriverSide) + var i = 0 + // AggregatingAccumulator runs on executor, we should serialize all TypedImperativeAggregate. + while (i < typedImperatives.length) { + typedImperatives(i).serializeAggregateBufferInPlace(buffer) + i += 1 + } + this + } + /** * Get the output schema of the aggregating accumulator. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CollectMetricsExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CollectMetricsExec.scala index b0bbb52bc4990..c9e1bc3486e89 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CollectMetricsExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CollectMetricsExec.scala @@ -22,6 +22,8 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.{Attribute, NamedExpression, SortOrder} import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.types.StructType /** @@ -69,7 +71,11 @@ case class CollectMetricsExec( // - Performance issues due to excessive serialization. val updater = collector.copyAndReset() TaskContext.get().addTaskCompletionListener[Unit] { _ => - collector.setState(updater) + if (collector.isZero) { + collector.setState(updater) + } else { + collector.merge(updater) + } } rows.map { r => @@ -80,14 +86,17 @@ case class CollectMetricsExec( } } -object CollectMetricsExec { +object CollectMetricsExec extends AdaptiveSparkPlanHelper { /** * Recursively collect all collected metrics from a query tree. */ def collect(plan: SparkPlan): Map[String, Row] = { - val metrics = plan.collectWithSubqueries { - case collector: CollectMetricsExec => collector.name -> collector.collectedMetrics + val metrics = collectWithSubqueries(plan) { + case collector: CollectMetricsExec => + Map(collector.name -> collector.collectedMetrics) + case tableScan: InMemoryTableScanExec => + CollectMetricsExec.collect(tableScan.relation.cachedPlan) } - metrics.toMap + metrics.reduceOption(_ ++ _).getOrElse(Map.empty) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala index 8d542792a0e28..ccb525d2e192e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala @@ -261,12 +261,12 @@ private object RowToColumnConverter { case DoubleType => DoubleConverter case StringType => StringConverter case CalendarIntervalType => CalendarConverter - case at: ArrayType => new ArrayConverter(getConverterForType(at.elementType, nullable)) + case at: ArrayType => ArrayConverter(getConverterForType(at.elementType, at.containsNull)) case st: StructType => new StructConverter(st.fields.map( (f) => getConverterForType(f.dataType, f.nullable))) case dt: DecimalType => new DecimalConverter(dt) - case mt: MapType => new MapConverter(getConverterForType(mt.keyType, nullable), - getConverterForType(mt.valueType, nullable)) + case mt: MapType => MapConverter(getConverterForType(mt.keyType, nullable = false), + getConverterForType(mt.valueType, mt.valueContainsNull)) case unknown => throw new UnsupportedOperationException( s"Type $unknown not supported") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/RemoveRedundantProjects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/RemoveRedundantProjects.scala index bfb6e805c0541..8f4ce0f49a89a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/RemoveRedundantProjects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/RemoveRedundantProjects.scala @@ -48,8 +48,10 @@ object RemoveRedundantProjects extends Rule[SparkPlan] { private def removeProject(plan: SparkPlan, requireOrdering: Boolean): SparkPlan = { plan match { case p @ ProjectExec(_, child) => - if (isRedundant(p, child, requireOrdering)) { + if (isRedundant(p, child, requireOrdering) && canRemove(p, child)) { val newPlan = removeProject(child, requireOrdering) + // The `newPlan` should retain the logical plan link already. We call `setLogicalLink` + // here to make sure the `newPlan` sets the `LOGICAL_PLAN_TAG` tag. newPlan.setLogicalLink(child.logicalLink.get) newPlan } else { @@ -97,6 +99,7 @@ object RemoveRedundantProjects extends Rule[SparkPlan] { // If a DataSourceV2ScanExec node does not support columnar, a ProjectExec node is required // to convert the rows to UnsafeRow. See DataSourceV2Strategy for more details. case d: DataSourceV2ScanExecBase if !d.supportsColumnar => false + case FilterExec(_, d: DataSourceV2ScanExecBase) if !d.supportsColumnar => false case _ => if (requireOrdering) { project.output.map(_.exprId.id) == child.output.map(_.exprId.id) && @@ -109,4 +112,11 @@ object RemoveRedundantProjects extends Rule[SparkPlan] { } } } + + // SPARK-36020: Currently a project can only be removed if (1) its logical link is empty or (2) + // its logical link is the same as the child's logical link. This is to ensure the physical + // plan node can correctly map to its logical plan node in AQE. + private def canRemove(project: ProjectExec, child: SparkPlan): Boolean = { + project.logicalLink.isEmpty || project.logicalLink.exists(child.logicalLink.contains) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEOptimizer.scala index 04b8ade8ac629..bddb81ccc10c8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEOptimizer.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.adaptive import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LogicalPlanIntegrity, PlanHelper} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.DataType import org.apache.spark.util.Utils /** @@ -55,9 +56,12 @@ class AQEOptimizer(conf: SQLConf) extends RuleExecutor[LogicalPlan] { } } - override protected def isPlanIntegral(plan: LogicalPlan): Boolean = { - !Utils.isTesting || (plan.resolved && - plan.find(PlanHelper.specialExpressionsInUnsupportedOperator(_).nonEmpty).isEmpty && - LogicalPlanIntegrity.checkIfExprIdsAreGloballyUnique(plan)) + override protected def isPlanIntegral( + previousPlan: LogicalPlan, + currentPlan: LogicalPlan): Boolean = { + !Utils.isTesting || (currentPlan.resolved && + currentPlan.find(PlanHelper.specialExpressionsInUnsupportedOperator(_).nonEmpty).isEmpty && + LogicalPlanIntegrity.checkIfExprIdsAreGloballyUnique(currentPlan) && + DataType.equalsIgnoreNullability(previousPlan.schema, currentPlan.schema)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 41048fc16235e..bb0ffaa2b819f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -419,7 +419,8 @@ case class AdaptiveSparkPlanExec( // Check the `stageCache` again for reuse. If a match is found, ditch the new stage // and reuse the existing stage found in the `stageCache`, otherwise update the // `stageCache` with the new stage. - val queryStage = context.stageCache.getOrElseUpdate(e.canonicalized, newStage) + val queryStage = context.stageCache.getOrElseUpdate( + newStage.plan.canonicalized, newStage) if (queryStage.ne(newStage)) { newStage = reuseQueryStage(queryStage, e) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala index 49a4c25fa637f..2319c9ed1cfd3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala @@ -179,12 +179,12 @@ case class CustomShuffleReaderExec private( } private lazy val shuffleRDD: RDD[_] = { - sendDriverMetrics() - - shuffleStage.map { stage => - stage.shuffle.getShuffleRDD(partitionSpecs.toArray) - }.getOrElse { - throw new IllegalStateException("operating on canonicalized plan") + shuffleStage match { + case Some(stage) => + sendDriverMetrics() + stage.shuffle.getShuffleRDD(partitionSpecs.toArray) + case _ => + throw new IllegalStateException("operating on canonicalized plan") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/analysis/DetectAmbiguousSelfJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/analysis/DetectAmbiguousSelfJoin.scala index b26a0785b7f4f..781a7ab60c224 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/analysis/DetectAmbiguousSelfJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/analysis/DetectAmbiguousSelfJoin.scala @@ -57,8 +57,8 @@ object DetectAmbiguousSelfJoin extends Rule[LogicalPlan] { } object LogicalPlanWithDatasetId { - def unapply(p: LogicalPlan): Option[(LogicalPlan, Long)] = { - p.getTagValue(Dataset.DATASET_ID_TAG).map(id => p -> id) + def unapply(p: LogicalPlan): Option[(LogicalPlan, mutable.HashSet[Long])] = { + p.getTagValue(Dataset.DATASET_ID_TAG).map(ids => p -> ids) } } @@ -89,9 +89,9 @@ object DetectAmbiguousSelfJoin extends Rule[LogicalPlan] { val inputAttrs = AttributeSet(plan.children.flatMap(_.output)) plan.foreach { - case LogicalPlanWithDatasetId(p, id) if dsIdSet.contains(id) => + case LogicalPlanWithDatasetId(p, ids) if dsIdSet.intersect(ids).nonEmpty => colRefs.foreach { ref => - if (id == ref.datasetId) { + if (ids.contains(ref.datasetId)) { if (ref.colPos < 0 || ref.colPos >= p.output.length) { throw new IllegalStateException("[BUG] Hit an invalid Dataset column reference: " + s"$ref. Please open a JIRA ticket to report it.") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index d651132d7abf5..4fcd67b18f467 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -690,12 +690,13 @@ case class CoalesceExec(numPartitions: Int, child: SparkPlan) extends UnaryExecN } protected override def doExecute(): RDD[InternalRow] = { - if (numPartitions == 1 && child.execute().getNumPartitions < 1) { + val rdd = child.execute() + if (numPartitions == 1 && rdd.getNumPartitions < 1) { // Make sure we don't output an RDD with 0 partitions, when claiming that we have a // `SinglePartition`. new CoalesceExec.EmptyRDDWithPartitions(sparkContext, numPartitions) } else { - child.execute().coalesce(numPartitions, shuffle = false) + rdd.coalesce(numPartitions, shuffle = false) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/DisableUnnecessaryBucketedScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/DisableUnnecessaryBucketedScan.scala index 6b195b3b49f09..62969dc043328 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/DisableUnnecessaryBucketedScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/DisableUnnecessaryBucketedScan.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.bucketing -import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, HashClusteredDistribution} +import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, HashClusteredDistribution} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{FileSourceScanExec, FilterExec, ProjectExec, SortExec, SparkPlan} import org.apache.spark.sql.execution.aggregate.BaseAggregateExec @@ -120,7 +120,7 @@ object DisableUnnecessaryBucketedScan extends Rule[SparkPlan] { private def hasInterestingPartition(plan: SparkPlan): Boolean = { plan.requiredChildDistribution.exists { - case _: ClusteredDistribution | _: HashClusteredDistribution => true + case _: ClusteredDistribution | _: HashClusteredDistribution | AllTuples => true case _ => false } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala index a56007f5d5d95..c9de8c7e1a9d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala @@ -19,12 +19,13 @@ package org.apache.spark.sql.execution.command import org.apache.hadoop.conf.Configuration +import org.apache.spark.SparkContext import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} import org.apache.spark.sql.execution.datasources.BasicWriteJobStatsTracker -import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.util.SerializableConfiguration /** @@ -73,4 +74,26 @@ object DataWritingCommand { attr.withName(outputName) } } + + /** + * When execute CTAS operators, Spark will use [[InsertIntoHadoopFsRelationCommand]] + * or [[InsertIntoHiveTable]] command to write data, they both inherit metrics from + * [[DataWritingCommand]], but after running [[InsertIntoHadoopFsRelationCommand]] + * or [[InsertIntoHiveTable]], we only update metrics in these two command through + * [[BasicWriteJobStatsTracker]], we also need to propogate metrics to the command + * that actually calls [[InsertIntoHadoopFsRelationCommand]] or [[InsertIntoHiveTable]]. + * + * @param sparkContext Current SparkContext. + * @param command Command to execute writing data. + * @param metrics Metrics of real DataWritingCommand. + */ + def propogateMetrics( + sparkContext: SparkContext, + command: DataWritingCommand, + metrics: Map[String, SQLMetric]): Unit = { + command.metrics.foreach { case (key, metric) => metrics(key).set(metric.value) } + SQLMetrics.postDriverMetricUpdates(sparkContext, + sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY), + metrics.values.toSeq) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala index 7d92e6e189fb2..888fb33764588 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala @@ -138,7 +138,7 @@ case class SetCommand(kv: Option[(String, Option[String])]) extends RunnableComm s"showing ${SQLConf.SHUFFLE_PARTITIONS.key} instead.") Seq(Row( SQLConf.SHUFFLE_PARTITIONS.key, - sparkSession.sessionState.conf.numShufflePartitions.toString)) + sparkSession.sessionState.conf.defaultNumShufflePartitions.toString)) } (keyValueOutput, runFunc) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index be7fa7b1b447e..dc26e00599aea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -217,7 +217,7 @@ case class CreateDataSourceTableAsSelectCommand( catalogTable = if (tableExists) Some(table) else None) try { - dataSource.writeAndRead(mode, query, outputColumnNames, physicalPlan) + dataSource.writeAndRead(mode, query, outputColumnNames, physicalPlan, metrics) } catch { case ex: AnalysisException => logError(s"Failed to write to table ${table.identifier.unquotedString}", ex) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 2f96c453a6e30..89c41ae660217 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -281,6 +281,7 @@ case class AlterTableSetPropertiesCommand( properties = table.properties ++ properties, comment = properties.get(TableCatalog.PROP_COMMENT).orElse(table.comment)) catalog.alterTable(newTable) + catalog.invalidateCachedTable(tableName) Seq.empty[Row] } @@ -319,6 +320,7 @@ case class AlterTableUnsetPropertiesCommand( val newProperties = table.properties.filter { case (k, _) => !propKeys.contains(k) } val newTable = table.copy(properties = newProperties, comment = tableComment) catalog.alterTable(newTable) + catalog.invalidateCachedTable(tableName) Seq.empty[Row] } @@ -678,7 +680,15 @@ case class AlterTableRecoverPartitionsCommand( // This is always the case for Hive format tables, but is not true for Datasource tables created // before Spark 2.1 unless they are converted via `msck repair table`. spark.sessionState.catalog.alterTable(table.copy(tracksPartitionsInCatalog = true)) - spark.catalog.refreshTable(tableIdentWithDB) + try { + spark.catalog.refreshTable(tableIdentWithDB) + } catch { + case NonFatal(e) => + logError(s"Cannot refresh the table '$tableIdentWithDB'. A query of the table " + + "might return wrong result if the table was cached. To avoid such issue, you should " + + "uncache the table manually via the UNCACHE TABLE command after table recovering will " + + "complete fully.", e) + } logInfo(s"Recovered all partitions ($total).") Seq.empty[Row] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index f88200e19448b..b557fd276d3de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -693,7 +693,12 @@ case class DescribeTableCommand( s"DESC PARTITION is not allowed on a view: ${table.identifier}") } DDLUtils.verifyPartitionProviderIsHive(spark, metadata, "DESC PARTITION") - val partition = catalog.getPartition(table, partitionSpec) + val normalizedPartSpec = PartitioningUtils.normalizePartitionSpec( + partitionSpec, + metadata.partitionSchema, + table.quotedString, + spark.sessionState.conf.resolver) + val partition = catalog.getPartition(table, normalizedPartSpec) if (isExtended) describeFormattedDetailedPartitionInfo(table, metadata, partition, result) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 960fe4ad22836..c58c6b4d34baa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -351,10 +351,26 @@ object ViewHelper { "spark.sql.codegen.", "spark.sql.execution.", "spark.sql.shuffle.", - "spark.sql.adaptive.") + "spark.sql.adaptive.", + // ignore optimization configs used in `RelationConversions` + "spark.sql.hive.convertMetastoreParquet", + "spark.sql.hive.convertMetastoreOrc", + "spark.sql.hive.convertInsertingPartitionedTable", + "spark.sql.hive.convertMetastoreCtas", + SQLConf.ADDITIONAL_REMOTE_REPOSITORIES.key) + + private val configAllowList = Seq( + SQLConf.DISABLE_HINTS.key + ) + /** + * Capture view config either of: + * 1. exists in allowList + * 2. do not exists in denyList + */ private def shouldCaptureConfig(key: String): Boolean = { - !configPrefixDenyList.exists(prefix => key.startsWith(prefix)) + configAllowList.exists(prefix => key.equals(prefix)) || + !configPrefixDenyList.exists(prefix => key.startsWith(prefix)) } import CatalogTable._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index e84f5943d30f0..476174a2cad9f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -43,6 +43,7 @@ import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 +import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources.{RateStreamProvider, TextSocketSourceProvider} import org.apache.spark.sql.internal.SQLConf @@ -518,7 +519,8 @@ case class DataSource( mode: SaveMode, data: LogicalPlan, outputColumnNames: Seq[String], - physicalPlan: SparkPlan): BaseRelation = { + physicalPlan: SparkPlan, + metrics: Map[String, SQLMetric]): BaseRelation = { val outputColumns = DataWritingCommand.logicalPlanOutputWithNames(data, outputColumnNames) if (outputColumns.map(_.dataType).exists(_.isInstanceOf[CalendarIntervalType])) { throw new AnalysisException("Cannot save interval data type into external storage.") @@ -546,6 +548,7 @@ case class DataSource( partitionColumns = resolvedPartCols, outputColumnNames = outputColumnNames) resolved.run(sparkSession, physicalPlan) + DataWritingCommand.propogateMetrics(sparkSession.sparkContext, resolved, metrics) // Replace the schema with that of the DataFrame we just wrote out to avoid re-inferring copy(userSpecifiedSchema = Some(outputColumns.toStructType.asNullable)).resolveRelation() case _ => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index e4f001d61a767..439fe75e430ae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -471,7 +471,7 @@ object DataSourceStrategy */ protected[sql] def normalizeExprs( exprs: Seq[Expression], - attributes: Seq[AttributeReference]): Seq[Expression] = { + attributes: Seq[Attribute]): Seq[Expression] = { exprs.map { e => e transform { case a: AttributeReference => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala index a0b191e60f376..4ed8943ef46f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala @@ -40,6 +40,8 @@ import org.apache.spark.sql.types.{StructField, StructType} case class HadoopFsRelation( location: FileIndex, partitionSchema: StructType, + // The top-level columns in `dataSchema` should match the actual physical file schema, otherwise + // the ORC data source may not work with the by-ordinal mode. dataSchema: StructType, bucketSpec: Option[BucketSpec], fileFormat: FileFormat, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index 69123ee7af5b9..947b1e904f96c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCoercion import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.getPartitionValueString import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal} import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateFormatter, DateTimeUtils, TimestampFormatter} import org.apache.spark.sql.types._ @@ -349,7 +350,7 @@ object PartitioningUtils { */ def getPathFragment(spec: TablePartitionSpec, partitionSchema: StructType): String = { partitionSchema.map { field => - escapePathName(field.name) + "=" + escapePathName(spec(field.name)) + escapePathName(field.name) + "=" + getPartitionValueString(spec(field.name)) }.mkString("/") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala index 0c56e7675da6f..6a33d1c1b124d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala @@ -110,8 +110,7 @@ private[sql] object PruneFileSourcePartitions } case op @ PhysicalOperation(projects, filters, - v2Relation @ DataSourceV2ScanRelation(_, scan: FileScan, output)) - if filters.nonEmpty && scan.readDataSchema.nonEmpty => + v2Relation @ DataSourceV2ScanRelation(_, scan: FileScan, output)) if filters.nonEmpty => val (partitionKeyFilters, dataFilters) = getPartitionKeyFiltersAndDataFilters(scan.sparkSession, v2Relation, scan.readPartitionSchema, filters, output) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaPruning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaPruning.scala index 76a6a48ca0b0c..55ae49e31a1a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaPruning.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaPruning.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructType} +import org.apache.spark.sql.util.SchemaUtils._ /** * Prunes unnecessary physical columns given a [[PhysicalOperation]] over a data source relation. @@ -83,8 +84,8 @@ object SchemaPruning extends Rule[LogicalPlan] { val prunedRelation = leafNodeBuilder(prunedDataSchema) val projectionOverSchema = ProjectionOverSchema(prunedDataSchema) - Some(buildNewProjection(normalizedProjects, normalizedFilters, prunedRelation, - projectionOverSchema)) + Some(buildNewProjection(projects, normalizedProjects, normalizedFilters, + prunedRelation, projectionOverSchema)) } else { None } @@ -126,6 +127,7 @@ object SchemaPruning extends Rule[LogicalPlan] { */ private def buildNewProjection( projects: Seq[NamedExpression], + normalizedProjects: Seq[NamedExpression], filters: Seq[Expression], leafNode: LeafNode, projectionOverSchema: ProjectionOverSchema): Project = { @@ -144,7 +146,7 @@ object SchemaPruning extends Rule[LogicalPlan] { // Construct the new projections of our Project by // rewriting the original projections - val newProjects = projects.map(_.transformDown { + val newProjects = normalizedProjects.map(_.transformDown { case projectionOverSchema(expr) => expr }).map { case expr: NamedExpression => expr } @@ -152,7 +154,7 @@ object SchemaPruning extends Rule[LogicalPlan] { logDebug(s"New projects:\n${newProjects.map(_.treeString).mkString("\n")}") } - Project(newProjects, projectionChild) + Project(restoreOriginalOutputNames(newProjects, projects.map(_.name)), projectionChild) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 34101397134a4..e92382a783975 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -271,11 +271,21 @@ class ParquetFileFormat S3FileUtils.tryOpenClose(sharedConf, filePath) lazy val footerFileMetaData = ParquetFileReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData + val datetimeRebaseMode = DataSourceUtils.datetimeRebaseMode( + footerFileMetaData.getKeyValueMetaData.get, + SQLConf.get.getConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ)) // Try to push down filters when filter push-down is enabled. val pushed = if (enableParquetFilterPushDown) { val parquetSchema = footerFileMetaData.getSchema - val parquetFilters = new ParquetFilters(parquetSchema, pushDownDate, pushDownTimestamp, - pushDownDecimal, pushDownStringStartWith, pushDownInFilterThreshold, isCaseSensitive) + val parquetFilters = new ParquetFilters( + parquetSchema, + pushDownDate, + pushDownTimestamp, + pushDownDecimal, + pushDownStringStartWith, + pushDownInFilterThreshold, + isCaseSensitive, + datetimeRebaseMode) filters // Collects all converted Parquet filter predicates. Notice that not all predicates can be // converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap` @@ -301,9 +311,6 @@ class ParquetFileFormat None } - val datetimeRebaseMode = DataSourceUtils.datetimeRebaseMode( - footerFileMetaData.getKeyValueMetaData.get, - SQLConf.get.getConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ)) val int96RebaseMode = DataSourceUtils.int96RebaseMode( footerFileMetaData.getKeyValueMetaData.get, SQLConf.get.getConf(SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_READ)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index 73910c3943e9a..87fc4c16f2443 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -34,6 +34,8 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.catalyst.util.RebaseDateTime.{rebaseGregorianToJulianDays, rebaseGregorianToJulianMicros} +import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy import org.apache.spark.sql.sources import org.apache.spark.unsafe.types.UTF8String @@ -47,7 +49,8 @@ class ParquetFilters( pushDownDecimal: Boolean, pushDownStartWith: Boolean, pushDownInFilterThreshold: Int, - caseSensitive: Boolean) { + caseSensitive: Boolean, + datetimeRebaseMode: LegacyBehaviorPolicy.Value) { // A map which contains parquet field name and data type, if predicate push down applies. // // Each key in `nameToParquetField` represents a column; `dots` are used as separators for @@ -123,14 +126,26 @@ class ParquetFilters( private val ParquetTimestampMicrosType = ParquetSchemaType(TIMESTAMP_MICROS, INT64, 0, null) private val ParquetTimestampMillisType = ParquetSchemaType(TIMESTAMP_MILLIS, INT64, 0, null) - private def dateToDays(date: Any): Int = date match { - case d: Date => DateTimeUtils.fromJavaDate(d) - case ld: LocalDate => DateTimeUtils.localDateToDays(ld) + private def dateToDays(date: Any): Int = { + val gregorianDays = date match { + case d: Date => DateTimeUtils.fromJavaDate(d) + case ld: LocalDate => DateTimeUtils.localDateToDays(ld) + } + datetimeRebaseMode match { + case LegacyBehaviorPolicy.LEGACY => rebaseGregorianToJulianDays(gregorianDays) + case _ => gregorianDays + } } - private def timestampToMicros(v: Any): JLong = v match { - case i: Instant => DateTimeUtils.instantToMicros(i) - case t: Timestamp => DateTimeUtils.fromJavaTimestamp(t) + private def timestampToMicros(v: Any): JLong = { + val gregorianMicros = v match { + case i: Instant => DateTimeUtils.instantToMicros(i) + case t: Timestamp => DateTimeUtils.fromJavaTimestamp(t) + } + datetimeRebaseMode match { + case LegacyBehaviorPolicy.LEGACY => rebaseGregorianToJulianMicros(gregorianMicros) + case _ => gregorianMicros + } } private def decimalToInt32(decimal: JBigDecimal): Integer = decimal.unscaledValue().intValue() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index dca12ff6b4deb..12a71b5af349b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -550,7 +550,7 @@ private[parquet] class ParquetRowConverter( val repeatedType = parquetSchema.getType(0) val elementType = catalystSchema.elementType - // At this stage, we're not sure whether the repeated field maps to the element type or is + // At this stage, we need to figure out if the repeated field maps to the element type or is // just the syntactic repeated group of the 3-level standard LIST layout. Take the following // Parquet LIST-annotated group type as an example: // @@ -574,12 +574,20 @@ private[parquet] class ParquetRowConverter( // // ARRAY>> // + // // Here we try to convert field `list` into a Catalyst type to see whether the converted type - // matches the Catalyst array element type. If it doesn't match, then it's case 1; otherwise, - // it's case 2. + // matches the Catalyst array element type. + // + // If the guessed element type from the above does not match the Catalyst type (for example, + // in case of schema evolution), we need to check if the repeated type matches one of the + // backward-compatibility rules for legacy LIST types (see the link above). + // + // If the element type does not match the Catalyst type and the underlying repeated type + // does not belong to the legacy LIST type, then it is case 1; otherwise, it is case 2. val guessedElementType = schemaConverter.convertField(repeatedType) + val isLegacy = schemaConverter.isElementType(repeatedType, parquetSchema.getName) - if (DataType.equalsIgnoreCompatibleNullability(guessedElementType, elementType)) { + if (DataType.equalsIgnoreCompatibleNullability(guessedElementType, elementType) || isLegacy) { // If the repeated field corresponds to the element type, creates a new converter using the // type of the repeated field. newConverter(repeatedType, elementType, new ParentContainerUpdater { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala index ff804e25ede4b..d456a35b83c61 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala @@ -241,7 +241,7 @@ class ParquetToSparkSchemaConverter( // Here we implement Parquet LIST backwards-compatibility rules. // See: https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#backward-compatibility-rules // scalastyle:on - private def isElementType(repeatedType: Type, parentName: String): Boolean = { + private[parquet] def isElementType(repeatedType: Type, parentName: String): Boolean = { { // For legacy 2-level list types with primitive element type, e.g.: // diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 643607a905d7a..96e009b292d95 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -213,7 +213,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case OverwriteByExpression(r: DataSourceV2Relation, deleteExpr, query, writeOptions, _) => // fail if any filter cannot be converted. correctness depends on removing all matching data. val filters = splitConjunctivePredicates(deleteExpr).map { - filter => DataSourceStrategy.translateFilter(deleteExpr, + filter => DataSourceStrategy.translateFilter(filter, supportNestedPredicatePushdown = true).getOrElse( throw new AnalysisException(s"Cannot translate expression to source filter: $filter")) }.toArray diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala index 363dd154b5fbb..ac63725b774d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala @@ -24,8 +24,9 @@ import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging import org.apache.spark.internal.config.IO_WARNING_LARGEFILETHRESHOLD import org.apache.spark.sql.{AnalysisException, SparkSession} -import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionSet} +import org.apache.spark.sql.catalyst.expressions.{AttributeSet, Expression, ExpressionSet} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.connector.read.{Batch, InputPartition, Scan, Statistics, SupportsReportStatistics} import org.apache.spark.sql.execution.PartitionedFileUtil import org.apache.spark.sql.execution.datasources._ @@ -84,11 +85,24 @@ trait FileScan extends Scan protected def seqToString(seq: Seq[Any]): String = seq.mkString("[", ", ", "]") + private lazy val (normalizedPartitionFilters, normalizedDataFilters) = { + val output = readSchema().toAttributes + val partitionFilterAttributes = AttributeSet(partitionFilters).map(a => a.name -> a).toMap + val dataFiltersAttributes = AttributeSet(dataFilters).map(a => a.name -> a).toMap + val normalizedPartitionFilters = ExpressionSet(partitionFilters.map( + QueryPlan.normalizeExpressions(_, + output.map(a => partitionFilterAttributes.getOrElse(a.name, a))))) + val normalizedDataFilters = ExpressionSet(dataFilters.map( + QueryPlan.normalizeExpressions(_, + output.map(a => dataFiltersAttributes.getOrElse(a.name, a))))) + (normalizedPartitionFilters, normalizedDataFilters) + } + override def equals(obj: Any): Boolean = obj match { case f: FileScan => - fileIndex == f.fileIndex && readSchema == f.readSchema - ExpressionSet(partitionFilters) == ExpressionSet(f.partitionFilters) && - ExpressionSet(dataFilters) == ExpressionSet(f.dataFilters) + fileIndex == f.fileIndex && readSchema == f.readSchema && + normalizedPartitionFilters == f.normalizedPartitionFilters && + normalizedDataFilters == f.normalizedDataFilters case _ => false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala index 167ba45b888a3..1f57f17911457 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala @@ -81,6 +81,10 @@ object PushDownUtils extends PredicateHelper { relation: DataSourceV2Relation, projects: Seq[NamedExpression], filters: Seq[Expression]): (Scan, Seq[AttributeReference]) = { + val exprs = projects ++ filters + val requiredColumns = AttributeSet(exprs.flatMap(_.references)) + val neededOutput = relation.output.filter(requiredColumns.contains) + scanBuilder match { case r: SupportsPushDownRequiredColumns if SQLConf.get.nestedSchemaPruningEnabled => val rootFields = SchemaPruning.identifyRootFields(projects, filters) @@ -89,14 +93,12 @@ object PushDownUtils extends PredicateHelper { } else { new StructType() } - r.pruneColumns(prunedSchema) + val neededFieldNames = neededOutput.map(_.name).toSet + r.pruneColumns(StructType(prunedSchema.filter(f => neededFieldNames.contains(f.name)))) val scan = r.build() scan -> toOutputAttrs(scan.readSchema(), relation) case r: SupportsPushDownRequiredColumns => - val exprs = projects ++ filters - val requiredColumns = AttributeSet(exprs.flatMap(_.references)) - val neededOutput = relation.output.filter(requiredColumns.contains) r.pruneColumns(neededOutput.toStructType) val scan = r.build() // always project, in case the relation's output has been updated and doesn't match diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala index d2180566790ac..daef7571806f7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.connector.read.{Scan, V1Scan} import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.sources import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.SchemaUtils._ object V2ScanRelationPushDown extends Rule[LogicalPlan] { import DataSourceV2Implicits._ @@ -79,7 +80,7 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] { val newProjects = normalizedProjects .map(projectionFunc) .asInstanceOf[Seq[NamedExpression]] - Project(newProjects, withFilter) + Project(restoreOriginalOutputNames(newProjects, project.map(_.name)), withFilter) } else { withFilter } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala index e4d5e9b2d9f6d..7ea45c4e5a7b7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala @@ -135,11 +135,21 @@ case class ParquetPartitionReaderFactory( lazy val footerFileMetaData = ParquetFileReader.readFooter(conf, filePath, SKIP_ROW_GROUPS).getFileMetaData + val datetimeRebaseMode = DataSourceUtils.datetimeRebaseMode( + footerFileMetaData.getKeyValueMetaData.get, + SQLConf.get.getConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ)) // Try to push down filters when filter push-down is enabled. val pushed = if (enableParquetFilterPushDown) { val parquetSchema = footerFileMetaData.getSchema - val parquetFilters = new ParquetFilters(parquetSchema, pushDownDate, pushDownTimestamp, - pushDownDecimal, pushDownStringStartWith, pushDownInFilterThreshold, isCaseSensitive) + val parquetFilters = new ParquetFilters( + parquetSchema, + pushDownDate, + pushDownTimestamp, + pushDownDecimal, + pushDownStringStartWith, + pushDownInFilterThreshold, + isCaseSensitive, + datetimeRebaseMode) filters // Collects all converted Parquet filter predicates. Notice that not all predicates can be // converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap` @@ -172,9 +182,6 @@ case class ParquetPartitionReaderFactory( if (pushed.isDefined) { ParquetInputFormat.setFilterPredicate(hadoopAttemptContext.getConfiguration, pushed.get) } - val datetimeRebaseMode = DataSourceUtils.datetimeRebaseMode( - footerFileMetaData.getKeyValueMetaData.get, - SQLConf.get.getConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ)) val int96RebaseMode = DataSourceUtils.int96RebaseMode( footerFileMetaData.getKeyValueMetaData.get, SQLConf.get.getConf(SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_READ)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScanBuilder.scala index 2f861356e9499..2bf6d798358de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScanBuilder.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.connector.read.{Scan, SupportsPushDownFilters} import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.parquet.{ParquetFilters, SparkToParquetSchemaConverter} import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder +import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -51,8 +52,17 @@ case class ParquetScanBuilder( val isCaseSensitive = sqlConf.caseSensitiveAnalysis val parquetSchema = new SparkToParquetSchemaConverter(sparkSession.sessionState.conf).convert(schema) - val parquetFilters = new ParquetFilters(parquetSchema, pushDownDate, pushDownTimestamp, - pushDownDecimal, pushDownStringStartWith, pushDownInFilterThreshold, isCaseSensitive) + val parquetFilters = new ParquetFilters( + parquetSchema, + pushDownDate, + pushDownTimestamp, + pushDownDecimal, + pushDownStringStartWith, + pushDownInFilterThreshold, + isCaseSensitive, + // The rebase mode doesn't matter here because the filters are used to determine + // whether they is convertible. + LegacyBehaviorPolicy.CORRECTED) parquetFilters.convertibleFilters(this.filters).toArray } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PartitionPruning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PartitionPruning.scala index e30f9b65a2c2c..7fac91a337adc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PartitionPruning.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PartitionPruning.scala @@ -163,6 +163,7 @@ object PartitionPruning extends Rule[LogicalPlan] with PredicateHelper { case _: BinaryComparison => true case _: In | _: InSet => true case _: StringPredicate => true + case _: MultiLikeBase => true case _ => false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala index 2a9e15851e9f1..cec1286c98a7e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala @@ -46,7 +46,7 @@ case class BroadcastHashJoinExec( left: SparkPlan, right: SparkPlan, isNullAwareAntiJoin: Boolean = false) - extends HashJoin with CodegenSupport { + extends HashJoin { if (isNullAwareAntiJoin) { require(leftKeys.length == 1, "leftKeys length should be 1") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index 53bd591d98a2e..42219ee615c5f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -138,7 +138,13 @@ trait HashJoin extends BaseJoinExec with CodegenSupport { UnsafeProjection.create(streamedBoundKeys) @transient protected[this] lazy val boundCondition = if (condition.isDefined) { - Predicate.create(condition.get, streamedPlan.output ++ buildPlan.output).eval _ + if (joinType == FullOuter && buildSide == BuildLeft) { + // Put join left side before right side. This is to be consistent with + // `ShuffledHashJoinExec.fullOuterJoin`. + Predicate.create(condition.get, buildPlan.output ++ streamedPlan.output).eval _ + } else { + Predicate.create(condition.get, streamedPlan.output ++ buildPlan.output).eval _ + } } else { (r: InternalRow) => true } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index a91cc0782e1f8..fec766207b4ba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -480,6 +480,7 @@ private[joins] object UnsafeHashedRelation { // scalastyle:on throwerror } } else if (isNullAware) { + binaryMap.free() return HashedRelationWithAllNullKeys } } @@ -1064,6 +1065,7 @@ private[joins] object LongHashedRelation { val key = rowKey.getLong(0) map.append(key, unsafeRow) } else if (isNullAware) { + map.free() return HashedRelationWithAllNullKeys } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala index e6cfbb0a01e07..c62b121456e80 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala @@ -124,14 +124,18 @@ trait BaseLimitExec extends LimitExec with CodegenSupport { } protected override def doProduce(ctx: CodegenContext): String = { - child.asInstanceOf[CodegenSupport].produce(ctx, this) - } - - override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = { // The counter name is already obtained by the upstream operators via `limitNotReachedChecks`. // Here we have to inline it to not change its name. This is fine as we won't have many limit // operators in one query. + // + // Note: create counter variable here instead of `doConsume()` to avoid compilation error, + // because upstream operators might not call `doConsume()` here + // (e.g. `HashJoin.codegenInner()`). ctx.addMutableState(CodeGenerator.JAVA_INT, countTerm, forceInline = true, useFreshName = false) + child.asInstanceOf[CodegenSupport].produce(ctx, this) + } + + override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = { s""" | if ($countTerm < $limit) { | $countTerm += 1; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 6b0d33b819a20..1b145f23fc8a9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -314,26 +314,28 @@ abstract class StreamExecution( startLatch.countDown() // While active, repeatedly attempt to run batches. - SparkSession.setActiveSession(sparkSession) - - updateStatusMessage("Initializing sources") - // force initialization of the logical plan so that the sources can be created - logicalPlan - - // Adaptive execution can change num shuffle partitions, disallow - sparkSessionForStream.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false") - // Disable cost-based join optimization as we do not want stateful operations to be rearranged - sparkSessionForStream.conf.set(SQLConf.CBO_ENABLED.key, "false") - offsetSeqMetadata = OffsetSeqMetadata( - batchWatermarkMs = 0, batchTimestampMs = 0, sparkSessionForStream.conf) - - if (state.compareAndSet(INITIALIZING, ACTIVE)) { - // Unblock `awaitInitialization` - initializationLatch.countDown() - runActivatedStream(sparkSessionForStream) - updateStatusMessage("Stopped") - } else { - // `stop()` is already called. Let `finally` finish the cleanup. + sparkSessionForStream.withActive { + // Adaptive execution can change num shuffle partitions, disallow + sparkSessionForStream.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false") + // Disable cost-based join optimization as we do not want stateful operations + // to be rearranged + sparkSessionForStream.conf.set(SQLConf.CBO_ENABLED.key, "false") + + updateStatusMessage("Initializing sources") + // force initialization of the logical plan so that the sources can be created + logicalPlan + + offsetSeqMetadata = OffsetSeqMetadata( + batchWatermarkMs = 0, batchTimestampMs = 0, sparkSessionForStream.conf) + + if (state.compareAndSet(INITIALIZING, ACTIVE)) { + // Unblock `awaitInitialization` + initializationLatch.countDown() + runActivatedStream(sparkSessionForStream) + updateStatusMessage("Stopped") + } else { + // `stop()` is already called. Let `finally` finish the cleanup. + } } } catch { case e if isInterruptedByStop(e, sparkSession.sparkContext) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index e6b41cdb3eb18..6bc49b6ec0cf6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -3644,22 +3644,22 @@ object functions { } private def createLambda(f: Column => Column) = { - val x = UnresolvedNamedLambdaVariable(Seq("x")) + val x = UnresolvedNamedLambdaVariable(Seq(UnresolvedNamedLambdaVariable.freshVarName("x"))) val function = f(Column(x)).expr LambdaFunction(function, Seq(x)) } private def createLambda(f: (Column, Column) => Column) = { - val x = UnresolvedNamedLambdaVariable(Seq("x")) - val y = UnresolvedNamedLambdaVariable(Seq("y")) + val x = UnresolvedNamedLambdaVariable(Seq(UnresolvedNamedLambdaVariable.freshVarName("x"))) + val y = UnresolvedNamedLambdaVariable(Seq(UnresolvedNamedLambdaVariable.freshVarName("y"))) val function = f(Column(x), Column(y)).expr LambdaFunction(function, Seq(x, y)) } private def createLambda(f: (Column, Column, Column) => Column) = { - val x = UnresolvedNamedLambdaVariable(Seq("x")) - val y = UnresolvedNamedLambdaVariable(Seq("y")) - val z = UnresolvedNamedLambdaVariable(Seq("z")) + val x = UnresolvedNamedLambdaVariable(Seq(UnresolvedNamedLambdaVariable.freshVarName("x"))) + val y = UnresolvedNamedLambdaVariable(Seq(UnresolvedNamedLambdaVariable.freshVarName("y"))) + val z = UnresolvedNamedLambdaVariable(Seq(UnresolvedNamedLambdaVariable.freshVarName("z"))) val function = f(Column(x), Column(y), Column(z)).expr LambdaFunction(function, Seq(x, y, z)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index d82fa9e88592f..06c75791ad5c8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -64,8 +64,10 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * @since 2.0.0 */ def schema(schema: StructType): DataStreamReader = { - val replaced = CharVarcharUtils.failIfHasCharVarchar(schema).asInstanceOf[StructType] - this.userSpecifiedSchema = Option(replaced) + if (schema != null) { + val replaced = CharVarcharUtils.failIfHasCharVarchar(schema).asInstanceOf[StructType] + this.userSpecifiedSchema = Option(replaced) + } this } @@ -77,10 +79,7 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * @since 2.3.0 */ def schema(schemaString: String): DataStreamReader = { - val rawSchema = StructType.fromDDL(schemaString) - val schema = CharVarcharUtils.failIfHasCharVarchar(rawSchema).asInstanceOf[StructType] - this.userSpecifiedSchema = Option(schema) - this + schema(StructType.fromDDL(schemaString)) } /** diff --git a/sql/core/src/test/resources/sql-tests/inputs/describe.sql b/sql/core/src/test/resources/sql-tests/inputs/describe.sql index a0ee9322372ba..deff5bb7ca6fc 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/describe.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/describe.sql @@ -43,6 +43,8 @@ DESC EXTENDED t PARTITION (c='Us', d=1); DESC FORMATTED t PARTITION (c='Us', d=1); +DESC EXTENDED t PARTITION (C='Us', D=1); + -- NoSuchPartitionException: Partition not found in table DESC t PARTITION (c='Us', d=2); diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql index eabbd0a932253..81712bfac239a 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql @@ -128,3 +128,13 @@ WHERE NOT EXISTS (SELECT (SELECT max(t2b) ON t2a = t1a WHERE t2c = t3c) AND t3a = t1a); + +-- SPARK-34876: Non-nullable aggregates should not return NULL in a correlated subquery +SELECT t1a, + (SELECT count(t2d) FROM t2 WHERE t2a = t1a) count_t2, + (SELECT count_if(t2d > 0) FROM t2 WHERE t2a = t1a) count_if_t2, + (SELECT approx_count_distinct(t2d) FROM t2 WHERE t2a = t1a) approx_count_distinct_t2, + (SELECT collect_list(t2d) FROM t2 WHERE t2a = t1a) collect_list_t2, + (SELECT collect_set(t2d) FROM t2 WHERE t2a = t1a) collect_set_t2, + (SELECT hex(count_min_sketch(t2d, 0.5d, 0.5d, 1)) FROM t2 WHERE t2a = t1a) collect_set_t2 +FROM t1; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/results/describe.sql.out b/sql/core/src/test/resources/sql-tests/results/describe.sql.out index 93b0cc3fe97e1..b0fc2a4250669 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 41 +-- Number of queries: 42 -- !query @@ -324,6 +324,37 @@ Location [not included in comparison]/{warehouse_dir}/t Storage Properties [a=1, b=2] +-- !query +DESC EXTENDED t PARTITION (C='Us', D=1) +-- !query schema +struct +-- !query output +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + +# Detailed Partition Information +Database default +Table t +Partition Values [c=Us, d=1] +Location [not included in comparison]/{warehouse_dir}/t/c=Us/d=1 +Storage Properties [a=1, b=2] +Created Time [not included in comparison] +Last Access [not included in comparison] + +# Storage Information +Num Buckets 2 +Bucket Columns [`a`] +Sort Columns [`b`] +Location [not included in comparison]/{warehouse_dir}/t +Storage Properties [a=1, b=2] + + -- !query DESC t PARTITION (c='Us', d=2) -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out index 184b8daf9d28e..16570c659dc38 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 11 +-- Number of queries: 12 -- !query @@ -196,3 +196,29 @@ val1d NULL val1e 10 val1e 10 val1e 10 + + +-- !query +SELECT t1a, + (SELECT count(t2d) FROM t2 WHERE t2a = t1a) count_t2, + (SELECT count_if(t2d > 0) FROM t2 WHERE t2a = t1a) count_if_t2, + (SELECT approx_count_distinct(t2d) FROM t2 WHERE t2a = t1a) approx_count_distinct_t2, + (SELECT collect_list(t2d) FROM t2 WHERE t2a = t1a) collect_list_t2, + (SELECT collect_set(t2d) FROM t2 WHERE t2a = t1a) collect_set_t2, + (SELECT hex(count_min_sketch(t2d, 0.5d, 0.5d, 1)) FROM t2 WHERE t2a = t1a) collect_set_t2 +FROM t1 +-- !query schema +struct,collect_set_t2:array,collect_set_t2:string> +-- !query output +val1a 0 0 0 [] [] 0000000100000000000000000000000100000004000000005D8D6AB90000000000000000000000000000000000000000000000000000000000000000 +val1a 0 0 0 [] [] 0000000100000000000000000000000100000004000000005D8D6AB90000000000000000000000000000000000000000000000000000000000000000 +val1a 0 0 0 [] [] 0000000100000000000000000000000100000004000000005D8D6AB90000000000000000000000000000000000000000000000000000000000000000 +val1a 0 0 0 [] [] 0000000100000000000000000000000100000004000000005D8D6AB90000000000000000000000000000000000000000000000000000000000000000 +val1b 6 6 3 [19,119,319,19,19,19] [19,119,319] 0000000100000000000000060000000100000004000000005D8D6AB90000000000000000000000000000000400000000000000010000000000000001 +val1c 2 2 2 [219,19] [219,19] 0000000100000000000000020000000100000004000000005D8D6AB90000000000000000000000000000000100000000000000000000000000000001 +val1d 0 0 0 [] [] 0000000100000000000000000000000100000004000000005D8D6AB90000000000000000000000000000000000000000000000000000000000000000 +val1d 0 0 0 [] [] 0000000100000000000000000000000100000004000000005D8D6AB90000000000000000000000000000000000000000000000000000000000000000 +val1d 0 0 0 [] [] 0000000100000000000000000000000100000004000000005D8D6AB90000000000000000000000000000000000000000000000000000000000000000 +val1e 1 1 1 [19] [19] 0000000100000000000000010000000100000004000000005D8D6AB90000000000000000000000000000000100000000000000000000000000000000 +val1e 1 1 1 [19] [19] 0000000100000000000000010000000100000004000000005D8D6AB90000000000000000000000000000000100000000000000000000000000000000 +val1e 1 1 1 [19] [19] 0000000100000000000000010000000100000004000000005D8D6AB90000000000000000000000000000000100000000000000000000000000000000 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out index 054ee00ecc2ae..43506b49a6683 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out @@ -100,6 +100,14 @@ WHERE udf(t1.v) >= (SELECT min(udf(t2.v)) FROM t2 WHERE t2.k = t1.k) -- !query schema -struct +struct<> -- !query output -two +org.apache.spark.sql.AnalysisException +Correlated column is not allowed in predicate (CAST(udf(cast(k as string)) AS STRING) = CAST(udf(cast(outer(k#x) as string)) AS STRING)): +Aggregate [cast(udf(cast(max(cast(udf(cast(v#x as string)) as int)) as string)) as int) AS CAST(udf(cast(max(cast(udf(cast(v as string)) as int)) as string)) AS INT)#x] ++- Filter (cast(udf(cast(k#x as string)) as string) = cast(udf(cast(outer(k#x) as string)) as string)) + +- SubqueryAlias t2 + +- View (`t2`, [k#x,v#x]) + +- Project [k#x, v#x] + +- SubqueryAlias t2 + +- LocalRelation [k#x, v#x] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt index 25da173c8ecde..cfdd41a08844c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt @@ -117,7 +117,7 @@ Input [5]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_store_sk#4, ss_ticket_number#5, Output [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, hd_vehicle_count#16] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,Unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -125,7 +125,7 @@ Input [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, hd_vehicle_count#16] -Condition : ((((isnotnull(hd_vehicle_count#16) AND ((hd_buy_potential#14 = >10000) OR (hd_buy_potential#14 = Unknown))) AND (hd_vehicle_count#16 > 0)) AND (CASE WHEN (hd_vehicle_count#16 > 0) THEN (cast(hd_dep_count#15 as double) / cast(hd_vehicle_count#16 as double)) ELSE null END > 1.0)) AND isnotnull(hd_demo_sk#13)) +Condition : ((((isnotnull(hd_vehicle_count#16) AND ((hd_buy_potential#14 = >10000 ) OR (hd_buy_potential#14 = Unknown ))) AND (hd_vehicle_count#16 > 0)) AND (CASE WHEN (hd_vehicle_count#16 > 0) THEN (cast(hd_dep_count#15 as double) / cast(hd_vehicle_count#16 as double)) ELSE null END > 1.0)) AND isnotnull(hd_demo_sk#13)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#13] @@ -153,7 +153,7 @@ Results [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19] (26) Exchange Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), true, [id=#20] +Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#20] (27) HashAggregate [codegen id : 5] Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19] @@ -195,7 +195,7 @@ Input [8]: [ss_ticket_number#5, ss_customer_sk#2, cnt#22, c_customer_sk#24, c_sa (35) Exchange Input [6]: [c_last_name#27, c_first_name#26, c_salutation#25, c_preferred_cust_flag#28, ss_ticket_number#5, cnt#22] -Arguments: rangepartitioning(cnt#22 DESC NULLS LAST, 5), true, [id=#29] +Arguments: rangepartitioning(cnt#22 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [id=#29] (36) Sort [codegen id : 7] Input [6]: [c_last_name#27, c_first_name#26, c_salutation#25, c_preferred_cust_flag#28, ss_ticket_number#5, cnt#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt index b4dd8173664b6..f4dc8d6c5d6df 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt @@ -8,24 +8,24 @@ TakeOrderedAndProject (27) +- Exchange (21) +- * HashAggregate (20) +- * Project (19) - +- * SortMergeJoin Inner (18) - :- * Sort (12) - : +- Exchange (11) - : +- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.web_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.date_dim (4) - +- * Sort (17) - +- Exchange (16) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (12) + : +- * SortMergeJoin Inner (11) + : :- * Sort (5) + : : +- Exchange (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.web_sales (1) + : +- * Sort (10) + : +- Exchange (9) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.item (6) + +- BroadcastExchange (17) + +- * Project (16) +- * Filter (15) +- * ColumnarToRow (14) - +- Scan parquet default.item (13) + +- Scan parquet default.date_dim (13) (1) Scan parquet default.web_sales @@ -35,118 +35,118 @@ Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] Condition : (isnotnull(ws_item_sk#2) AND isnotnull(ws_sold_date_sk#1)) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct +(4) Exchange +Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] +Arguments: hashpartitioning(ws_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#4] -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] +(5) Sort [codegen id : 2] +Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] +Arguments: [ws_item_sk#2 ASC NULLS FIRST], false, 0 -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] -Condition : (((isnotnull(d_date#5) AND (d_date#5 >= 10644)) AND (d_date#5 <= 10674)) AND isnotnull(d_date_sk#4)) +(6) Scan parquet default.item +Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_date#5] +(7) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] +(8) Filter [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None +(9) Exchange +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: hashpartitioning(i_item_sk#5, 5), ENSURE_REQUIREMENTS, [id=#11] -(10) Project [codegen id : 2] -Output [2]: [ws_item_sk#2, ws_ext_sales_price#3] -Input [4]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3, d_date_sk#4] +(10) Sort [codegen id : 4] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5 ASC NULLS FIRST], false, 0 -(11) Exchange -Input [2]: [ws_item_sk#2, ws_ext_sales_price#3] -Arguments: hashpartitioning(ws_item_sk#2, 5), true, [id=#7] +(11) SortMergeJoin [codegen id : 6] +Left keys [1]: [ws_item_sk#2] +Right keys [1]: [i_item_sk#5] +Join condition: None -(12) Sort [codegen id : 3] -Input [2]: [ws_item_sk#2, ws_ext_sales_price#3] -Arguments: [ws_item_sk#2 ASC NULLS FIRST], false, 0 +(12) Project [codegen id : 6] +Output [7]: [ws_sold_date_sk#1, ws_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [9]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) Scan parquet default.item -Output [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(13) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_date#13] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(14) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(14) ColumnarToRow [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] -(15) Filter [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Condition : (i_category#13 IN (Sports,Books,Home) AND isnotnull(i_item_sk#8)) +(15) Filter [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) -(16) Exchange -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: hashpartitioning(i_item_sk#8, 5), true, [id=#14] +(16) Project [codegen id : 5] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_date#13] -(17) Sort [codegen id : 5] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: [i_item_sk#8 ASC NULLS FIRST], false, 0 +(17) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#2] -Right keys [1]: [i_item_sk#8] +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#1] +Right keys [1]: [d_date_sk#12] Join condition: None (19) Project [codegen id : 6] -Output [6]: [ws_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Input [8]: [ws_item_sk#2, ws_ext_sales_price#3, i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +Output [6]: [ws_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [8]: [ws_sold_date_sk#1, ws_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#12] (20) HashAggregate [codegen id : 6] -Input [6]: [ws_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [ws_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#3))] Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] (21) Exchange -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Arguments: hashpartitioning(i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, 5), true, [id=#17] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [id=#17] (22) HashAggregate [codegen id : 7] -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#3))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#3))#18] -Results [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS _w1#21, i_item_id#9] +Results [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS _w1#21, i_item_id#6] (23) Exchange -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: hashpartitioning(i_class#12, 5), true, [id=#22] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [id=#22] (24) Sort [codegen id : 8] -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: [i_class#12 ASC NULLS FIRST], false, 0 +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 (25) Window -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: [sum(_w1#21) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#12] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: [sum(_w1#21) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#9] (26) Project [codegen id : 9] -Output [7]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24, i_item_id#9] -Input [9]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9, _we0#23] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24, i_item_id#6] +Input [9]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6, _we0#23] (27) TakeOrderedAndProject -Input [7]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24, i_item_id#9] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST, i_item_desc#10 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24, i_item_id#6] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/simplified.txt index e9b94a6b4651c..2207b0fee23ce 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/simplified.txt @@ -14,34 +14,34 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (6) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - SortMergeJoin [ws_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (3) - Sort [ws_item_sk] - InputAdapter - Exchange [ws_item_sk] #3 - WholeStageCodegen (2) - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + SortMergeJoin [ws_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ws_item_sk] + InputAdapter + Exchange [ws_item_sk] #3 + WholeStageCodegen (1) Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_ext_sales_price] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #4 + WholeStageCodegen (3) + Filter [i_category,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter - WholeStageCodegen (5) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #5 - WholeStageCodegen (4) - Filter [i_category,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastExchange #5 + WholeStageCodegen (5) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt index 327e7db702faa..3dc7034581000 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt @@ -4,8 +4,8 @@ +- * HashAggregate (36) +- * Project (35) +- * BroadcastHashJoin Inner BuildRight (34) - :- * Project (29) - : +- * BroadcastHashJoin Inner BuildRight (28) + :- * Project (28) + : +- * BroadcastHashJoin Inner BuildRight (27) : :- * Project (22) : : +- * BroadcastHashJoin Inner BuildRight (21) : : :- * Project (15) @@ -27,16 +27,16 @@ : : +- * Project (19) : : +- * Filter (18) : : +- * ColumnarToRow (17) - : : +- Scan parquet default.customer_address (16) - : +- BroadcastExchange (27) - : +- * Project (26) - : +- * Filter (25) - : +- * ColumnarToRow (24) - : +- Scan parquet default.date_dim (23) + : : +- Scan parquet default.date_dim (16) + : +- BroadcastExchange (26) + : +- * Filter (25) + : +- * ColumnarToRow (24) + : +- Scan parquet default.store (23) +- BroadcastExchange (33) - +- * Filter (32) - +- * ColumnarToRow (31) - +- Scan parquet default.store (30) + +- * Project (32) + +- * Filter (31) + +- * ColumnarToRow (30) + +- Scan parquet default.customer_address (29) (1) Scan parquet default.store_sales @@ -57,7 +57,7 @@ Condition : ((((((isnotnull(ss_store_sk#5) AND isnotnull(ss_addr_sk#4)) AND isno Output [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] +PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -65,7 +65,7 @@ Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] (6) Filter [codegen id : 1] Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] -Condition : (isnotnull(cd_demo_sk#11) AND ((((cd_marital_status#12 = M) AND (cd_education_status#13 = Advanced Degree)) OR ((cd_marital_status#12 = S) AND (cd_education_status#13 = College))) OR ((cd_marital_status#12 = W) AND (cd_education_status#13 = 2 yr Degree)))) +Condition : (isnotnull(cd_demo_sk#11) AND ((((cd_marital_status#12 = M) AND (cd_education_status#13 = Advanced Degree )) OR ((cd_marital_status#12 = S) AND (cd_education_status#13 = College ))) OR ((cd_marital_status#12 = W) AND (cd_education_status#13 = 2 yr Degree )))) (7) BroadcastExchange Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] @@ -74,7 +74,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (8) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_cdemo_sk#2] Right keys [1]: [cd_demo_sk#11] -Join condition: ((((((cd_marital_status#12 = M) AND (cd_education_status#13 = Advanced Degree)) AND (ss_sales_price#7 >= 100.00)) AND (ss_sales_price#7 <= 150.00)) OR ((((cd_marital_status#12 = S) AND (cd_education_status#13 = College)) AND (ss_sales_price#7 >= 50.00)) AND (ss_sales_price#7 <= 100.00))) OR ((((cd_marital_status#12 = W) AND (cd_education_status#13 = 2 yr Degree)) AND (ss_sales_price#7 >= 150.00)) AND (ss_sales_price#7 <= 200.00))) +Join condition: ((((((cd_marital_status#12 = M) AND (cd_education_status#13 = Advanced Degree )) AND (ss_sales_price#7 >= 100.00)) AND (ss_sales_price#7 <= 150.00)) OR ((((cd_marital_status#12 = S) AND (cd_education_status#13 = College )) AND (ss_sales_price#7 >= 50.00)) AND (ss_sales_price#7 <= 100.00))) OR ((((cd_marital_status#12 = W) AND (cd_education_status#13 = 2 yr Degree )) AND (ss_sales_price#7 >= 150.00)) AND (ss_sales_price#7 <= 200.00))) (9) Project [codegen id : 6] Output [11]: [ss_sold_date_sk#1, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, cd_marital_status#12, cd_education_status#13] @@ -101,100 +101,100 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (14) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_hdemo_sk#3] Right keys [1]: [hd_demo_sk#15] -Join condition: (((((((cd_marital_status#12 = M) AND (cd_education_status#13 = Advanced Degree)) AND (ss_sales_price#7 >= 100.00)) AND (ss_sales_price#7 <= 150.00)) AND (hd_dep_count#16 = 3)) OR (((((cd_marital_status#12 = S) AND (cd_education_status#13 = College)) AND (ss_sales_price#7 >= 50.00)) AND (ss_sales_price#7 <= 100.00)) AND (hd_dep_count#16 = 1))) OR (((((cd_marital_status#12 = W) AND (cd_education_status#13 = 2 yr Degree)) AND (ss_sales_price#7 >= 150.00)) AND (ss_sales_price#7 <= 200.00)) AND (hd_dep_count#16 = 1))) +Join condition: (((((((cd_marital_status#12 = M) AND (cd_education_status#13 = Advanced Degree )) AND (ss_sales_price#7 >= 100.00)) AND (ss_sales_price#7 <= 150.00)) AND (hd_dep_count#16 = 3)) OR (((((cd_marital_status#12 = S) AND (cd_education_status#13 = College )) AND (ss_sales_price#7 >= 50.00)) AND (ss_sales_price#7 <= 100.00)) AND (hd_dep_count#16 = 1))) OR (((((cd_marital_status#12 = W) AND (cd_education_status#13 = 2 yr Degree )) AND (ss_sales_price#7 >= 150.00)) AND (ss_sales_price#7 <= 200.00)) AND (hd_dep_count#16 = 1))) (15) Project [codegen id : 6] Output [7]: [ss_sold_date_sk#1, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] Input [13]: [ss_sold_date_sk#1, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, cd_marital_status#12, cd_education_status#13, hd_demo_sk#15, hd_dep_count#16] -(16) Scan parquet default.customer_address -Output [3]: [ca_address_sk#18, ca_state#19, ca_country#20] +(16) Scan parquet default.date_dim +Output [2]: [d_date_sk#18, d_year#19] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk), Or(Or(In(ca_state, [TX,OH]),In(ca_state, [OR,NM,KY])),In(ca_state, [VA,TX,MS]))] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct (17) ColumnarToRow [codegen id : 3] -Input [3]: [ca_address_sk#18, ca_state#19, ca_country#20] +Input [2]: [d_date_sk#18, d_year#19] (18) Filter [codegen id : 3] -Input [3]: [ca_address_sk#18, ca_state#19, ca_country#20] -Condition : (((isnotnull(ca_country#20) AND (ca_country#20 = United States)) AND isnotnull(ca_address_sk#18)) AND ((ca_state#19 IN (TX,OH) OR ca_state#19 IN (OR,NM,KY)) OR ca_state#19 IN (VA,TX,MS))) +Input [2]: [d_date_sk#18, d_year#19] +Condition : ((isnotnull(d_year#19) AND (d_year#19 = 2001)) AND isnotnull(d_date_sk#18)) (19) Project [codegen id : 3] -Output [2]: [ca_address_sk#18, ca_state#19] -Input [3]: [ca_address_sk#18, ca_state#19, ca_country#20] +Output [1]: [d_date_sk#18] +Input [2]: [d_date_sk#18, d_year#19] (20) BroadcastExchange -Input [2]: [ca_address_sk#18, ca_state#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [d_date_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] (21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#4] -Right keys [1]: [ca_address_sk#18] -Join condition: ((((ca_state#19 IN (TX,OH) AND (ss_net_profit#10 >= 100.00)) AND (ss_net_profit#10 <= 200.00)) OR ((ca_state#19 IN (OR,NM,KY) AND (ss_net_profit#10 >= 150.00)) AND (ss_net_profit#10 <= 300.00))) OR ((ca_state#19 IN (VA,TX,MS) AND (ss_net_profit#10 >= 50.00)) AND (ss_net_profit#10 <= 250.00))) +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#18] +Join condition: None (22) Project [codegen id : 6] -Output [5]: [ss_sold_date_sk#1, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] -Input [9]: [ss_sold_date_sk#1, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, ca_address_sk#18, ca_state#19] +Output [6]: [ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] +Input [8]: [ss_sold_date_sk#1, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, d_date_sk#18] -(23) Scan parquet default.date_dim -Output [2]: [d_date_sk#22, d_year#23] +(23) Scan parquet default.store +Output [1]: [s_store_sk#21] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#22, d_year#23] +Input [1]: [s_store_sk#21] (25) Filter [codegen id : 4] -Input [2]: [d_date_sk#22, d_year#23] -Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(26) Project [codegen id : 4] -Output [1]: [d_date_sk#22] -Input [2]: [d_date_sk#22, d_year#23] +Input [1]: [s_store_sk#21] +Condition : isnotnull(s_store_sk#21) -(27) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +(26) BroadcastExchange +Input [1]: [s_store_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#22] +(27) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_store_sk#5] +Right keys [1]: [s_store_sk#21] Join condition: None -(29) Project [codegen id : 6] -Output [4]: [ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] -Input [6]: [ss_sold_date_sk#1, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, d_date_sk#22] +(28) Project [codegen id : 6] +Output [5]: [ss_addr_sk#4, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] +Input [7]: [ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, s_store_sk#21] -(30) Scan parquet default.store -Output [1]: [s_store_sk#25] +(29) Scan parquet default.customer_address +Output [3]: [ca_address_sk#23, ca_state#24, ca_country#25] Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk), Or(Or(In(ca_state, [TX,OH]),In(ca_state, [OR,NM,KY])),In(ca_state, [VA,TX,MS]))] +ReadSchema: struct + +(30) ColumnarToRow [codegen id : 5] +Input [3]: [ca_address_sk#23, ca_state#24, ca_country#25] -(31) ColumnarToRow [codegen id : 5] -Input [1]: [s_store_sk#25] +(31) Filter [codegen id : 5] +Input [3]: [ca_address_sk#23, ca_state#24, ca_country#25] +Condition : (((isnotnull(ca_country#25) AND (ca_country#25 = United States)) AND isnotnull(ca_address_sk#23)) AND ((ca_state#24 IN (TX,OH) OR ca_state#24 IN (OR,NM,KY)) OR ca_state#24 IN (VA,TX,MS))) -(32) Filter [codegen id : 5] -Input [1]: [s_store_sk#25] -Condition : isnotnull(s_store_sk#25) +(32) Project [codegen id : 5] +Output [2]: [ca_address_sk#23, ca_state#24] +Input [3]: [ca_address_sk#23, ca_state#24, ca_country#25] (33) BroadcastExchange -Input [1]: [s_store_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] +Input [2]: [ca_address_sk#23, ca_state#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (34) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#5] -Right keys [1]: [s_store_sk#25] -Join condition: None +Left keys [1]: [ss_addr_sk#4] +Right keys [1]: [ca_address_sk#23] +Join condition: ((((ca_state#24 IN (TX,OH) AND (ss_net_profit#10 >= 100.00)) AND (ss_net_profit#10 <= 200.00)) OR ((ca_state#24 IN (OR,NM,KY) AND (ss_net_profit#10 >= 150.00)) AND (ss_net_profit#10 <= 300.00))) OR ((ca_state#24 IN (VA,TX,MS) AND (ss_net_profit#10 >= 50.00)) AND (ss_net_profit#10 <= 250.00))) (35) Project [codegen id : 6] Output [3]: [ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] -Input [5]: [ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, s_store_sk#25] +Input [7]: [ss_addr_sk#4, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, ca_address_sk#23, ca_state#24] (36) HashAggregate [codegen id : 6] Input [3]: [ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/simplified.txt index 45d6c8f3b0bae..b457788dbd0b2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/simplified.txt @@ -5,11 +5,11 @@ WholeStageCodegen (7) WholeStageCodegen (6) HashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum,sum,count,sum,count,sum,count,sum] Project [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_store_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + Project [ss_addr_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_addr_sk,ss_store_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [ss_sold_date_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] Project [ss_sold_date_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,cd_marital_status,cd_education_status] @@ -35,23 +35,23 @@ WholeStageCodegen (7) InputAdapter BroadcastExchange #4 WholeStageCodegen (3) - Project [ca_address_sk,ca_state] - Filter [ca_country,ca_address_sk,ca_state] + Project [d_date_sk] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state,ca_country] + Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + Filter [s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_store_sk] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk] + Project [ca_address_sk,ca_state] + Filter [ca_country,ca_address_sk,ca_state] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_state,ca_country] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt index 12e95ba50cd0d..f7927aad003a2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt @@ -34,261 +34,272 @@ TakeOrderedAndProject (53) +- * Sort (46) +- Exchange (45) +- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (38) - : +- * SortMergeJoin Inner (37) - : :- * Sort (31) - : : +- Exchange (30) - : : +- * Project (29) - : : +- * Filter (28) - : : +- * ColumnarToRow (27) - : : +- Scan parquet default.customer (26) - : +- * Sort (36) - : +- Exchange (35) - : +- * Filter (34) - : +- * ColumnarToRow (33) - : +- Scan parquet default.customer_demographics (32) - +- BroadcastExchange (42) - +- * Filter (41) - +- * ColumnarToRow (40) - +- Scan parquet default.customer_address (39) + +- * SortMergeJoin Inner (43) + :- * Sort (37) + : +- Exchange (36) + : +- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (29) + : : +- * Filter (28) + : : +- * ColumnarToRow (27) + : : +- Scan parquet default.customer (26) + : +- BroadcastExchange (33) + : +- * Filter (32) + : +- * ColumnarToRow (31) + : +- Scan parquet default.customer_address (30) + +- * Sort (42) + +- Exchange (41) + +- * Filter (40) + +- * ColumnarToRow (39) + +- Scan parquet default.customer_demographics (38) (1) Scan parquet default.catalog_sales -Output [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] +Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 4] -Input [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] (3) Filter [codegen id : 4] -Input [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] -Condition : (((isnotnull(cs_bill_cdemo_sk#3) AND isnotnull(cs_bill_customer_sk#2)) AND isnotnull(cs_sold_date_sk#1)) AND isnotnull(cs_item_sk#4)) +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) (4) Scan parquet default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] +Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_education_status,Unknown), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_education_status,Unknown ), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] +Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] (6) Filter [codegen id : 1] -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] -Condition : ((((isnotnull(cd_gender#11) AND isnotnull(cd_education_status#12)) AND (cd_gender#11 = F)) AND (cd_education_status#12 = Unknown)) AND isnotnull(cd_demo_sk#10)) +Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] +Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = F)) AND (cd_education_status#13 = Unknown )) AND isnotnull(cd_demo_sk#11)) (7) Project [codegen id : 1] -Output [2]: [cd_demo_sk#10, cd_dep_count#13] -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] +Output [2]: [cd_demo_sk#11, cd_dep_count#14] +Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] (8) BroadcastExchange -Input [2]: [cd_demo_sk#10, cd_dep_count#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Input [2]: [cd_demo_sk#11, cd_dep_count#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] (9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_bill_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (10) Project [codegen id : 4] -Output [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [11]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_demo_sk#10, cd_dep_count#13] +Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] (11) Scan parquet default.date_dim -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#16, d_year#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#15, d_year#16] +Input [2]: [d_date_sk#16, d_year#17] (13) Filter [codegen id : 2] -Input [2]: [d_date_sk#15, d_year#16] -Condition : ((isnotnull(d_year#16) AND (d_year#16 = 1998)) AND isnotnull(d_date_sk#15)) +Input [2]: [d_date_sk#16, d_year#17] +Condition : ((isnotnull(d_year#17) AND (d_year#17 = 1998)) AND isnotnull(d_date_sk#16)) (14) Project [codegen id : 2] -Output [1]: [d_date_sk#15] -Input [2]: [d_date_sk#15, d_year#16] +Output [1]: [d_date_sk#16] +Input [2]: [d_date_sk#16, d_year#17] (15) BroadcastExchange -Input [1]: [d_date_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Input [1]: [d_date_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#16] Join condition: None (17) Project [codegen id : 4] -Output [8]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [10]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, d_date_sk#15] +Output [8]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14] +Input [10]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, d_date_sk#16] (18) Scan parquet default.item -Output [2]: [i_item_sk#18, i_item_id#19] +Output [2]: [i_item_sk#19, i_item_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#18, i_item_id#19] +Input [2]: [i_item_sk#19, i_item_id#20] (20) Filter [codegen id : 3] -Input [2]: [i_item_sk#18, i_item_id#19] -Condition : isnotnull(i_item_sk#18) +Input [2]: [i_item_sk#19, i_item_id#20] +Condition : isnotnull(i_item_sk#19) (21) BroadcastExchange -Input [2]: [i_item_sk#18, i_item_id#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] +Input [2]: [i_item_sk#19, i_item_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] (22) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#18] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#19] Join condition: None (23) Project [codegen id : 4] -Output [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] -Input [10]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_sk#18, i_item_id#19] +Output [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20] +Input [10]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_sk#19, i_item_id#20] (24) Exchange -Input [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] -Arguments: hashpartitioning(cs_bill_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#22] (25) Sort [codegen id : 5] -Input [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] -Arguments: [cs_bill_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20] +Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (26) Scan parquet default.customer -Output [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [1,6,8,9,12,2]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 6] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +(27) ColumnarToRow [codegen id : 7] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] -(28) Filter [codegen id : 6] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] -Condition : (((c_birth_month#25 IN (1,6,8,9,12,2) AND isnotnull(c_customer_sk#22)) AND isnotnull(c_current_cdemo_sk#23)) AND isnotnull(c_current_addr_sk#24)) +(28) Filter [codegen id : 7] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Condition : (((c_birth_month#26 IN (1,6,8,9,12,2) AND isnotnull(c_customer_sk#23)) AND isnotnull(c_current_cdemo_sk#24)) AND isnotnull(c_current_addr_sk#25)) -(29) Project [codegen id : 6] -Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +(29) Project [codegen id : 7] +Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] -(30) Exchange -Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] -Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), ENSURE_REQUIREMENTS, [id=#27] - -(31) Sort [codegen id : 7] -Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] -Arguments: [c_current_cdemo_sk#23 ASC NULLS FIRST], false, 0 - -(32) Scan parquet default.customer_demographics -Output [1]: [cd_demo_sk#28] +(30) Scan parquet default.customer_address +Output [4]: [ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(33) ColumnarToRow [codegen id : 8] -Input [1]: [cd_demo_sk#28] +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [In(ca_state, [MS,IN,ND,OK,NM,VA]), IsNotNull(ca_address_sk)] +ReadSchema: struct -(34) Filter [codegen id : 8] -Input [1]: [cd_demo_sk#28] -Condition : isnotnull(cd_demo_sk#28) +(31) ColumnarToRow [codegen id : 6] +Input [4]: [ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] -(35) Exchange -Input [1]: [cd_demo_sk#28] -Arguments: hashpartitioning(cd_demo_sk#28, 5), ENSURE_REQUIREMENTS, [id=#29] +(32) Filter [codegen id : 6] +Input [4]: [ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] +Condition : (ca_state#30 IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#28)) -(36) Sort [codegen id : 9] -Input [1]: [cd_demo_sk#28] -Arguments: [cd_demo_sk#28 ASC NULLS FIRST], false, 0 +(33) BroadcastExchange +Input [4]: [ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] -(37) SortMergeJoin [codegen id : 11] -Left keys [1]: [c_current_cdemo_sk#23] -Right keys [1]: [cd_demo_sk#28] +(34) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_addr_sk#25] +Right keys [1]: [ca_address_sk#28] Join condition: None -(38) Project [codegen id : 11] -Output [3]: [c_customer_sk#22, c_current_addr_sk#24, c_birth_year#26] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, cd_demo_sk#28] +(35) Project [codegen id : 7] +Output [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Input [8]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27, ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] -(39) Scan parquet default.customer_address -Output [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +(36) Exchange +Input [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Arguments: hashpartitioning(c_current_cdemo_sk#24, 5), ENSURE_REQUIREMENTS, [id=#33] + +(37) Sort [codegen id : 8] +Input [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Arguments: [c_current_cdemo_sk#24 ASC NULLS FIRST], false, 0 + +(38) Scan parquet default.customer_demographics +Output [1]: [cd_demo_sk#34] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_state, [MS,IN,ND,OK,NM,VA]), IsNotNull(ca_address_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk)] +ReadSchema: struct + +(39) ColumnarToRow [codegen id : 9] +Input [1]: [cd_demo_sk#34] -(40) ColumnarToRow [codegen id : 10] -Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +(40) Filter [codegen id : 9] +Input [1]: [cd_demo_sk#34] +Condition : isnotnull(cd_demo_sk#34) -(41) Filter [codegen id : 10] -Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] -Condition : (ca_state#32 IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#30)) +(41) Exchange +Input [1]: [cd_demo_sk#34] +Arguments: hashpartitioning(cd_demo_sk#34, 5), ENSURE_REQUIREMENTS, [id=#35] -(42) BroadcastExchange -Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +(42) Sort [codegen id : 10] +Input [1]: [cd_demo_sk#34] +Arguments: [cd_demo_sk#34 ASC NULLS FIRST], false, 0 -(43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#30] +(43) SortMergeJoin [codegen id : 11] +Left keys [1]: [c_current_cdemo_sk#24] +Right keys [1]: [cd_demo_sk#34] Join condition: None (44) Project [codegen id : 11] -Output [5]: [c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] -Input [7]: [c_customer_sk#22, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +Output [5]: [c_customer_sk#23, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Input [7]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31, cd_demo_sk#34] (45) Exchange -Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] -Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [5]: [c_customer_sk#23, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, [id=#36] (46) Sort [codegen id : 12] -Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] -Arguments: [c_customer_sk#22 ASC NULLS FIRST], false, 0 +Input [5]: [c_customer_sk#23, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Arguments: [c_customer_sk#23 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 13] -Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#22] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#23] Join condition: None (48) Project [codegen id : 13] -Output [11]: [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#33, ca_state#32, ca_county#31] -Input [13]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] +Output [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#20, ca_country#31, ca_state#30, ca_county#29] +Input [13]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20, c_customer_sk#23, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] (49) Expand [codegen id : 13] -Input [11]: [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#33, ca_state#32, ca_county#31] -Arguments: [List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#33, ca_state#32, ca_county#31, 0), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#33, ca_state#32, null, 1), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#33, null, null, 3), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, null, null, null, 7), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, null, null, null, null, 15)], [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40] +Input [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#20, ca_country#31, ca_state#30, ca_county#29] +Arguments: [List(cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#20, ca_country#31, ca_state#30, ca_county#29, 0), List(cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#20, ca_country#31, ca_state#30, null, 1), List(cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#20, ca_country#31, null, null, 3), List(cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#20, null, null, null, 7), List(cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, null, null, null, null, 15)], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#37, ca_country#38, ca_state#39, ca_county#40, spark_grouping_id#41] (50) HashAggregate [codegen id : 13] -Input [12]: [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40] -Keys [5]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40] -Functions [7]: [partial_avg(cast(cs_quantity#5 as decimal(12,2))), partial_avg(cast(cs_list_price#6 as decimal(12,2))), partial_avg(cast(cs_coupon_amt#8 as decimal(12,2))), partial_avg(cast(cs_sales_price#7 as decimal(12,2))), partial_avg(cast(cs_net_profit#9 as decimal(12,2))), partial_avg(cast(c_birth_year#26 as decimal(12,2))), partial_avg(cast(cd_dep_count#13 as decimal(12,2)))] -Aggregate Attributes [14]: [sum#41, count#42, sum#43, count#44, sum#45, count#46, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54] -Results [19]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] +Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#37, ca_country#38, ca_state#39, ca_county#40, spark_grouping_id#41] +Keys [5]: [i_item_id#37, ca_country#38, ca_state#39, ca_county#40, spark_grouping_id#41] +Functions [7]: [partial_avg(cast(cs_quantity#4 as decimal(12,2))), partial_avg(cast(cs_list_price#5 as decimal(12,2))), partial_avg(cast(cs_coupon_amt#7 as decimal(12,2))), partial_avg(cast(cs_sales_price#6 as decimal(12,2))), partial_avg(cast(cs_net_profit#8 as decimal(12,2))), partial_avg(cast(c_birth_year#27 as decimal(12,2))), partial_avg(cast(cd_dep_count#14 as decimal(12,2)))] +Aggregate Attributes [14]: [sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] +Results [19]: [i_item_id#37, ca_country#38, ca_state#39, ca_county#40, spark_grouping_id#41, sum#56, count#57, sum#58, count#59, sum#60, count#61, sum#62, count#63, sum#64, count#65, sum#66, count#67, sum#68, count#69] (51) Exchange -Input [19]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] -Arguments: hashpartitioning(i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [19]: [i_item_id#37, ca_country#38, ca_state#39, ca_county#40, spark_grouping_id#41, sum#56, count#57, sum#58, count#59, sum#60, count#61, sum#62, count#63, sum#64, count#65, sum#66, count#67, sum#68, count#69] +Arguments: hashpartitioning(i_item_id#37, ca_country#38, ca_state#39, ca_county#40, spark_grouping_id#41, 5), ENSURE_REQUIREMENTS, [id=#70] (52) HashAggregate [codegen id : 14] -Input [19]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] -Keys [5]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40] -Functions [7]: [avg(cast(cs_quantity#5 as decimal(12,2))), avg(cast(cs_list_price#6 as decimal(12,2))), avg(cast(cs_coupon_amt#8 as decimal(12,2))), avg(cast(cs_sales_price#7 as decimal(12,2))), avg(cast(cs_net_profit#9 as decimal(12,2))), avg(cast(c_birth_year#26 as decimal(12,2))), avg(cast(cd_dep_count#13 as decimal(12,2)))] -Aggregate Attributes [7]: [avg(cast(cs_quantity#5 as decimal(12,2)))#70, avg(cast(cs_list_price#6 as decimal(12,2)))#71, avg(cast(cs_coupon_amt#8 as decimal(12,2)))#72, avg(cast(cs_sales_price#7 as decimal(12,2)))#73, avg(cast(cs_net_profit#9 as decimal(12,2)))#74, avg(cast(c_birth_year#26 as decimal(12,2)))#75, avg(cast(cd_dep_count#13 as decimal(12,2)))#76] -Results [11]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, avg(cast(cs_quantity#5 as decimal(12,2)))#70 AS agg1#77, avg(cast(cs_list_price#6 as decimal(12,2)))#71 AS agg2#78, avg(cast(cs_coupon_amt#8 as decimal(12,2)))#72 AS agg3#79, avg(cast(cs_sales_price#7 as decimal(12,2)))#73 AS agg4#80, avg(cast(cs_net_profit#9 as decimal(12,2)))#74 AS agg5#81, avg(cast(c_birth_year#26 as decimal(12,2)))#75 AS agg6#82, avg(cast(cd_dep_count#13 as decimal(12,2)))#76 AS agg7#83] +Input [19]: [i_item_id#37, ca_country#38, ca_state#39, ca_county#40, spark_grouping_id#41, sum#56, count#57, sum#58, count#59, sum#60, count#61, sum#62, count#63, sum#64, count#65, sum#66, count#67, sum#68, count#69] +Keys [5]: [i_item_id#37, ca_country#38, ca_state#39, ca_county#40, spark_grouping_id#41] +Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_price#5 as decimal(12,2))), avg(cast(cs_coupon_amt#7 as decimal(12,2))), avg(cast(cs_sales_price#6 as decimal(12,2))), avg(cast(cs_net_profit#8 as decimal(12,2))), avg(cast(c_birth_year#27 as decimal(12,2))), avg(cast(cd_dep_count#14 as decimal(12,2)))] +Aggregate Attributes [7]: [avg(cast(cs_quantity#4 as decimal(12,2)))#71, avg(cast(cs_list_price#5 as decimal(12,2)))#72, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#73, avg(cast(cs_sales_price#6 as decimal(12,2)))#74, avg(cast(cs_net_profit#8 as decimal(12,2)))#75, avg(cast(c_birth_year#27 as decimal(12,2)))#76, avg(cast(cd_dep_count#14 as decimal(12,2)))#77] +Results [11]: [i_item_id#37, ca_country#38, ca_state#39, ca_county#40, avg(cast(cs_quantity#4 as decimal(12,2)))#71 AS agg1#78, avg(cast(cs_list_price#5 as decimal(12,2)))#72 AS agg2#79, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#73 AS agg3#80, avg(cast(cs_sales_price#6 as decimal(12,2)))#74 AS agg4#81, avg(cast(cs_net_profit#8 as decimal(12,2)))#75 AS agg5#82, avg(cast(c_birth_year#27 as decimal(12,2)))#76 AS agg6#83, avg(cast(cd_dep_count#14 as decimal(12,2)))#77 AS agg7#84] (53) TakeOrderedAndProject -Input [11]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, agg1#77, agg2#78, agg3#79, agg4#80, agg5#81, agg6#82, agg7#83] -Arguments: 100, [ca_country#37 ASC NULLS FIRST, ca_state#38 ASC NULLS FIRST, ca_county#39 ASC NULLS FIRST, i_item_id#36 ASC NULLS FIRST], [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, agg1#77, agg2#78, agg3#79, agg4#80, agg5#81, agg6#82, agg7#83] +Input [11]: [i_item_id#37, ca_country#38, ca_state#39, ca_county#40, agg1#78, agg2#79, agg3#80, agg4#81, agg5#82, agg6#83, agg7#84] +Arguments: 100, [ca_country#38 ASC NULLS FIRST, ca_state#39 ASC NULLS FIRST, ca_county#40 ASC NULLS FIRST, i_item_id#37 ASC NULLS FIRST], [i_item_id#37, ca_country#38, ca_state#39, ca_county#40, agg1#78, agg2#79, agg3#80, agg4#81, agg5#82, agg6#83, agg7#84] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +ReusedExchange (54) + + +(54) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#16] + diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/simplified.txt index 8069d43c3451a..8c76e7cab3310 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/simplified.txt @@ -54,34 +54,34 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag Exchange [c_customer_sk] #6 WholeStageCodegen (11) Project [c_customer_sk,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_customer_sk,c_current_addr_sk,c_birth_year] - SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] - InputAdapter - WholeStageCodegen (7) - Sort [c_current_cdemo_sk] - InputAdapter - Exchange [c_current_cdemo_sk] #7 - WholeStageCodegen (6) + SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] + InputAdapter + WholeStageCodegen (8) + Sort [c_current_cdemo_sk] + InputAdapter + Exchange [c_current_cdemo_sk] #7 + WholeStageCodegen (7) + Project [c_customer_sk,c_current_cdemo_sk,c_birth_year,ca_county,ca_state,ca_country] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Project [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] Filter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - InputAdapter - WholeStageCodegen (9) - Sort [cd_demo_sk] - InputAdapter - Exchange [cd_demo_sk] #8 - WholeStageCodegen (8) - Filter [cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (6) + Filter [ca_state,ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (10) - Filter [ca_state,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + WholeStageCodegen (10) + Sort [cd_demo_sk] + InputAdapter + Exchange [cd_demo_sk] #9 + WholeStageCodegen (9) + Filter [cd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_demographics [cd_demo_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt index 4627bc19f25f0..89dfa65b4aa37 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt @@ -4,248 +4,248 @@ TakeOrderedAndProject (45) +- Exchange (43) +- * HashAggregate (42) +- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (34) - : +- * SortMergeJoin Inner (33) - : :- * Sort (18) - : : +- Exchange (17) - : : +- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : :- BroadcastExchange (5) - : : : : +- * Project (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.date_dim (1) - : : : +- * Filter (8) - : : : +- * ColumnarToRow (7) - : : : +- Scan parquet default.store_sales (6) - : : +- BroadcastExchange (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.store (11) - : +- * Sort (32) - : +- Exchange (31) - : +- * Project (30) - : +- * SortMergeJoin Inner (29) - : :- * Sort (23) - : : +- Exchange (22) - : : +- * Filter (21) - : : +- * ColumnarToRow (20) - : : +- Scan parquet default.customer (19) - : +- * Sort (28) - : +- Exchange (27) - : +- * Filter (26) - : +- * ColumnarToRow (25) - : +- Scan parquet default.customer_address (24) - +- BroadcastExchange (39) - +- * Project (38) - +- * Filter (37) - +- * ColumnarToRow (36) - +- Scan parquet default.item (35) - - -(1) Scan parquet default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] + +- * SortMergeJoin Inner (40) + :- * Sort (25) + : +- Exchange (24) + : +- * Project (23) + : +- * BroadcastHashJoin Inner BuildRight (22) + : :- * Project (17) + : : +- * BroadcastHashJoin Inner BuildRight (16) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.item (4) + : : +- BroadcastExchange (15) + : : +- * Project (14) + : : +- * Filter (13) + : : +- * ColumnarToRow (12) + : : +- Scan parquet default.date_dim (11) + : +- BroadcastExchange (21) + : +- * Filter (20) + : +- * ColumnarToRow (19) + : +- Scan parquet default.store (18) + +- * Sort (39) + +- Exchange (38) + +- * Project (37) + +- * SortMergeJoin Inner (36) + :- * Sort (30) + : +- Exchange (29) + : +- * Filter (28) + : +- * ColumnarToRow (27) + : +- Scan parquet default.customer (26) + +- * Sort (35) + +- Exchange (34) + +- * Filter (33) + +- * ColumnarToRow (32) + +- Scan parquet default.customer_address (31) + + +(1) Scan parquet default.store_sales +Output [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] +ReadSchema: struct + +(2) ColumnarToRow [codegen id : 4] +Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5] + +(3) Filter [codegen id : 4] +Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5] +Condition : (((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_item_sk#2)) AND isnotnull(ss_customer_sk#3)) AND isnotnull(ss_store_sk#4)) + +(4) Scan parquet default.item +Output [6]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, i_manager_id#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,8), IsNotNull(i_item_sk)] +ReadSchema: struct + +(5) ColumnarToRow [codegen id : 1] +Input [6]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, i_manager_id#11] + +(6) Filter [codegen id : 1] +Input [6]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, i_manager_id#11] +Condition : ((isnotnull(i_manager_id#11) AND (i_manager_id#11 = 8)) AND isnotnull(i_item_sk#6)) -(2) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +(7) Project [codegen id : 1] +Output [5]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] +Input [6]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, i_manager_id#11] -(3) Filter [codegen id : 1] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1998)) AND isnotnull(d_date_sk#1)) +(8) BroadcastExchange +Input [5]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] -(4) Project [codegen id : 1] -Output [1]: [d_date_sk#1] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +(9) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_item_sk#2] +Right keys [1]: [i_item_sk#6] +Join condition: None -(5) BroadcastExchange -Input [1]: [d_date_sk#1] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#4] +(10) Project [codegen id : 4] +Output [8]: [ss_sold_date_sk#1, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] +Input [10]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] -(6) Scan parquet default.store_sales -Output [5]: [ss_sold_date_sk#5, ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9] +(11) Scan parquet default.date_dim +Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1998), IsNotNull(d_date_sk)] +ReadSchema: struct + +(12) ColumnarToRow [codegen id : 2] +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] + +(13) Filter [codegen id : 2] +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] +Condition : ((((isnotnull(d_moy#15) AND isnotnull(d_year#14)) AND (d_moy#15 = 11)) AND (d_year#14 = 1998)) AND isnotnull(d_date_sk#13)) -(7) ColumnarToRow -Input [5]: [ss_sold_date_sk#5, ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9] +(14) Project [codegen id : 2] +Output [1]: [d_date_sk#13] +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -(8) Filter -Input [5]: [ss_sold_date_sk#5, ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9] -Condition : (((isnotnull(ss_sold_date_sk#5) AND isnotnull(ss_item_sk#6)) AND isnotnull(ss_customer_sk#7)) AND isnotnull(ss_store_sk#8)) +(15) BroadcastExchange +Input [1]: [d_date_sk#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#5] +(16) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#13] Join condition: None -(10) Project [codegen id : 3] -Output [4]: [ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9] -Input [6]: [d_date_sk#1, ss_sold_date_sk#5, ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9] +(17) Project [codegen id : 4] +Output [7]: [ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] +Input [9]: [ss_sold_date_sk#1, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, d_date_sk#13] -(11) Scan parquet default.store -Output [2]: [s_store_sk#10, s_zip#11] +(18) Scan parquet default.store +Output [2]: [s_store_sk#17, s_zip#18] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_zip), IsNotNull(s_store_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#10, s_zip#11] +(19) ColumnarToRow [codegen id : 3] +Input [2]: [s_store_sk#17, s_zip#18] -(13) Filter [codegen id : 2] -Input [2]: [s_store_sk#10, s_zip#11] -Condition : (isnotnull(s_zip#11) AND isnotnull(s_store_sk#10)) +(20) Filter [codegen id : 3] +Input [2]: [s_store_sk#17, s_zip#18] +Condition : (isnotnull(s_zip#18) AND isnotnull(s_store_sk#17)) -(14) BroadcastExchange -Input [2]: [s_store_sk#10, s_zip#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#12] +(21) BroadcastExchange +Input [2]: [s_store_sk#17, s_zip#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#19] -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_store_sk#8] -Right keys [1]: [s_store_sk#10] +(22) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#4] +Right keys [1]: [s_store_sk#17] Join condition: None -(16) Project [codegen id : 3] -Output [4]: [ss_item_sk#6, ss_customer_sk#7, ss_ext_sales_price#9, s_zip#11] -Input [6]: [ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9, s_store_sk#10, s_zip#11] +(23) Project [codegen id : 4] +Output [7]: [ss_customer_sk#3, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_zip#18] +Input [9]: [ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_store_sk#17, s_zip#18] -(17) Exchange -Input [4]: [ss_item_sk#6, ss_customer_sk#7, ss_ext_sales_price#9, s_zip#11] -Arguments: hashpartitioning(ss_customer_sk#7, 5), ENSURE_REQUIREMENTS, [id=#13] +(24) Exchange +Input [7]: [ss_customer_sk#3, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_zip#18] +Arguments: hashpartitioning(ss_customer_sk#3, 5), ENSURE_REQUIREMENTS, [id=#20] -(18) Sort [codegen id : 4] -Input [4]: [ss_item_sk#6, ss_customer_sk#7, ss_ext_sales_price#9, s_zip#11] -Arguments: [ss_customer_sk#7 ASC NULLS FIRST], false, 0 +(25) Sort [codegen id : 5] +Input [7]: [ss_customer_sk#3, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_zip#18] +Arguments: [ss_customer_sk#3 ASC NULLS FIRST], false, 0 -(19) Scan parquet default.customer -Output [2]: [c_customer_sk#14, c_current_addr_sk#15] +(26) Scan parquet default.customer +Output [2]: [c_customer_sk#21, c_current_addr_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 5] -Input [2]: [c_customer_sk#14, c_current_addr_sk#15] +(27) ColumnarToRow [codegen id : 6] +Input [2]: [c_customer_sk#21, c_current_addr_sk#22] -(21) Filter [codegen id : 5] -Input [2]: [c_customer_sk#14, c_current_addr_sk#15] -Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_current_addr_sk#15)) +(28) Filter [codegen id : 6] +Input [2]: [c_customer_sk#21, c_current_addr_sk#22] +Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_current_addr_sk#22)) -(22) Exchange -Input [2]: [c_customer_sk#14, c_current_addr_sk#15] -Arguments: hashpartitioning(c_current_addr_sk#15, 5), ENSURE_REQUIREMENTS, [id=#16] +(29) Exchange +Input [2]: [c_customer_sk#21, c_current_addr_sk#22] +Arguments: hashpartitioning(c_current_addr_sk#22, 5), ENSURE_REQUIREMENTS, [id=#23] -(23) Sort [codegen id : 6] -Input [2]: [c_customer_sk#14, c_current_addr_sk#15] -Arguments: [c_current_addr_sk#15 ASC NULLS FIRST], false, 0 +(30) Sort [codegen id : 7] +Input [2]: [c_customer_sk#21, c_current_addr_sk#22] +Arguments: [c_current_addr_sk#22 ASC NULLS FIRST], false, 0 -(24) Scan parquet default.customer_address -Output [2]: [ca_address_sk#17, ca_zip#18] +(31) Scan parquet default.customer_address +Output [2]: [ca_address_sk#24, ca_zip#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_zip)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#17, ca_zip#18] +(32) ColumnarToRow [codegen id : 8] +Input [2]: [ca_address_sk#24, ca_zip#25] -(26) Filter [codegen id : 7] -Input [2]: [ca_address_sk#17, ca_zip#18] -Condition : (isnotnull(ca_address_sk#17) AND isnotnull(ca_zip#18)) +(33) Filter [codegen id : 8] +Input [2]: [ca_address_sk#24, ca_zip#25] +Condition : (isnotnull(ca_address_sk#24) AND isnotnull(ca_zip#25)) -(27) Exchange -Input [2]: [ca_address_sk#17, ca_zip#18] -Arguments: hashpartitioning(ca_address_sk#17, 5), ENSURE_REQUIREMENTS, [id=#19] +(34) Exchange +Input [2]: [ca_address_sk#24, ca_zip#25] +Arguments: hashpartitioning(ca_address_sk#24, 5), ENSURE_REQUIREMENTS, [id=#26] -(28) Sort [codegen id : 8] -Input [2]: [ca_address_sk#17, ca_zip#18] -Arguments: [ca_address_sk#17 ASC NULLS FIRST], false, 0 +(35) Sort [codegen id : 9] +Input [2]: [ca_address_sk#24, ca_zip#25] +Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 -(29) SortMergeJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#15] -Right keys [1]: [ca_address_sk#17] +(36) SortMergeJoin [codegen id : 10] +Left keys [1]: [c_current_addr_sk#22] +Right keys [1]: [ca_address_sk#24] Join condition: None -(30) Project [codegen id : 9] -Output [2]: [c_customer_sk#14, ca_zip#18] -Input [4]: [c_customer_sk#14, c_current_addr_sk#15, ca_address_sk#17, ca_zip#18] - -(31) Exchange -Input [2]: [c_customer_sk#14, ca_zip#18] -Arguments: hashpartitioning(c_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#20] - -(32) Sort [codegen id : 10] -Input [2]: [c_customer_sk#14, ca_zip#18] -Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 +(37) Project [codegen id : 10] +Output [2]: [c_customer_sk#21, ca_zip#25] +Input [4]: [c_customer_sk#21, c_current_addr_sk#22, ca_address_sk#24, ca_zip#25] -(33) SortMergeJoin [codegen id : 12] -Left keys [1]: [ss_customer_sk#7] -Right keys [1]: [c_customer_sk#14] -Join condition: NOT (substr(ca_zip#18, 1, 5) = substr(s_zip#11, 1, 5)) +(38) Exchange +Input [2]: [c_customer_sk#21, ca_zip#25] +Arguments: hashpartitioning(c_customer_sk#21, 5), ENSURE_REQUIREMENTS, [id=#27] -(34) Project [codegen id : 12] -Output [2]: [ss_item_sk#6, ss_ext_sales_price#9] -Input [6]: [ss_item_sk#6, ss_customer_sk#7, ss_ext_sales_price#9, s_zip#11, c_customer_sk#14, ca_zip#18] +(39) Sort [codegen id : 11] +Input [2]: [c_customer_sk#21, ca_zip#25] +Arguments: [c_customer_sk#21 ASC NULLS FIRST], false, 0 -(35) Scan parquet default.item -Output [6]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25, i_manager_id#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,8), IsNotNull(i_item_sk)] -ReadSchema: struct - -(36) ColumnarToRow [codegen id : 11] -Input [6]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25, i_manager_id#26] - -(37) Filter [codegen id : 11] -Input [6]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25, i_manager_id#26] -Condition : ((isnotnull(i_manager_id#26) AND (i_manager_id#26 = 8)) AND isnotnull(i_item_sk#21)) - -(38) Project [codegen id : 11] -Output [5]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25] -Input [6]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25, i_manager_id#26] - -(39) BroadcastExchange -Input [5]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] - -(40) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [ss_item_sk#6] -Right keys [1]: [i_item_sk#21] -Join condition: None +(40) SortMergeJoin [codegen id : 12] +Left keys [1]: [ss_customer_sk#3] +Right keys [1]: [c_customer_sk#21] +Join condition: NOT (substr(ca_zip#25, 1, 5) = substr(s_zip#18, 1, 5)) (41) Project [codegen id : 12] -Output [5]: [ss_ext_sales_price#9, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25] -Input [7]: [ss_item_sk#6, ss_ext_sales_price#9, i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25] +Output [5]: [ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] +Input [9]: [ss_customer_sk#3, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_zip#18, c_customer_sk#21, ca_zip#25] (42) HashAggregate [codegen id : 12] -Input [5]: [ss_ext_sales_price#9, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25] -Keys [4]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#9))] +Input [5]: [ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] +Keys [4]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum#28] -Results [5]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25, sum#29] +Results [5]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10, sum#29] (43) Exchange -Input [5]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25, sum#29] -Arguments: hashpartitioning(i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [5]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10, sum#29] +Arguments: hashpartitioning(i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10, 5), ENSURE_REQUIREMENTS, [id=#30] (44) HashAggregate [codegen id : 13] -Input [5]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25, sum#29] -Keys [4]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#9))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#9))#31] -Results [5]: [i_brand_id#22 AS brand_id#32, i_brand#23 AS brand#33, i_manufact_id#24, i_manufact#25, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#9))#31,17,2) AS ext_price#34] +Input [5]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10, sum#29] +Keys [4]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#31] +Results [5]: [i_brand_id#7 AS brand_id#32, i_brand#8 AS brand#33, i_manufact_id#9, i_manufact#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#31,17,2) AS ext_price#34] (45) TakeOrderedAndProject -Input [5]: [brand_id#32, brand#33, i_manufact_id#24, i_manufact#25, ext_price#34] -Arguments: 100, [ext_price#34 DESC NULLS LAST, brand#33 ASC NULLS FIRST, brand_id#32 ASC NULLS FIRST, i_manufact_id#24 ASC NULLS FIRST, i_manufact#25 ASC NULLS FIRST], [brand_id#32, brand#33, i_manufact_id#24, i_manufact#25, ext_price#34] +Input [5]: [brand_id#32, brand#33, i_manufact_id#9, i_manufact#10, ext_price#34] +Arguments: 100, [ext_price#34 DESC NULLS LAST, brand#33 ASC NULLS FIRST, brand_id#32 ASC NULLS FIRST, i_manufact_id#9 ASC NULLS FIRST, i_manufact#10 ASC NULLS FIRST], [brand_id#32, brand#33, i_manufact_id#9, i_manufact#10, ext_price#34] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt index b6441c5fe72c1..05fa3f82e27df 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt @@ -6,71 +6,71 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] WholeStageCodegen (12) HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - SortMergeJoin [ss_customer_sk,c_customer_sk,ca_zip,s_zip] - InputAdapter - WholeStageCodegen (4) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #2 - WholeStageCodegen (3) - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk,ca_zip,s_zip] + InputAdapter + WholeStageCodegen (5) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #2 + WholeStageCodegen (4) + Project [ss_customer_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] + Project [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - Filter [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) - Filter [s_zip,s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_zip] - InputAdapter - WholeStageCodegen (10) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #5 - WholeStageCodegen (9) - Project [c_customer_sk,ca_zip] - SortMergeJoin [c_current_addr_sk,ca_address_sk] - InputAdapter - WholeStageCodegen (6) - Sort [c_current_addr_sk] - InputAdapter - Exchange [c_current_addr_sk] #6 - WholeStageCodegen (5) - Filter [c_customer_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - WholeStageCodegen (8) - Sort [ca_address_sk] + Project [d_date_sk] + Filter [d_moy,d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + Filter [s_zip,s_store_sk] + ColumnarToRow InputAdapter - Exchange [ca_address_sk] #7 - WholeStageCodegen (7) - Filter [ca_address_sk,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_zip] + Scan parquet default.store [s_store_sk,s_zip] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - Project [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - Filter [i_manager_id,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] + WholeStageCodegen (11) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #6 + WholeStageCodegen (10) + Project [c_customer_sk,ca_zip] + SortMergeJoin [c_current_addr_sk,ca_address_sk] + InputAdapter + WholeStageCodegen (7) + Sort [c_current_addr_sk] + InputAdapter + Exchange [c_current_addr_sk] #7 + WholeStageCodegen (6) + Filter [c_customer_sk,c_current_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] + InputAdapter + WholeStageCodegen (9) + Sort [ca_address_sk] + InputAdapter + Exchange [ca_address_sk] #8 + WholeStageCodegen (8) + Filter [ca_address_sk,ca_zip] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_zip] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt index 2ef3660bc7ba4..a5b9623cbd96e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt @@ -8,24 +8,24 @@ TakeOrderedAndProject (27) +- Exchange (21) +- * HashAggregate (20) +- * Project (19) - +- * SortMergeJoin Inner (18) - :- * Sort (12) - : +- Exchange (11) - : +- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.catalog_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.date_dim (4) - +- * Sort (17) - +- Exchange (16) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (12) + : +- * SortMergeJoin Inner (11) + : :- * Sort (5) + : : +- Exchange (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.catalog_sales (1) + : +- * Sort (10) + : +- Exchange (9) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.item (6) + +- BroadcastExchange (17) + +- * Project (16) +- * Filter (15) +- * ColumnarToRow (14) - +- Scan parquet default.item (13) + +- Scan parquet default.date_dim (13) (1) Scan parquet default.catalog_sales @@ -35,118 +35,118 @@ Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_sold_date_sk#1)) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct +(4) Exchange +Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] +Arguments: hashpartitioning(cs_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#4] -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] +(5) Sort [codegen id : 2] +Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] +Arguments: [cs_item_sk#2 ASC NULLS FIRST], false, 0 -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] -Condition : (((isnotnull(d_date#5) AND (d_date#5 >= 10644)) AND (d_date#5 <= 10674)) AND isnotnull(d_date_sk#4)) +(6) Scan parquet default.item +Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_date#5] +(7) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] +(8) Filter [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None +(9) Exchange +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: hashpartitioning(i_item_sk#5, 5), ENSURE_REQUIREMENTS, [id=#11] -(10) Project [codegen id : 2] -Output [2]: [cs_item_sk#2, cs_ext_sales_price#3] -Input [4]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3, d_date_sk#4] +(10) Sort [codegen id : 4] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5 ASC NULLS FIRST], false, 0 -(11) Exchange -Input [2]: [cs_item_sk#2, cs_ext_sales_price#3] -Arguments: hashpartitioning(cs_item_sk#2, 5), true, [id=#7] +(11) SortMergeJoin [codegen id : 6] +Left keys [1]: [cs_item_sk#2] +Right keys [1]: [i_item_sk#5] +Join condition: None -(12) Sort [codegen id : 3] -Input [2]: [cs_item_sk#2, cs_ext_sales_price#3] -Arguments: [cs_item_sk#2 ASC NULLS FIRST], false, 0 +(12) Project [codegen id : 6] +Output [7]: [cs_sold_date_sk#1, cs_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [9]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) Scan parquet default.item -Output [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(13) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_date#13] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(14) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(14) ColumnarToRow [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] -(15) Filter [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Condition : (i_category#13 IN (Sports,Books,Home) AND isnotnull(i_item_sk#8)) +(15) Filter [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) -(16) Exchange -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: hashpartitioning(i_item_sk#8, 5), true, [id=#14] +(16) Project [codegen id : 5] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_date#13] -(17) Sort [codegen id : 5] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: [i_item_sk#8 ASC NULLS FIRST], false, 0 +(17) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [i_item_sk#8] +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_sold_date_sk#1] +Right keys [1]: [d_date_sk#12] Join condition: None (19) Project [codegen id : 6] -Output [6]: [cs_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Input [8]: [cs_item_sk#2, cs_ext_sales_price#3, i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +Output [6]: [cs_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [8]: [cs_sold_date_sk#1, cs_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#12] (20) HashAggregate [codegen id : 6] -Input [6]: [cs_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [cs_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#3))] Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] (21) Exchange -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Arguments: hashpartitioning(i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, 5), true, [id=#17] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [id=#17] (22) HashAggregate [codegen id : 7] -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#3))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#3))#18] -Results [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS _w1#21, i_item_id#9] +Results [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS _w1#21, i_item_id#6] (23) Exchange -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: hashpartitioning(i_class#12, 5), true, [id=#22] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [id=#22] (24) Sort [codegen id : 8] -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: [i_class#12 ASC NULLS FIRST], false, 0 +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 (25) Window -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: [sum(_w1#21) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#12] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: [sum(_w1#21) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#9] (26) Project [codegen id : 9] -Output [7]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24, i_item_id#9] -Input [9]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9, _we0#23] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24, i_item_id#6] +Input [9]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6, _we0#23] (27) TakeOrderedAndProject -Input [7]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24, i_item_id#9] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST, i_item_desc#10 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24, i_item_id#6] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/simplified.txt index 1e8ab18f5e21a..965b68218eb0a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/simplified.txt @@ -14,34 +14,34 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (6) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - SortMergeJoin [cs_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (3) - Sort [cs_item_sk] - InputAdapter - Exchange [cs_item_sk] #3 - WholeStageCodegen (2) - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + SortMergeJoin [cs_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [cs_item_sk] + InputAdapter + Exchange [cs_item_sk] #3 + WholeStageCodegen (1) Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_ext_sales_price] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #4 + WholeStageCodegen (3) + Filter [i_category,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter - WholeStageCodegen (5) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #5 - WholeStageCodegen (4) - Filter [i_category,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastExchange #5 + WholeStageCodegen (5) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt index 093c4eed6cf11..351c7f0024dd0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt @@ -1,567 +1,587 @@ == Physical Plan == -* Project (48) -+- * Filter (47) - +- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * SortMergeJoin Inner (39) - :- * Sort (33) - : +- Exchange (32) - : +- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (18) - : : +- * SortMergeJoin Inner (17) - : : :- * Sort (11) - : : : +- Exchange (10) - : : : +- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.item (4) - : : +- * Sort (16) - : : +- Exchange (15) - : : +- * Filter (14) - : : +- * ColumnarToRow (13) - : : +- Scan parquet default.customer (12) - : +- BroadcastExchange (29) - : +- * Project (28) - : +- * BroadcastHashJoin Inner BuildLeft (27) - : :- BroadcastExchange (23) - : : +- * Project (22) - : : +- * Filter (21) - : : +- * ColumnarToRow (20) - : : +- Scan parquet default.store (19) - : +- * Filter (26) - : +- * ColumnarToRow (25) - : +- Scan parquet default.customer_address (24) - +- * Sort (38) - +- Exchange (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet default.store_returns (34) +* Project (50) ++- * Filter (49) + +- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * Project (42) + +- * SortMergeJoin Inner (41) + :- * Sort (34) + : +- Exchange (33) + : +- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (19) + : : +- * SortMergeJoin Inner (18) + : : :- * Sort (12) + : : : +- Exchange (11) + : : : +- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Project (4) + : : : : +- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Filter (7) + : : : +- * ColumnarToRow (6) + : : : +- Scan parquet default.item (5) + : : +- * Sort (17) + : : +- Exchange (16) + : : +- * Filter (15) + : : +- * ColumnarToRow (14) + : : +- Scan parquet default.customer (13) + : +- BroadcastExchange (30) + : +- * Project (29) + : +- * BroadcastHashJoin Inner BuildLeft (28) + : :- BroadcastExchange (24) + : : +- * Project (23) + : : +- * Filter (22) + : : +- * ColumnarToRow (21) + : : +- Scan parquet default.store (20) + : +- * Filter (27) + : +- * ColumnarToRow (26) + : +- Scan parquet default.customer_address (25) + +- * Sort (40) + +- Exchange (39) + +- * Project (38) + +- * Filter (37) + +- * ColumnarToRow (36) + +- Scan parquet default.store_returns (35) (1) Scan parquet default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] (3) Filter [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) -(4) Scan parquet default.item -Output [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(4) Project [codegen id : 2] +Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] + +(5) Scan parquet default.item +Output [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] ReadSchema: struct -(5) ColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(6) ColumnarToRow [codegen id : 1] +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -(6) Filter [codegen id : 1] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Condition : ((isnotnull(i_color#9) AND (i_color#9 = pale)) AND isnotnull(i_item_sk#6)) +(7) Filter [codegen id : 1] +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Condition : ((isnotnull(i_color#10) AND (i_color#10 = pale )) AND isnotnull(i_item_sk#7)) -(7) BroadcastExchange -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#12] +(8) BroadcastExchange +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#13] -(8) BroadcastHashJoin [codegen id : 2] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +Right keys [1]: [i_item_sk#7] Join condition: None -(9) Project [codegen id : 2] -Output [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(10) Project [codegen id : 2] +Output [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -(10) Exchange -Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#13] +(11) Exchange +Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#14] -(11) Sort [codegen id : 3] -Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(12) Sort [codegen id : 3] +Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 -(12) Scan parquet default.customer -Output [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(13) Scan parquet default.customer +Output [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(13) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(14) ColumnarToRow [codegen id : 4] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] -(14) Filter [codegen id : 4] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) +(15) Filter [codegen id : 4] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_birth_country#18)) -(15) Exchange -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#18] +(16) Exchange +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: hashpartitioning(c_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#19] -(16) Sort [codegen id : 5] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 +(17) Sort [codegen id : 5] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: [c_customer_sk#15 ASC NULLS FIRST], false, 0 -(17) SortMergeJoin [codegen id : 8] +(18) SortMergeJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#14] +Right keys [1]: [c_customer_sk#15] Join condition: None -(18) Project [codegen id : 8] -Output [12]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(19) Project [codegen id : 8] +Output [12]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] -(19) Scan parquet default.store -Output [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(20) Scan parquet default.store +Output [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 6] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(21) ColumnarToRow [codegen id : 6] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] -(21) Filter [codegen id : 6] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] -Condition : (((isnotnull(s_market_id#21) AND (s_market_id#21 = 8)) AND isnotnull(s_store_sk#19)) AND isnotnull(s_zip#23)) +(22) Filter [codegen id : 6] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Condition : (((isnotnull(s_market_id#22) AND (s_market_id#22 = 8)) AND isnotnull(s_store_sk#20)) AND isnotnull(s_zip#24)) -(22) Project [codegen id : 6] -Output [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(23) Project [codegen id : 6] +Output [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] -(23) BroadcastExchange -Input [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] -Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id=#24] +(24) BroadcastExchange +Input [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] +Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id=#25] -(24) Scan parquet default.customer_address -Output [3]: [ca_state#25, ca_zip#26, ca_country#27] +(25) Scan parquet default.customer_address +Output [3]: [ca_state#26, ca_zip#27, ca_country#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(25) ColumnarToRow -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] +(26) ColumnarToRow +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] -(26) Filter -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) +(27) Filter +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] +Condition : (isnotnull(ca_country#28) AND isnotnull(ca_zip#27)) -(27) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [s_zip#23] -Right keys [1]: [ca_zip#26] +(28) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [s_zip#24] +Right keys [1]: [ca_zip#27] Join condition: None -(28) Project [codegen id : 7] -Output [5]: [s_store_sk#19, s_store_name#20, s_state#22, ca_state#25, ca_country#27] -Input [7]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23, ca_state#25, ca_zip#26, ca_country#27] +(29) Project [codegen id : 7] +Output [5]: [s_store_sk#20, s_store_name#21, s_state#23, ca_state#26, ca_country#28] +Input [7]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24, ca_state#26, ca_zip#27, ca_country#28] -(29) BroadcastExchange -Input [5]: [s_store_sk#19, s_store_name#20, s_state#22, ca_state#25, ca_country#27] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[4, string, true])),false), [id=#28] +(30) BroadcastExchange +Input [5]: [s_store_sk#20, s_store_name#21, s_state#23, ca_state#26, ca_country#28] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[4, string, true])),false), [id=#29] -(30) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [ss_store_sk#3, c_birth_country#17] -Right keys [2]: [s_store_sk#19, upper(ca_country#27)] +(31) BroadcastHashJoin [codegen id : 8] +Left keys [2]: [ss_store_sk#3, c_birth_country#18] +Right keys [2]: [s_store_sk#20, upper(ca_country#28)] Join condition: None -(31) Project [codegen id : 8] -Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25] -Input [17]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17, s_store_sk#19, s_store_name#20, s_state#22, ca_state#25, ca_country#27] +(32) Project [codegen id : 8] +Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, s_store_name#21, s_state#23, ca_state#26] +Input [17]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18, s_store_sk#20, s_store_name#21, s_state#23, ca_state#26, ca_country#28] -(32) Exchange -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#29] +(33) Exchange +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, s_store_name#21, s_state#23, ca_state#26] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#30] -(33) Sort [codegen id : 9] -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25] +(34) Sort [codegen id : 9] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, s_store_name#21, s_state#23, ca_state#26] Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 -(34) Scan parquet default.store_returns -Output [2]: [sr_item_sk#30, sr_ticket_number#31] +(35) Scan parquet default.store_returns +Output [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 10] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] +(36) ColumnarToRow [codegen id : 10] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] + +(37) Filter [codegen id : 10] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Condition : (isnotnull(sr_ticket_number#32) AND isnotnull(sr_item_sk#31)) -(36) Filter [codegen id : 10] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) +(38) Project [codegen id : 10] +Output [2]: [sr_item_sk#31, sr_ticket_number#32] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] -(37) Exchange -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#32] +(39) Exchange +Input [2]: [sr_item_sk#31, sr_ticket_number#32] +Arguments: hashpartitioning(sr_ticket_number#32, sr_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#34] -(38) Sort [codegen id : 11] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: [sr_ticket_number#31 ASC NULLS FIRST, sr_item_sk#30 ASC NULLS FIRST], false, 0 +(40) Sort [codegen id : 11] +Input [2]: [sr_item_sk#31, sr_ticket_number#32] +Arguments: [sr_ticket_number#32 ASC NULLS FIRST, sr_item_sk#31 ASC NULLS FIRST], false, 0 -(39) SortMergeJoin [codegen id : 12] +(41) SortMergeJoin [codegen id : 12] Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#31, sr_item_sk#30] +Right keys [2]: [sr_ticket_number#32, sr_item_sk#31] Join condition: None -(40) Project [codegen id : 12] -Output [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25, sr_item_sk#30, sr_ticket_number#31] +(42) Project [codegen id : 12] +Output [11]: [ss_net_paid#5, s_store_name#21, s_state#23, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, ca_state#26] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, s_store_name#21, s_state#23, ca_state#26, sr_item_sk#31, sr_ticket_number#32] -(41) HashAggregate [codegen id : 12] -Input [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Keys [10]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8] +(43) HashAggregate [codegen id : 12] +Input [11]: [ss_net_paid#5, s_store_name#21, s_state#23, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, ca_state#26] +Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#33] -Results [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#34] +Aggregate Attributes [1]: [sum#35] +Results [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] -(42) Exchange -Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#34] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), ENSURE_REQUIREMENTS, [id=#35] +(44) Exchange +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#37] -(43) HashAggregate [codegen id : 13] -Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#34] -Keys [10]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8] +(45) HashAggregate [codegen id : 13] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] +Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#36] -Results [4]: [c_last_name#16, c_first_name#15, s_store_name#20, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#36,17,2) AS netpaid#37] - -(44) HashAggregate [codegen id : 13] -Input [4]: [c_last_name#16, c_first_name#15, s_store_name#20, netpaid#37] -Keys [3]: [c_last_name#16, c_first_name#15, s_store_name#20] -Functions [1]: [partial_sum(netpaid#37)] -Aggregate Attributes [2]: [sum#38, isEmpty#39] -Results [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] - -(45) Exchange -Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, 5), ENSURE_REQUIREMENTS, [id=#42] - -(46) HashAggregate [codegen id : 14] -Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] -Keys [3]: [c_last_name#16, c_first_name#15, s_store_name#20] -Functions [1]: [sum(netpaid#37)] -Aggregate Attributes [1]: [sum(netpaid#37)#43] -Results [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum(netpaid#37)#43 AS paid#44, sum(netpaid#37)#43 AS sum(netpaid#37)#45] - -(47) Filter [codegen id : 14] -Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, paid#44, sum(netpaid#37)#45] -Condition : (isnotnull(sum(netpaid#37)#45) AND (cast(sum(netpaid#37)#45 as decimal(33,8)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(33,8)))) - -(48) Project [codegen id : 14] -Output [4]: [c_last_name#16, c_first_name#15, s_store_name#20, paid#44] -Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, paid#44, sum(netpaid#37)#45] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#38] +Results [4]: [c_last_name#17, c_first_name#16, s_store_name#21, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#38,17,2) AS netpaid#39] + +(46) HashAggregate [codegen id : 13] +Input [4]: [c_last_name#17, c_first_name#16, s_store_name#21, netpaid#39] +Keys [3]: [c_last_name#17, c_first_name#16, s_store_name#21] +Functions [1]: [partial_sum(netpaid#39)] +Aggregate Attributes [2]: [sum#40, isEmpty#41] +Results [5]: [c_last_name#17, c_first_name#16, s_store_name#21, sum#42, isEmpty#43] + +(47) Exchange +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#21, sum#42, isEmpty#43] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#21, 5), ENSURE_REQUIREMENTS, [id=#44] + +(48) HashAggregate [codegen id : 14] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#21, sum#42, isEmpty#43] +Keys [3]: [c_last_name#17, c_first_name#16, s_store_name#21] +Functions [1]: [sum(netpaid#39)] +Aggregate Attributes [1]: [sum(netpaid#39)#45] +Results [5]: [c_last_name#17, c_first_name#16, s_store_name#21, sum(netpaid#39)#45 AS paid#46, sum(netpaid#39)#45 AS sum(netpaid#39)#47] + +(49) Filter [codegen id : 14] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#21, paid#46, sum(netpaid#39)#47] +Condition : (isnotnull(sum(netpaid#39)#47) AND (cast(sum(netpaid#39)#47 as decimal(33,8)) > cast(Subquery scalar-subquery#48, [id=#49] as decimal(33,8)))) + +(50) Project [codegen id : 14] +Output [4]: [c_last_name#17, c_first_name#16, s_store_name#21, paid#46] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#21, paid#46, sum(netpaid#39)#47] ===== Subqueries ===== -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#46, [id=#47] -* HashAggregate (100) -+- Exchange (99) - +- * HashAggregate (98) - +- * HashAggregate (97) - +- Exchange (96) - +- * HashAggregate (95) - +- * Project (94) - +- * SortMergeJoin Inner (93) - :- * Sort (87) - : +- Exchange (86) - : +- * Project (85) - : +- * SortMergeJoin Inner (84) - : :- * Sort (78) - : : +- Exchange (77) - : : +- * Project (76) - : : +- * SortMergeJoin Inner (75) - : : :- * Sort (69) - : : : +- Exchange (68) - : : : +- * Project (67) - : : : +- * SortMergeJoin Inner (66) - : : : :- * Sort (60) - : : : : +- Exchange (59) - : : : : +- * Project (58) - : : : : +- * BroadcastHashJoin Inner BuildRight (57) - : : : : :- * Filter (51) - : : : : : +- * ColumnarToRow (50) - : : : : : +- Scan parquet default.store_sales (49) - : : : : +- BroadcastExchange (56) - : : : : +- * Project (55) - : : : : +- * Filter (54) - : : : : +- * ColumnarToRow (53) - : : : : +- Scan parquet default.store (52) - : : : +- * Sort (65) - : : : +- Exchange (64) - : : : +- * Filter (63) - : : : +- * ColumnarToRow (62) - : : : +- Scan parquet default.customer (61) - : : +- * Sort (74) - : : +- Exchange (73) - : : +- * Filter (72) - : : +- * ColumnarToRow (71) - : : +- Scan parquet default.item (70) - : +- * Sort (83) - : +- Exchange (82) - : +- * Filter (81) - : +- * ColumnarToRow (80) - : +- Scan parquet default.customer_address (79) - +- * Sort (92) - +- Exchange (91) - +- * Filter (90) - +- * ColumnarToRow (89) - +- Scan parquet default.store_returns (88) - - -(49) Scan parquet default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#48, [id=#49] +* HashAggregate (104) ++- Exchange (103) + +- * HashAggregate (102) + +- * HashAggregate (101) + +- Exchange (100) + +- * HashAggregate (99) + +- * Project (98) + +- * SortMergeJoin Inner (97) + :- * Sort (90) + : +- Exchange (89) + : +- * Project (88) + : +- * SortMergeJoin Inner (87) + : :- * Sort (81) + : : +- Exchange (80) + : : +- * Project (79) + : : +- * SortMergeJoin Inner (78) + : : :- * Sort (72) + : : : +- Exchange (71) + : : : +- * Project (70) + : : : +- * SortMergeJoin Inner (69) + : : : :- * Sort (63) + : : : : +- Exchange (62) + : : : : +- * Project (61) + : : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : : :- * Project (54) + : : : : : +- * Filter (53) + : : : : : +- * ColumnarToRow (52) + : : : : : +- Scan parquet default.store_sales (51) + : : : : +- BroadcastExchange (59) + : : : : +- * Project (58) + : : : : +- * Filter (57) + : : : : +- * ColumnarToRow (56) + : : : : +- Scan parquet default.store (55) + : : : +- * Sort (68) + : : : +- Exchange (67) + : : : +- * Filter (66) + : : : +- * ColumnarToRow (65) + : : : +- Scan parquet default.item (64) + : : +- * Sort (77) + : : +- Exchange (76) + : : +- * Filter (75) + : : +- * ColumnarToRow (74) + : : +- Scan parquet default.customer (73) + : +- * Sort (86) + : +- Exchange (85) + : +- * Filter (84) + : +- * ColumnarToRow (83) + : +- Scan parquet default.customer_address (82) + +- * Sort (96) + +- Exchange (95) + +- * Project (94) + +- * Filter (93) + +- * ColumnarToRow (92) + +- Scan parquet default.store_returns (91) + + +(51) Scan parquet default.store_sales +Output [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(50) ColumnarToRow [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +(52) ColumnarToRow [codegen id : 2] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] -(51) Filter [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) +(53) Filter [codegen id : 2] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +Condition : (((isnotnull(ss_ticket_number#53) AND isnotnull(ss_item_sk#50)) AND isnotnull(ss_store_sk#52)) AND isnotnull(ss_customer_sk#51)) -(52) Scan parquet default.store -Output [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(54) Project [codegen id : 2] +Output [5]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] + +(55) Scan parquet default.store +Output [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(53) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(56) ColumnarToRow [codegen id : 1] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] -(54) Filter [codegen id : 1] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] -Condition : (((isnotnull(s_market_id#21) AND (s_market_id#21 = 8)) AND isnotnull(s_store_sk#19)) AND isnotnull(s_zip#23)) +(57) Filter [codegen id : 1] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Condition : (((isnotnull(s_market_id#58) AND (s_market_id#58 = 8)) AND isnotnull(s_store_sk#56)) AND isnotnull(s_zip#60)) -(55) Project [codegen id : 1] -Output [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(58) Project [codegen id : 1] +Output [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] -(56) BroadcastExchange -Input [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] +(59) BroadcastExchange +Input [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] -(57) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#19] +(60) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_store_sk#52] +Right keys [1]: [s_store_sk#56] Join condition: None -(58) Project [codegen id : 2] -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] +(61) Project [codegen id : 2] +Output [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Input [9]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] -(59) Exchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#49] +(62) Exchange +Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Arguments: hashpartitioning(ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#62] -(60) Sort [codegen id : 3] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] -Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 +(63) Sort [codegen id : 3] +Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Arguments: [ss_item_sk#50 ASC NULLS FIRST], false, 0 -(61) Scan parquet default.customer -Output [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(64) Scan parquet default.item +Output [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct -(62) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(65) ColumnarToRow [codegen id : 4] +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -(63) Filter [codegen id : 4] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) +(66) Filter [codegen id : 4] +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Condition : isnotnull(i_item_sk#63) -(64) Exchange -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#50] +(67) Exchange +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: hashpartitioning(i_item_sk#63, 5), ENSURE_REQUIREMENTS, [id=#69] -(65) Sort [codegen id : 5] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 +(68) Sort [codegen id : 5] +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: [i_item_sk#63 ASC NULLS FIRST], false, 0 -(66) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#14] +(69) SortMergeJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#50] +Right keys [1]: [i_item_sk#63] Join condition: None -(67) Project [codegen id : 6] -Output [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(70) Project [codegen id : 6] +Output [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Input [13]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -(68) Exchange -Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#51] +(71) Exchange +Input [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [id=#70] -(69) Sort [codegen id : 7] -Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +(72) Sort [codegen id : 7] +Input [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: [ss_customer_sk#51 ASC NULLS FIRST], false, 0 -(70) Scan parquet default.item -Output [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(73) Scan parquet default.customer +Output [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] +ReadSchema: struct -(71) ColumnarToRow [codegen id : 8] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(74) ColumnarToRow [codegen id : 8] +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] -(72) Filter [codegen id : 8] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Condition : isnotnull(i_item_sk#6) +(75) Filter [codegen id : 8] +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +Condition : (isnotnull(c_customer_sk#71) AND isnotnull(c_birth_country#74)) -(73) Exchange -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(i_item_sk#6, 5), ENSURE_REQUIREMENTS, [id=#52] +(76) Exchange +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: hashpartitioning(c_customer_sk#71, 5), ENSURE_REQUIREMENTS, [id=#75] -(74) Sort [codegen id : 9] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 +(77) Sort [codegen id : 9] +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: [c_customer_sk#71 ASC NULLS FIRST], false, 0 -(75) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +(78) SortMergeJoin [codegen id : 10] +Left keys [1]: [ss_customer_sk#51] +Right keys [1]: [c_customer_sk#71] Join condition: None -(76) Project [codegen id : 10] -Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(79) Project [codegen id : 10] +Output [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Input [16]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] -(77) Exchange -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(c_birth_country#17, s_zip#23, 5), ENSURE_REQUIREMENTS, [id=#53] +(80) Exchange +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: hashpartitioning(c_birth_country#74, s_zip#60, 5), ENSURE_REQUIREMENTS, [id=#76] -(78) Sort [codegen id : 11] -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: [c_birth_country#17 ASC NULLS FIRST, s_zip#23 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 11] +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: [c_birth_country#74 ASC NULLS FIRST, s_zip#60 ASC NULLS FIRST], false, 0 -(79) Scan parquet default.customer_address -Output [3]: [ca_state#25, ca_zip#26, ca_country#27] +(82) Scan parquet default.customer_address +Output [3]: [ca_state#77, ca_zip#78, ca_country#79] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(80) ColumnarToRow [codegen id : 12] -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] +(83) ColumnarToRow [codegen id : 12] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] -(81) Filter [codegen id : 12] -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) +(84) Filter [codegen id : 12] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Condition : (isnotnull(ca_country#79) AND isnotnull(ca_zip#78)) -(82) Exchange -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: hashpartitioning(upper(ca_country#27), ca_zip#26, 5), ENSURE_REQUIREMENTS, [id=#54] +(85) Exchange +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Arguments: hashpartitioning(upper(ca_country#79), ca_zip#78, 5), ENSURE_REQUIREMENTS, [id=#80] -(83) Sort [codegen id : 13] -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: [upper(ca_country#27) ASC NULLS FIRST, ca_zip#26 ASC NULLS FIRST], false, 0 +(86) Sort [codegen id : 13] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Arguments: [upper(ca_country#79) ASC NULLS FIRST, ca_zip#78 ASC NULLS FIRST], false, 0 -(84) SortMergeJoin [codegen id : 14] -Left keys [2]: [c_birth_country#17, s_zip#23] -Right keys [2]: [upper(ca_country#27), ca_zip#26] +(87) SortMergeJoin [codegen id : 14] +Left keys [2]: [c_birth_country#74, s_zip#60] +Right keys [2]: [upper(ca_country#79), ca_zip#78] Join condition: None -(85) Project [codegen id : 14] -Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] -Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25, ca_zip#26, ca_country#27] +(88) Project [codegen id : 14] +Output [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Input [17]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74, ca_state#77, ca_zip#78, ca_country#79] -(86) Exchange -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#55] +(89) Exchange +Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Arguments: hashpartitioning(cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint), 5), ENSURE_REQUIREMENTS, [id=#81] -(87) Sort [codegen id : 15] -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] -Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 +(90) Sort [codegen id : 15] +Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Arguments: [cast(ss_ticket_number#53 as bigint) ASC NULLS FIRST, cast(ss_item_sk#50 as bigint) ASC NULLS FIRST], false, 0 -(88) Scan parquet default.store_returns -Output [2]: [sr_item_sk#30, sr_ticket_number#31] +(91) Scan parquet default.store_returns +Output [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(89) ColumnarToRow [codegen id : 16] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] +(92) ColumnarToRow [codegen id : 16] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] -(90) Filter [codegen id : 16] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) +(93) Filter [codegen id : 16] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] +Condition : (isnotnull(sr_ticket_number#83) AND isnotnull(sr_item_sk#82)) -(91) Exchange -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#56] +(94) Project [codegen id : 16] +Output [2]: [sr_item_sk#82, sr_ticket_number#83] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] -(92) Sort [codegen id : 17] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: [sr_ticket_number#31 ASC NULLS FIRST, sr_item_sk#30 ASC NULLS FIRST], false, 0 +(95) Exchange +Input [2]: [sr_item_sk#82, sr_ticket_number#83] +Arguments: hashpartitioning(sr_ticket_number#83, sr_item_sk#82, 5), ENSURE_REQUIREMENTS, [id=#85] -(93) SortMergeJoin [codegen id : 18] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#31, sr_item_sk#30] -Join condition: None - -(94) Project [codegen id : 18] -Output [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25, sr_item_sk#30, sr_ticket_number#31] - -(95) HashAggregate [codegen id : 18] -Input [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Keys [10]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#57] -Results [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#58] +(96) Sort [codegen id : 17] +Input [2]: [sr_item_sk#82, sr_ticket_number#83] +Arguments: [sr_ticket_number#83 ASC NULLS FIRST, sr_item_sk#82 ASC NULLS FIRST], false, 0 -(96) Exchange -Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#58] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), ENSURE_REQUIREMENTS, [id=#59] - -(97) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#58] -Keys [10]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#60] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#60,17,2) AS netpaid#37] +(97) SortMergeJoin [codegen id : 18] +Left keys [2]: [cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint)] +Right keys [2]: [sr_ticket_number#83, sr_item_sk#82] +Join condition: None -(98) HashAggregate [codegen id : 19] -Input [1]: [netpaid#37] +(98) Project [codegen id : 18] +Output [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77, sr_item_sk#82, sr_ticket_number#83] + +(99) HashAggregate [codegen id : 18] +Input [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#54))] +Aggregate Attributes [1]: [sum#86] +Results [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#87] + +(100) Exchange +Input [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#87] +Arguments: hashpartitioning(c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, 5), ENSURE_REQUIREMENTS, [id=#88] + +(101) HashAggregate [codegen id : 19] +Input [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#87] +Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] +Functions [1]: [sum(UnscaledValue(ss_net_paid#54))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#54))#89] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#54))#89,17,2) AS netpaid#39] + +(102) HashAggregate [codegen id : 19] +Input [1]: [netpaid#39] Keys: [] -Functions [1]: [partial_avg(netpaid#37)] -Aggregate Attributes [2]: [sum#61, count#62] -Results [2]: [sum#63, count#64] +Functions [1]: [partial_avg(netpaid#39)] +Aggregate Attributes [2]: [sum#90, count#91] +Results [2]: [sum#92, count#93] -(99) Exchange -Input [2]: [sum#63, count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] +(103) Exchange +Input [2]: [sum#92, count#93] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#94] -(100) HashAggregate [codegen id : 20] -Input [2]: [sum#63, count#64] +(104) HashAggregate [codegen id : 20] +Input [2]: [sum#92, count#93] Keys: [] -Functions [1]: [avg(netpaid#37)] -Aggregate Attributes [1]: [avg(netpaid#37)#66] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#37)#66)), DecimalType(24,8), true) AS (CAST(0.05 AS DECIMAL(21,6)) * CAST(avg(netpaid) AS DECIMAL(21,6)))#67] +Functions [1]: [avg(netpaid#39)] +Aggregate Attributes [1]: [avg(netpaid#39)#95] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#95)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#96] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt index 7de562c5d59a1..10f874f8f5543 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt @@ -21,7 +21,7 @@ WholeStageCodegen (14) InputAdapter Exchange [ss_ticket_number,ss_item_sk] #12 WholeStageCodegen (14) - Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,c_first_name,c_last_name,i_current_price,i_size,i_color,i_units,i_manager_id,ca_state] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] SortMergeJoin [c_birth_country,s_zip,ca_country,ca_zip] InputAdapter WholeStageCodegen (11) @@ -29,21 +29,21 @@ WholeStageCodegen (14) InputAdapter Exchange [c_birth_country,s_zip] #13 WholeStageCodegen (10) - Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,c_first_name,c_last_name,c_birth_country,i_current_price,i_size,i_color,i_units,i_manager_id] - SortMergeJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (7) - Sort [ss_item_sk] + Sort [ss_customer_sk] InputAdapter - Exchange [ss_item_sk] #14 + Exchange [ss_customer_sk] #14 WholeStageCodegen (6) - Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,c_first_name,c_last_name,c_birth_country] - SortMergeJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (3) - Sort [ss_customer_sk] + Sort [ss_item_sk] InputAdapter - Exchange [ss_customer_sk] #15 + Exchange [ss_item_sk] #15 WholeStageCodegen (2) Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip] BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -61,24 +61,24 @@ WholeStageCodegen (14) Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter WholeStageCodegen (5) - Sort [c_customer_sk] + Sort [i_item_sk] InputAdapter - Exchange [c_customer_sk] #17 + Exchange [i_item_sk] #17 WholeStageCodegen (4) - Filter [c_customer_sk,c_birth_country] + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter WholeStageCodegen (9) - Sort [i_item_sk] + Sort [c_customer_sk] InputAdapter - Exchange [i_item_sk] #18 + Exchange [c_customer_sk] #18 WholeStageCodegen (8) - Filter [i_item_sk] + Filter [c_customer_sk,c_birth_country] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter WholeStageCodegen (13) Sort [ca_country,ca_zip] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt index 273950bed3546..97ee167a14b5a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt @@ -1,567 +1,587 @@ == Physical Plan == -* Project (48) -+- * Filter (47) - +- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * SortMergeJoin Inner (39) - :- * Sort (33) - : +- Exchange (32) - : +- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (18) - : : +- * SortMergeJoin Inner (17) - : : :- * Sort (11) - : : : +- Exchange (10) - : : : +- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.item (4) - : : +- * Sort (16) - : : +- Exchange (15) - : : +- * Filter (14) - : : +- * ColumnarToRow (13) - : : +- Scan parquet default.customer (12) - : +- BroadcastExchange (29) - : +- * Project (28) - : +- * BroadcastHashJoin Inner BuildLeft (27) - : :- BroadcastExchange (23) - : : +- * Project (22) - : : +- * Filter (21) - : : +- * ColumnarToRow (20) - : : +- Scan parquet default.store (19) - : +- * Filter (26) - : +- * ColumnarToRow (25) - : +- Scan parquet default.customer_address (24) - +- * Sort (38) - +- Exchange (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet default.store_returns (34) +* Project (50) ++- * Filter (49) + +- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * Project (42) + +- * SortMergeJoin Inner (41) + :- * Sort (34) + : +- Exchange (33) + : +- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (19) + : : +- * SortMergeJoin Inner (18) + : : :- * Sort (12) + : : : +- Exchange (11) + : : : +- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Project (4) + : : : : +- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Filter (7) + : : : +- * ColumnarToRow (6) + : : : +- Scan parquet default.item (5) + : : +- * Sort (17) + : : +- Exchange (16) + : : +- * Filter (15) + : : +- * ColumnarToRow (14) + : : +- Scan parquet default.customer (13) + : +- BroadcastExchange (30) + : +- * Project (29) + : +- * BroadcastHashJoin Inner BuildLeft (28) + : :- BroadcastExchange (24) + : : +- * Project (23) + : : +- * Filter (22) + : : +- * ColumnarToRow (21) + : : +- Scan parquet default.store (20) + : +- * Filter (27) + : +- * ColumnarToRow (26) + : +- Scan parquet default.customer_address (25) + +- * Sort (40) + +- Exchange (39) + +- * Project (38) + +- * Filter (37) + +- * ColumnarToRow (36) + +- Scan parquet default.store_returns (35) (1) Scan parquet default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] (3) Filter [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) -(4) Scan parquet default.item -Output [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(4) Project [codegen id : 2] +Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] + +(5) Scan parquet default.item +Output [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon ), IsNotNull(i_item_sk)] ReadSchema: struct -(5) ColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(6) ColumnarToRow [codegen id : 1] +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -(6) Filter [codegen id : 1] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Condition : ((isnotnull(i_color#9) AND (i_color#9 = chiffon)) AND isnotnull(i_item_sk#6)) +(7) Filter [codegen id : 1] +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Condition : ((isnotnull(i_color#10) AND (i_color#10 = chiffon )) AND isnotnull(i_item_sk#7)) -(7) BroadcastExchange -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#12] +(8) BroadcastExchange +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#13] -(8) BroadcastHashJoin [codegen id : 2] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +Right keys [1]: [i_item_sk#7] Join condition: None -(9) Project [codegen id : 2] -Output [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(10) Project [codegen id : 2] +Output [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -(10) Exchange -Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#13] +(11) Exchange +Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#14] -(11) Sort [codegen id : 3] -Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(12) Sort [codegen id : 3] +Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 -(12) Scan parquet default.customer -Output [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(13) Scan parquet default.customer +Output [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(13) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(14) ColumnarToRow [codegen id : 4] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] -(14) Filter [codegen id : 4] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) +(15) Filter [codegen id : 4] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_birth_country#18)) -(15) Exchange -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#18] +(16) Exchange +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: hashpartitioning(c_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#19] -(16) Sort [codegen id : 5] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 +(17) Sort [codegen id : 5] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: [c_customer_sk#15 ASC NULLS FIRST], false, 0 -(17) SortMergeJoin [codegen id : 8] +(18) SortMergeJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#14] +Right keys [1]: [c_customer_sk#15] Join condition: None -(18) Project [codegen id : 8] -Output [12]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(19) Project [codegen id : 8] +Output [12]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] -(19) Scan parquet default.store -Output [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(20) Scan parquet default.store +Output [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 6] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(21) ColumnarToRow [codegen id : 6] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] -(21) Filter [codegen id : 6] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] -Condition : (((isnotnull(s_market_id#21) AND (s_market_id#21 = 8)) AND isnotnull(s_store_sk#19)) AND isnotnull(s_zip#23)) +(22) Filter [codegen id : 6] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Condition : (((isnotnull(s_market_id#22) AND (s_market_id#22 = 8)) AND isnotnull(s_store_sk#20)) AND isnotnull(s_zip#24)) -(22) Project [codegen id : 6] -Output [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(23) Project [codegen id : 6] +Output [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] -(23) BroadcastExchange -Input [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] -Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id=#24] +(24) BroadcastExchange +Input [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] +Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id=#25] -(24) Scan parquet default.customer_address -Output [3]: [ca_state#25, ca_zip#26, ca_country#27] +(25) Scan parquet default.customer_address +Output [3]: [ca_state#26, ca_zip#27, ca_country#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(25) ColumnarToRow -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] +(26) ColumnarToRow +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] -(26) Filter -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) +(27) Filter +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] +Condition : (isnotnull(ca_country#28) AND isnotnull(ca_zip#27)) -(27) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [s_zip#23] -Right keys [1]: [ca_zip#26] +(28) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [s_zip#24] +Right keys [1]: [ca_zip#27] Join condition: None -(28) Project [codegen id : 7] -Output [5]: [s_store_sk#19, s_store_name#20, s_state#22, ca_state#25, ca_country#27] -Input [7]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23, ca_state#25, ca_zip#26, ca_country#27] +(29) Project [codegen id : 7] +Output [5]: [s_store_sk#20, s_store_name#21, s_state#23, ca_state#26, ca_country#28] +Input [7]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24, ca_state#26, ca_zip#27, ca_country#28] -(29) BroadcastExchange -Input [5]: [s_store_sk#19, s_store_name#20, s_state#22, ca_state#25, ca_country#27] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[4, string, true])),false), [id=#28] +(30) BroadcastExchange +Input [5]: [s_store_sk#20, s_store_name#21, s_state#23, ca_state#26, ca_country#28] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[4, string, true])),false), [id=#29] -(30) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [ss_store_sk#3, c_birth_country#17] -Right keys [2]: [s_store_sk#19, upper(ca_country#27)] +(31) BroadcastHashJoin [codegen id : 8] +Left keys [2]: [ss_store_sk#3, c_birth_country#18] +Right keys [2]: [s_store_sk#20, upper(ca_country#28)] Join condition: None -(31) Project [codegen id : 8] -Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25] -Input [17]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17, s_store_sk#19, s_store_name#20, s_state#22, ca_state#25, ca_country#27] +(32) Project [codegen id : 8] +Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, s_store_name#21, s_state#23, ca_state#26] +Input [17]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18, s_store_sk#20, s_store_name#21, s_state#23, ca_state#26, ca_country#28] -(32) Exchange -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#29] +(33) Exchange +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, s_store_name#21, s_state#23, ca_state#26] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#30] -(33) Sort [codegen id : 9] -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25] +(34) Sort [codegen id : 9] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, s_store_name#21, s_state#23, ca_state#26] Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 -(34) Scan parquet default.store_returns -Output [2]: [sr_item_sk#30, sr_ticket_number#31] +(35) Scan parquet default.store_returns +Output [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 10] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] +(36) ColumnarToRow [codegen id : 10] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] + +(37) Filter [codegen id : 10] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Condition : (isnotnull(sr_ticket_number#32) AND isnotnull(sr_item_sk#31)) -(36) Filter [codegen id : 10] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) +(38) Project [codegen id : 10] +Output [2]: [sr_item_sk#31, sr_ticket_number#32] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] -(37) Exchange -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#32] +(39) Exchange +Input [2]: [sr_item_sk#31, sr_ticket_number#32] +Arguments: hashpartitioning(sr_ticket_number#32, sr_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#34] -(38) Sort [codegen id : 11] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: [sr_ticket_number#31 ASC NULLS FIRST, sr_item_sk#30 ASC NULLS FIRST], false, 0 +(40) Sort [codegen id : 11] +Input [2]: [sr_item_sk#31, sr_ticket_number#32] +Arguments: [sr_ticket_number#32 ASC NULLS FIRST, sr_item_sk#31 ASC NULLS FIRST], false, 0 -(39) SortMergeJoin [codegen id : 12] +(41) SortMergeJoin [codegen id : 12] Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#31, sr_item_sk#30] +Right keys [2]: [sr_ticket_number#32, sr_item_sk#31] Join condition: None -(40) Project [codegen id : 12] -Output [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25, sr_item_sk#30, sr_ticket_number#31] +(42) Project [codegen id : 12] +Output [11]: [ss_net_paid#5, s_store_name#21, s_state#23, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, ca_state#26] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, s_store_name#21, s_state#23, ca_state#26, sr_item_sk#31, sr_ticket_number#32] -(41) HashAggregate [codegen id : 12] -Input [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Keys [10]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8] +(43) HashAggregate [codegen id : 12] +Input [11]: [ss_net_paid#5, s_store_name#21, s_state#23, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, ca_state#26] +Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#33] -Results [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#34] +Aggregate Attributes [1]: [sum#35] +Results [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] -(42) Exchange -Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#34] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), ENSURE_REQUIREMENTS, [id=#35] +(44) Exchange +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#37] -(43) HashAggregate [codegen id : 13] -Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#34] -Keys [10]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8] +(45) HashAggregate [codegen id : 13] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] +Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#36] -Results [4]: [c_last_name#16, c_first_name#15, s_store_name#20, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#36,17,2) AS netpaid#37] - -(44) HashAggregate [codegen id : 13] -Input [4]: [c_last_name#16, c_first_name#15, s_store_name#20, netpaid#37] -Keys [3]: [c_last_name#16, c_first_name#15, s_store_name#20] -Functions [1]: [partial_sum(netpaid#37)] -Aggregate Attributes [2]: [sum#38, isEmpty#39] -Results [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] - -(45) Exchange -Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, 5), ENSURE_REQUIREMENTS, [id=#42] - -(46) HashAggregate [codegen id : 14] -Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] -Keys [3]: [c_last_name#16, c_first_name#15, s_store_name#20] -Functions [1]: [sum(netpaid#37)] -Aggregate Attributes [1]: [sum(netpaid#37)#43] -Results [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum(netpaid#37)#43 AS paid#44, sum(netpaid#37)#43 AS sum(netpaid#37)#45] - -(47) Filter [codegen id : 14] -Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, paid#44, sum(netpaid#37)#45] -Condition : (isnotnull(sum(netpaid#37)#45) AND (cast(sum(netpaid#37)#45 as decimal(33,8)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(33,8)))) - -(48) Project [codegen id : 14] -Output [4]: [c_last_name#16, c_first_name#15, s_store_name#20, paid#44] -Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, paid#44, sum(netpaid#37)#45] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#38] +Results [4]: [c_last_name#17, c_first_name#16, s_store_name#21, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#38,17,2) AS netpaid#39] + +(46) HashAggregate [codegen id : 13] +Input [4]: [c_last_name#17, c_first_name#16, s_store_name#21, netpaid#39] +Keys [3]: [c_last_name#17, c_first_name#16, s_store_name#21] +Functions [1]: [partial_sum(netpaid#39)] +Aggregate Attributes [2]: [sum#40, isEmpty#41] +Results [5]: [c_last_name#17, c_first_name#16, s_store_name#21, sum#42, isEmpty#43] + +(47) Exchange +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#21, sum#42, isEmpty#43] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#21, 5), ENSURE_REQUIREMENTS, [id=#44] + +(48) HashAggregate [codegen id : 14] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#21, sum#42, isEmpty#43] +Keys [3]: [c_last_name#17, c_first_name#16, s_store_name#21] +Functions [1]: [sum(netpaid#39)] +Aggregate Attributes [1]: [sum(netpaid#39)#45] +Results [5]: [c_last_name#17, c_first_name#16, s_store_name#21, sum(netpaid#39)#45 AS paid#46, sum(netpaid#39)#45 AS sum(netpaid#39)#47] + +(49) Filter [codegen id : 14] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#21, paid#46, sum(netpaid#39)#47] +Condition : (isnotnull(sum(netpaid#39)#47) AND (cast(sum(netpaid#39)#47 as decimal(33,8)) > cast(Subquery scalar-subquery#48, [id=#49] as decimal(33,8)))) + +(50) Project [codegen id : 14] +Output [4]: [c_last_name#17, c_first_name#16, s_store_name#21, paid#46] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#21, paid#46, sum(netpaid#39)#47] ===== Subqueries ===== -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#46, [id=#47] -* HashAggregate (100) -+- Exchange (99) - +- * HashAggregate (98) - +- * HashAggregate (97) - +- Exchange (96) - +- * HashAggregate (95) - +- * Project (94) - +- * SortMergeJoin Inner (93) - :- * Sort (87) - : +- Exchange (86) - : +- * Project (85) - : +- * SortMergeJoin Inner (84) - : :- * Sort (78) - : : +- Exchange (77) - : : +- * Project (76) - : : +- * SortMergeJoin Inner (75) - : : :- * Sort (69) - : : : +- Exchange (68) - : : : +- * Project (67) - : : : +- * SortMergeJoin Inner (66) - : : : :- * Sort (60) - : : : : +- Exchange (59) - : : : : +- * Project (58) - : : : : +- * BroadcastHashJoin Inner BuildRight (57) - : : : : :- * Filter (51) - : : : : : +- * ColumnarToRow (50) - : : : : : +- Scan parquet default.store_sales (49) - : : : : +- BroadcastExchange (56) - : : : : +- * Project (55) - : : : : +- * Filter (54) - : : : : +- * ColumnarToRow (53) - : : : : +- Scan parquet default.store (52) - : : : +- * Sort (65) - : : : +- Exchange (64) - : : : +- * Filter (63) - : : : +- * ColumnarToRow (62) - : : : +- Scan parquet default.customer (61) - : : +- * Sort (74) - : : +- Exchange (73) - : : +- * Filter (72) - : : +- * ColumnarToRow (71) - : : +- Scan parquet default.item (70) - : +- * Sort (83) - : +- Exchange (82) - : +- * Filter (81) - : +- * ColumnarToRow (80) - : +- Scan parquet default.customer_address (79) - +- * Sort (92) - +- Exchange (91) - +- * Filter (90) - +- * ColumnarToRow (89) - +- Scan parquet default.store_returns (88) - - -(49) Scan parquet default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#48, [id=#49] +* HashAggregate (104) ++- Exchange (103) + +- * HashAggregate (102) + +- * HashAggregate (101) + +- Exchange (100) + +- * HashAggregate (99) + +- * Project (98) + +- * SortMergeJoin Inner (97) + :- * Sort (90) + : +- Exchange (89) + : +- * Project (88) + : +- * SortMergeJoin Inner (87) + : :- * Sort (81) + : : +- Exchange (80) + : : +- * Project (79) + : : +- * SortMergeJoin Inner (78) + : : :- * Sort (72) + : : : +- Exchange (71) + : : : +- * Project (70) + : : : +- * SortMergeJoin Inner (69) + : : : :- * Sort (63) + : : : : +- Exchange (62) + : : : : +- * Project (61) + : : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : : :- * Project (54) + : : : : : +- * Filter (53) + : : : : : +- * ColumnarToRow (52) + : : : : : +- Scan parquet default.store_sales (51) + : : : : +- BroadcastExchange (59) + : : : : +- * Project (58) + : : : : +- * Filter (57) + : : : : +- * ColumnarToRow (56) + : : : : +- Scan parquet default.store (55) + : : : +- * Sort (68) + : : : +- Exchange (67) + : : : +- * Filter (66) + : : : +- * ColumnarToRow (65) + : : : +- Scan parquet default.item (64) + : : +- * Sort (77) + : : +- Exchange (76) + : : +- * Filter (75) + : : +- * ColumnarToRow (74) + : : +- Scan parquet default.customer (73) + : +- * Sort (86) + : +- Exchange (85) + : +- * Filter (84) + : +- * ColumnarToRow (83) + : +- Scan parquet default.customer_address (82) + +- * Sort (96) + +- Exchange (95) + +- * Project (94) + +- * Filter (93) + +- * ColumnarToRow (92) + +- Scan parquet default.store_returns (91) + + +(51) Scan parquet default.store_sales +Output [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(50) ColumnarToRow [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +(52) ColumnarToRow [codegen id : 2] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] -(51) Filter [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) +(53) Filter [codegen id : 2] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +Condition : (((isnotnull(ss_ticket_number#53) AND isnotnull(ss_item_sk#50)) AND isnotnull(ss_store_sk#52)) AND isnotnull(ss_customer_sk#51)) -(52) Scan parquet default.store -Output [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(54) Project [codegen id : 2] +Output [5]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] + +(55) Scan parquet default.store +Output [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(53) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(56) ColumnarToRow [codegen id : 1] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] -(54) Filter [codegen id : 1] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] -Condition : (((isnotnull(s_market_id#21) AND (s_market_id#21 = 8)) AND isnotnull(s_store_sk#19)) AND isnotnull(s_zip#23)) +(57) Filter [codegen id : 1] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Condition : (((isnotnull(s_market_id#58) AND (s_market_id#58 = 8)) AND isnotnull(s_store_sk#56)) AND isnotnull(s_zip#60)) -(55) Project [codegen id : 1] -Output [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(58) Project [codegen id : 1] +Output [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] -(56) BroadcastExchange -Input [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] +(59) BroadcastExchange +Input [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] -(57) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#19] +(60) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_store_sk#52] +Right keys [1]: [s_store_sk#56] Join condition: None -(58) Project [codegen id : 2] -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] +(61) Project [codegen id : 2] +Output [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Input [9]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] -(59) Exchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#49] +(62) Exchange +Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Arguments: hashpartitioning(ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#62] -(60) Sort [codegen id : 3] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] -Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 +(63) Sort [codegen id : 3] +Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Arguments: [ss_item_sk#50 ASC NULLS FIRST], false, 0 -(61) Scan parquet default.customer -Output [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(64) Scan parquet default.item +Output [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct -(62) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(65) ColumnarToRow [codegen id : 4] +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -(63) Filter [codegen id : 4] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) +(66) Filter [codegen id : 4] +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Condition : isnotnull(i_item_sk#63) -(64) Exchange -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#50] +(67) Exchange +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: hashpartitioning(i_item_sk#63, 5), ENSURE_REQUIREMENTS, [id=#69] -(65) Sort [codegen id : 5] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 +(68) Sort [codegen id : 5] +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: [i_item_sk#63 ASC NULLS FIRST], false, 0 -(66) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#14] +(69) SortMergeJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#50] +Right keys [1]: [i_item_sk#63] Join condition: None -(67) Project [codegen id : 6] -Output [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(70) Project [codegen id : 6] +Output [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Input [13]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -(68) Exchange -Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#51] +(71) Exchange +Input [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [id=#70] -(69) Sort [codegen id : 7] -Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +(72) Sort [codegen id : 7] +Input [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: [ss_customer_sk#51 ASC NULLS FIRST], false, 0 -(70) Scan parquet default.item -Output [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(73) Scan parquet default.customer +Output [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] +ReadSchema: struct -(71) ColumnarToRow [codegen id : 8] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(74) ColumnarToRow [codegen id : 8] +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] -(72) Filter [codegen id : 8] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Condition : isnotnull(i_item_sk#6) +(75) Filter [codegen id : 8] +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +Condition : (isnotnull(c_customer_sk#71) AND isnotnull(c_birth_country#74)) -(73) Exchange -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(i_item_sk#6, 5), ENSURE_REQUIREMENTS, [id=#52] +(76) Exchange +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: hashpartitioning(c_customer_sk#71, 5), ENSURE_REQUIREMENTS, [id=#75] -(74) Sort [codegen id : 9] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 +(77) Sort [codegen id : 9] +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: [c_customer_sk#71 ASC NULLS FIRST], false, 0 -(75) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +(78) SortMergeJoin [codegen id : 10] +Left keys [1]: [ss_customer_sk#51] +Right keys [1]: [c_customer_sk#71] Join condition: None -(76) Project [codegen id : 10] -Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(79) Project [codegen id : 10] +Output [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Input [16]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] -(77) Exchange -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(c_birth_country#17, s_zip#23, 5), ENSURE_REQUIREMENTS, [id=#53] +(80) Exchange +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: hashpartitioning(c_birth_country#74, s_zip#60, 5), ENSURE_REQUIREMENTS, [id=#76] -(78) Sort [codegen id : 11] -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: [c_birth_country#17 ASC NULLS FIRST, s_zip#23 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 11] +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: [c_birth_country#74 ASC NULLS FIRST, s_zip#60 ASC NULLS FIRST], false, 0 -(79) Scan parquet default.customer_address -Output [3]: [ca_state#25, ca_zip#26, ca_country#27] +(82) Scan parquet default.customer_address +Output [3]: [ca_state#77, ca_zip#78, ca_country#79] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(80) ColumnarToRow [codegen id : 12] -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] +(83) ColumnarToRow [codegen id : 12] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] -(81) Filter [codegen id : 12] -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) +(84) Filter [codegen id : 12] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Condition : (isnotnull(ca_country#79) AND isnotnull(ca_zip#78)) -(82) Exchange -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: hashpartitioning(upper(ca_country#27), ca_zip#26, 5), ENSURE_REQUIREMENTS, [id=#54] +(85) Exchange +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Arguments: hashpartitioning(upper(ca_country#79), ca_zip#78, 5), ENSURE_REQUIREMENTS, [id=#80] -(83) Sort [codegen id : 13] -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: [upper(ca_country#27) ASC NULLS FIRST, ca_zip#26 ASC NULLS FIRST], false, 0 +(86) Sort [codegen id : 13] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Arguments: [upper(ca_country#79) ASC NULLS FIRST, ca_zip#78 ASC NULLS FIRST], false, 0 -(84) SortMergeJoin [codegen id : 14] -Left keys [2]: [c_birth_country#17, s_zip#23] -Right keys [2]: [upper(ca_country#27), ca_zip#26] +(87) SortMergeJoin [codegen id : 14] +Left keys [2]: [c_birth_country#74, s_zip#60] +Right keys [2]: [upper(ca_country#79), ca_zip#78] Join condition: None -(85) Project [codegen id : 14] -Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] -Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25, ca_zip#26, ca_country#27] +(88) Project [codegen id : 14] +Output [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Input [17]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74, ca_state#77, ca_zip#78, ca_country#79] -(86) Exchange -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#55] +(89) Exchange +Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Arguments: hashpartitioning(cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint), 5), ENSURE_REQUIREMENTS, [id=#81] -(87) Sort [codegen id : 15] -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] -Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 +(90) Sort [codegen id : 15] +Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Arguments: [cast(ss_ticket_number#53 as bigint) ASC NULLS FIRST, cast(ss_item_sk#50 as bigint) ASC NULLS FIRST], false, 0 -(88) Scan parquet default.store_returns -Output [2]: [sr_item_sk#30, sr_ticket_number#31] +(91) Scan parquet default.store_returns +Output [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(89) ColumnarToRow [codegen id : 16] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] +(92) ColumnarToRow [codegen id : 16] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] -(90) Filter [codegen id : 16] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) +(93) Filter [codegen id : 16] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] +Condition : (isnotnull(sr_ticket_number#83) AND isnotnull(sr_item_sk#82)) -(91) Exchange -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#56] +(94) Project [codegen id : 16] +Output [2]: [sr_item_sk#82, sr_ticket_number#83] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] -(92) Sort [codegen id : 17] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: [sr_ticket_number#31 ASC NULLS FIRST, sr_item_sk#30 ASC NULLS FIRST], false, 0 +(95) Exchange +Input [2]: [sr_item_sk#82, sr_ticket_number#83] +Arguments: hashpartitioning(sr_ticket_number#83, sr_item_sk#82, 5), ENSURE_REQUIREMENTS, [id=#85] -(93) SortMergeJoin [codegen id : 18] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#31, sr_item_sk#30] -Join condition: None - -(94) Project [codegen id : 18] -Output [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25, sr_item_sk#30, sr_ticket_number#31] - -(95) HashAggregate [codegen id : 18] -Input [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Keys [10]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#57] -Results [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#58] +(96) Sort [codegen id : 17] +Input [2]: [sr_item_sk#82, sr_ticket_number#83] +Arguments: [sr_ticket_number#83 ASC NULLS FIRST, sr_item_sk#82 ASC NULLS FIRST], false, 0 -(96) Exchange -Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#58] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), ENSURE_REQUIREMENTS, [id=#59] - -(97) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#58] -Keys [10]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#60] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#60,17,2) AS netpaid#37] +(97) SortMergeJoin [codegen id : 18] +Left keys [2]: [cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint)] +Right keys [2]: [sr_ticket_number#83, sr_item_sk#82] +Join condition: None -(98) HashAggregate [codegen id : 19] -Input [1]: [netpaid#37] +(98) Project [codegen id : 18] +Output [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77, sr_item_sk#82, sr_ticket_number#83] + +(99) HashAggregate [codegen id : 18] +Input [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#54))] +Aggregate Attributes [1]: [sum#86] +Results [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#87] + +(100) Exchange +Input [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#87] +Arguments: hashpartitioning(c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, 5), ENSURE_REQUIREMENTS, [id=#88] + +(101) HashAggregate [codegen id : 19] +Input [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#87] +Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] +Functions [1]: [sum(UnscaledValue(ss_net_paid#54))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#54))#89] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#54))#89,17,2) AS netpaid#39] + +(102) HashAggregate [codegen id : 19] +Input [1]: [netpaid#39] Keys: [] -Functions [1]: [partial_avg(netpaid#37)] -Aggregate Attributes [2]: [sum#61, count#62] -Results [2]: [sum#63, count#64] +Functions [1]: [partial_avg(netpaid#39)] +Aggregate Attributes [2]: [sum#90, count#91] +Results [2]: [sum#92, count#93] -(99) Exchange -Input [2]: [sum#63, count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] +(103) Exchange +Input [2]: [sum#92, count#93] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#94] -(100) HashAggregate [codegen id : 20] -Input [2]: [sum#63, count#64] +(104) HashAggregate [codegen id : 20] +Input [2]: [sum#92, count#93] Keys: [] -Functions [1]: [avg(netpaid#37)] -Aggregate Attributes [1]: [avg(netpaid#37)#66] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#37)#66)), DecimalType(24,8), true) AS (CAST(0.05 AS DECIMAL(21,6)) * CAST(avg(netpaid) AS DECIMAL(21,6)))#67] +Functions [1]: [avg(netpaid#39)] +Aggregate Attributes [1]: [avg(netpaid#39)#95] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#95)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#96] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt index 7de562c5d59a1..10f874f8f5543 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt @@ -21,7 +21,7 @@ WholeStageCodegen (14) InputAdapter Exchange [ss_ticket_number,ss_item_sk] #12 WholeStageCodegen (14) - Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,c_first_name,c_last_name,i_current_price,i_size,i_color,i_units,i_manager_id,ca_state] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] SortMergeJoin [c_birth_country,s_zip,ca_country,ca_zip] InputAdapter WholeStageCodegen (11) @@ -29,21 +29,21 @@ WholeStageCodegen (14) InputAdapter Exchange [c_birth_country,s_zip] #13 WholeStageCodegen (10) - Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,c_first_name,c_last_name,c_birth_country,i_current_price,i_size,i_color,i_units,i_manager_id] - SortMergeJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (7) - Sort [ss_item_sk] + Sort [ss_customer_sk] InputAdapter - Exchange [ss_item_sk] #14 + Exchange [ss_customer_sk] #14 WholeStageCodegen (6) - Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,c_first_name,c_last_name,c_birth_country] - SortMergeJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (3) - Sort [ss_customer_sk] + Sort [ss_item_sk] InputAdapter - Exchange [ss_customer_sk] #15 + Exchange [ss_item_sk] #15 WholeStageCodegen (2) Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip] BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -61,24 +61,24 @@ WholeStageCodegen (14) Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter WholeStageCodegen (5) - Sort [c_customer_sk] + Sort [i_item_sk] InputAdapter - Exchange [c_customer_sk] #17 + Exchange [i_item_sk] #17 WholeStageCodegen (4) - Filter [c_customer_sk,c_birth_country] + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter WholeStageCodegen (9) - Sort [i_item_sk] + Sort [c_customer_sk] InputAdapter - Exchange [i_item_sk] #18 + Exchange [c_customer_sk] #18 WholeStageCodegen (8) - Filter [i_item_sk] + Filter [c_customer_sk,c_birth_country] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter WholeStageCodegen (13) Sort [ca_country,ca_zip] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt index 3100e574e60e3..a531797a20c7e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt @@ -5,57 +5,57 @@ TakeOrderedAndProject (57) +- * HashAggregate (54) +- * Project (53) +- * SortMergeJoin Inner (52) - :- * Sort (27) - : +- Exchange (26) - : +- * Project (25) - : +- * SortMergeJoin Inner (24) - : :- * Sort (18) - : : +- Exchange (17) - : : +- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.store (11) - : +- * Sort (23) - : +- Exchange (22) - : +- * Filter (21) - : +- * ColumnarToRow (20) - : +- Scan parquet default.item (19) + :- * Sort (43) + : +- Exchange (42) + : +- * Project (41) + : +- * SortMergeJoin Inner (40) + : :- * Sort (27) + : : +- Exchange (26) + : : +- * Project (25) + : : +- * SortMergeJoin Inner (24) + : : :- * Sort (18) + : : : +- Exchange (17) + : : : +- * Project (16) + : : : +- * BroadcastHashJoin Inner BuildRight (15) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet default.store_sales (1) + : : : : +- BroadcastExchange (8) + : : : : +- * Project (7) + : : : : +- * Filter (6) + : : : : +- * ColumnarToRow (5) + : : : : +- Scan parquet default.date_dim (4) + : : : +- BroadcastExchange (14) + : : : +- * Filter (13) + : : : +- * ColumnarToRow (12) + : : : +- Scan parquet default.store (11) + : : +- * Sort (23) + : : +- Exchange (22) + : : +- * Filter (21) + : : +- * ColumnarToRow (20) + : : +- Scan parquet default.item (19) + : +- * Sort (39) + : +- Exchange (38) + : +- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Filter (30) + : : +- * ColumnarToRow (29) + : : +- Scan parquet default.store_returns (28) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- * Filter (33) + : +- * ColumnarToRow (32) + : +- Scan parquet default.date_dim (31) +- * Sort (51) +- Exchange (50) +- * Project (49) - +- * SortMergeJoin Inner (48) - :- * Sort (39) - : +- Exchange (38) - : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Filter (30) - : : +- * ColumnarToRow (29) - : : +- Scan parquet default.store_returns (28) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- * Filter (33) - : +- * ColumnarToRow (32) - : +- Scan parquet default.date_dim (31) - +- * Sort (47) - +- Exchange (46) - +- * Project (45) - +- * BroadcastHashJoin Inner BuildRight (44) - :- * Filter (42) - : +- * ColumnarToRow (41) - : +- Scan parquet default.catalog_sales (40) - +- ReusedExchange (43) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Filter (46) + : +- * ColumnarToRow (45) + : +- Scan parquet default.catalog_sales (44) + +- ReusedExchange (47) (1) Scan parquet default.store_sales @@ -224,76 +224,76 @@ Input [6]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_ (38) Exchange Input [4]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] -Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, 5), ENSURE_REQUIREMENTS, [id=#30] +Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24, 5), ENSURE_REQUIREMENTS, [id=#30] (39) Sort [codegen id : 11] Input [4]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] +Arguments: [sr_customer_sk#23 ASC NULLS FIRST, sr_item_sk#22 ASC NULLS FIRST, sr_ticket_number#24 ASC NULLS FIRST], false, 0 + +(40) SortMergeJoin [codegen id : 12] +Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] +Right keys [3]: [sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24] +Join condition: None + +(41) Project [codegen id : 12] +Output [8]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25] +Input [12]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] + +(42) Exchange +Input [8]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25] +Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, 5), ENSURE_REQUIREMENTS, [id=#31] + +(43) Sort [codegen id : 13] +Input [8]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25] Arguments: [sr_customer_sk#23 ASC NULLS FIRST, sr_item_sk#22 ASC NULLS FIRST], false, 0 -(40) Scan parquet default.catalog_sales -Output [4]: [cs_sold_date_sk#31, cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] +(44) Scan parquet default.catalog_sales +Output [4]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(41) ColumnarToRow [codegen id : 13] -Input [4]: [cs_sold_date_sk#31, cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] - -(42) Filter [codegen id : 13] -Input [4]: [cs_sold_date_sk#31, cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] -Condition : ((isnotnull(cs_bill_customer_sk#32) AND isnotnull(cs_item_sk#33)) AND isnotnull(cs_sold_date_sk#31)) +(45) ColumnarToRow [codegen id : 15] +Input [4]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] -(43) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#35] +(46) Filter [codegen id : 15] +Input [4]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] +Condition : ((isnotnull(cs_bill_customer_sk#33) AND isnotnull(cs_item_sk#34)) AND isnotnull(cs_sold_date_sk#32)) -(44) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cs_sold_date_sk#31] -Right keys [1]: [d_date_sk#35] -Join condition: None - -(45) Project [codegen id : 13] -Output [3]: [cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] -Input [5]: [cs_sold_date_sk#31, cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34, d_date_sk#35] - -(46) Exchange -Input [3]: [cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] -Arguments: hashpartitioning(cast(cs_bill_customer_sk#32 as bigint), cast(cs_item_sk#33 as bigint), 5), ENSURE_REQUIREMENTS, [id=#36] +(47) ReusedExchange [Reuses operator id: 35] +Output [1]: [d_date_sk#36] -(47) Sort [codegen id : 14] -Input [3]: [cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] -Arguments: [cast(cs_bill_customer_sk#32 as bigint) ASC NULLS FIRST, cast(cs_item_sk#33 as bigint) ASC NULLS FIRST], false, 0 - -(48) SortMergeJoin [codegen id : 15] -Left keys [2]: [sr_customer_sk#23, sr_item_sk#22] -Right keys [2]: [cast(cs_bill_customer_sk#32 as bigint), cast(cs_item_sk#33 as bigint)] +(48) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [cs_sold_date_sk#32] +Right keys [1]: [d_date_sk#36] Join condition: None (49) Project [codegen id : 15] -Output [5]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_net_profit#34] -Input [7]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] +Output [3]: [cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] +Input [5]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35, d_date_sk#36] (50) Exchange -Input [5]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_net_profit#34] -Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [3]: [cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] +Arguments: hashpartitioning(cast(cs_bill_customer_sk#33 as bigint), cast(cs_item_sk#34 as bigint), 5), ENSURE_REQUIREMENTS, [id=#37] (51) Sort [codegen id : 16] -Input [5]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_net_profit#34] -Arguments: [sr_customer_sk#23 ASC NULLS FIRST, sr_item_sk#22 ASC NULLS FIRST, sr_ticket_number#24 ASC NULLS FIRST], false, 0 +Input [3]: [cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] +Arguments: [cast(cs_bill_customer_sk#33 as bigint) ASC NULLS FIRST, cast(cs_item_sk#34 as bigint) ASC NULLS FIRST], false, 0 (52) SortMergeJoin [codegen id : 17] -Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] -Right keys [3]: [sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24] +Left keys [2]: [sr_customer_sk#23, sr_item_sk#22] +Right keys [2]: [cast(cs_bill_customer_sk#33 as bigint), cast(cs_item_sk#34 as bigint)] Join condition: None (53) Project [codegen id : 17] -Output [7]: [ss_net_profit#6, sr_net_loss#25, cs_net_profit#34, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] -Input [13]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_net_profit#34] +Output [7]: [ss_net_profit#6, sr_net_loss#25, cs_net_profit#35, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] +Input [11]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] (54) HashAggregate [codegen id : 17] -Input [7]: [ss_net_profit#6, sr_net_loss#25, cs_net_profit#34, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] +Input [7]: [ss_net_profit#6, sr_net_loss#25, cs_net_profit#35, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] Keys [4]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13] -Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#6)), partial_sum(UnscaledValue(sr_net_loss#25)), partial_sum(UnscaledValue(cs_net_profit#34))] +Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#6)), partial_sum(UnscaledValue(sr_net_loss#25)), partial_sum(UnscaledValue(cs_net_profit#35))] Aggregate Attributes [3]: [sum#38, sum#39, sum#40] Results [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, sum#41, sum#42, sum#43] @@ -304,9 +304,9 @@ Arguments: hashpartitioning(i_item_id#17, i_item_desc#18, s_store_id#12, s_store (56) HashAggregate [codegen id : 18] Input [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, sum#41, sum#42, sum#43] Keys [4]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13] -Functions [3]: [sum(UnscaledValue(ss_net_profit#6)), sum(UnscaledValue(sr_net_loss#25)), sum(UnscaledValue(cs_net_profit#34))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#6))#45, sum(UnscaledValue(sr_net_loss#25))#46, sum(UnscaledValue(cs_net_profit#34))#47] -Results [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, MakeDecimal(sum(UnscaledValue(ss_net_profit#6))#45,17,2) AS store_sales_profit#48, MakeDecimal(sum(UnscaledValue(sr_net_loss#25))#46,17,2) AS store_returns_loss#49, MakeDecimal(sum(UnscaledValue(cs_net_profit#34))#47,17,2) AS catalog_sales_profit#50] +Functions [3]: [sum(UnscaledValue(ss_net_profit#6)), sum(UnscaledValue(sr_net_loss#25)), sum(UnscaledValue(cs_net_profit#35))] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#6))#45, sum(UnscaledValue(sr_net_loss#25))#46, sum(UnscaledValue(cs_net_profit#35))#47] +Results [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, MakeDecimal(sum(UnscaledValue(ss_net_profit#6))#45,17,2) AS store_sales_profit#48, MakeDecimal(sum(UnscaledValue(sr_net_loss#25))#46,17,2) AS store_returns_loss#49, MakeDecimal(sum(UnscaledValue(cs_net_profit#35))#47,17,2) AS catalog_sales_profit#50] (57) TakeOrderedAndProject Input [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, store_sales_profit#48, store_returns_loss#49, catalog_sales_profit#50] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt index 9b53cdaa5dc67..ad9fa718ff2bd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt @@ -6,67 +6,67 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales WholeStageCodegen (17) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] [sum,sum,sum,sum,sum,sum] Project [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + SortMergeJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] InputAdapter - WholeStageCodegen (8) - Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] + WholeStageCodegen (13) + Sort [sr_customer_sk,sr_item_sk] InputAdapter - Exchange [ss_customer_sk,ss_item_sk,ss_ticket_number] #2 - WholeStageCodegen (7) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - SortMergeJoin [ss_item_sk,i_item_sk] + Exchange [sr_customer_sk,sr_item_sk] #2 + WholeStageCodegen (12) + Project [ss_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc,sr_item_sk,sr_customer_sk,sr_net_loss] + SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] InputAdapter - WholeStageCodegen (4) - Sort [ss_item_sk] + WholeStageCodegen (8) + Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] InputAdapter - Exchange [ss_item_sk] #3 - WholeStageCodegen (3) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Exchange [ss_customer_sk,ss_item_sk,ss_ticket_number] #3 + WholeStageCodegen (7) + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_id,s_store_name] - InputAdapter - WholeStageCodegen (6) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #6 - WholeStageCodegen (5) - Filter [i_item_sk] - ColumnarToRow + WholeStageCodegen (4) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #4 + WholeStageCodegen (3) + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_moy,d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + Filter [s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_store_sk,s_store_id,s_store_name] InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] - InputAdapter - WholeStageCodegen (16) - Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] - InputAdapter - Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #7 - WholeStageCodegen (15) - Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,cs_net_profit] - SortMergeJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + WholeStageCodegen (6) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #7 + WholeStageCodegen (5) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] InputAdapter WholeStageCodegen (11) - Sort [sr_customer_sk,sr_item_sk] + Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] InputAdapter - Exchange [sr_customer_sk,sr_item_sk] #8 + Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #8 WholeStageCodegen (10) Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss] BroadcastHashJoin [sr_returned_date_sk,d_date_sk] @@ -82,17 +82,17 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + WholeStageCodegen (16) + Sort [cs_bill_customer_sk,cs_item_sk] + InputAdapter + Exchange [cs_bill_customer_sk,cs_item_sk] #10 + WholeStageCodegen (15) + Project [cs_bill_customer_sk,cs_item_sk,cs_net_profit] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_net_profit] InputAdapter - WholeStageCodegen (14) - Sort [cs_bill_customer_sk,cs_item_sk] - InputAdapter - Exchange [cs_bill_customer_sk,cs_item_sk] #10 - WholeStageCodegen (13) - Project [cs_bill_customer_sk,cs_item_sk,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_net_profit] - InputAdapter - ReusedExchange [d_date_sk] #9 + ReusedExchange [d_date_sk] #9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt index cb8522545f1d3..2aa99626920ec 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt @@ -9,8 +9,8 @@ TakeOrderedAndProject (67) : +- * HashAggregate (30) : +- * Project (29) : +- * BroadcastHashJoin Inner BuildRight (28) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) + : :- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) : : :- * Project (10) : : : +- * BroadcastHashJoin Inner BuildRight (9) : : : :- * Filter (3) @@ -21,21 +21,21 @@ TakeOrderedAndProject (67) : : : +- * Filter (6) : : : +- * ColumnarToRow (5) : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (15) - : : +- * Project (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.customer_address (11) + : : +- BroadcastExchange (20) + : : +- * BroadcastHashJoin LeftSemi BuildRight (19) + : : :- * Filter (13) + : : : +- * ColumnarToRow (12) + : : : +- Scan parquet default.item (11) + : : +- BroadcastExchange (18) + : : +- * Project (17) + : : +- * Filter (16) + : : +- * ColumnarToRow (15) + : : +- Scan parquet default.item (14) : +- BroadcastExchange (27) - : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : :- * Filter (20) - : : +- * ColumnarToRow (19) - : : +- Scan parquet default.item (18) - : +- BroadcastExchange (25) - : +- * Project (24) - : +- * Filter (23) - : +- * ColumnarToRow (22) - : +- Scan parquet default.item (21) + : +- * Project (26) + : +- * Filter (25) + : +- * ColumnarToRow (24) + : +- Scan parquet default.customer_address (23) :- * HashAggregate (47) : +- Exchange (46) : +- * HashAggregate (45) @@ -69,310 +69,327 @@ TakeOrderedAndProject (67) (1) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] +Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 5] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] +Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] (3) Filter [codegen id : 5] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] -Condition : ((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_item_sk#2)) +Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] +Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) (4) Scan parquet default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_moy#7] +Output [3]: [d_date_sk#6, d_year#7, d_moy#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#6, d_moy#7] +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] (6) Filter [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#6, d_moy#7] -Condition : ((((isnotnull(d_year#6) AND isnotnull(d_moy#7)) AND (d_year#6 = 1998)) AND (d_moy#7 = 5)) AND isnotnull(d_date_sk#5)) +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 5)) AND isnotnull(d_date_sk#6)) (7) Project [codegen id : 1] -Output [1]: [d_date_sk#5] -Input [3]: [d_date_sk#5, d_year#6, d_moy#7] +Output [1]: [d_date_sk#6] +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] (8) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] +Input [1]: [d_date_sk#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#9] (9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#6] Join condition: None (10) Project [codegen id : 5] -Output [3]: [ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] -Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4, d_date_sk#5] +Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] +Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] -(11) Scan parquet default.customer_address -Output [2]: [ca_address_sk#9, ca_gmt_offset#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] - -(13) Filter [codegen id : 2] -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) - -(14) Project [codegen id : 2] -Output [1]: [ca_address_sk#9] -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] - -(15) BroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] - -(16) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#9] -Join condition: None - -(17) Project [codegen id : 5] -Output [2]: [ss_item_sk#2, ss_ext_sales_price#4] -Input [4]: [ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4, ca_address_sk#9] - -(18) Scan parquet default.item -Output [2]: [i_item_sk#12, i_manufact_id#13] +(11) Scan parquet default.item +Output [2]: [i_item_sk#10, i_manufact_id#11] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#12, i_manufact_id#13] +(12) ColumnarToRow [codegen id : 3] +Input [2]: [i_item_sk#10, i_manufact_id#11] -(20) Filter [codegen id : 4] -Input [2]: [i_item_sk#12, i_manufact_id#13] -Condition : isnotnull(i_item_sk#12) +(13) Filter [codegen id : 3] +Input [2]: [i_item_sk#10, i_manufact_id#11] +Condition : isnotnull(i_item_sk#10) -(21) Scan parquet default.item -Output [2]: [i_category#14, i_manufact_id#13] +(14) Scan parquet default.item +Output [2]: [i_category#12, i_manufact_id#13] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] ReadSchema: struct -(22) ColumnarToRow [codegen id : 3] -Input [2]: [i_category#14, i_manufact_id#13] +(15) ColumnarToRow [codegen id : 2] +Input [2]: [i_category#12, i_manufact_id#13] -(23) Filter [codegen id : 3] -Input [2]: [i_category#14, i_manufact_id#13] -Condition : (isnotnull(i_category#14) AND (i_category#14 = Electronics)) +(16) Filter [codegen id : 2] +Input [2]: [i_category#12, i_manufact_id#13] +Condition : (isnotnull(i_category#12) AND (i_category#12 = Electronics )) -(24) Project [codegen id : 3] -Output [1]: [i_manufact_id#13 AS i_manufact_id#13#15] -Input [2]: [i_category#14, i_manufact_id#13] +(17) Project [codegen id : 2] +Output [1]: [i_manufact_id#13] +Input [2]: [i_category#12, i_manufact_id#13] -(25) BroadcastExchange -Input [1]: [i_manufact_id#13#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +(18) BroadcastExchange +Input [1]: [i_manufact_id#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(26) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_manufact_id#13] -Right keys [1]: [i_manufact_id#13#15] +(19) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [i_manufact_id#11] +Right keys [1]: [i_manufact_id#13] Join condition: None +(20) BroadcastExchange +Input [2]: [i_item_sk#10, i_manufact_id#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] + +(21) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#10] +Join condition: None + +(22) Project [codegen id : 5] +Output [3]: [ss_addr_sk#2, ss_ext_sales_price#3, i_manufact_id#11] +Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, i_item_sk#10, i_manufact_id#11] + +(23) Scan parquet default.customer_address +Output [2]: [ca_address_sk#16, ca_gmt_offset#17] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] +ReadSchema: struct + +(24) ColumnarToRow [codegen id : 4] +Input [2]: [ca_address_sk#16, ca_gmt_offset#17] + +(25) Filter [codegen id : 4] +Input [2]: [ca_address_sk#16, ca_gmt_offset#17] +Condition : ((isnotnull(ca_gmt_offset#17) AND (ca_gmt_offset#17 = -5.00)) AND isnotnull(ca_address_sk#16)) + +(26) Project [codegen id : 4] +Output [1]: [ca_address_sk#16] +Input [2]: [ca_address_sk#16, ca_gmt_offset#17] + (27) BroadcastExchange -Input [2]: [i_item_sk#12, i_manufact_id#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] +Input [1]: [ca_address_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (28) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#12] +Left keys [1]: [ss_addr_sk#2] +Right keys [1]: [ca_address_sk#16] Join condition: None (29) Project [codegen id : 5] -Output [2]: [ss_ext_sales_price#4, i_manufact_id#13] -Input [4]: [ss_item_sk#2, ss_ext_sales_price#4, i_item_sk#12, i_manufact_id#13] +Output [2]: [ss_ext_sales_price#3, i_manufact_id#11] +Input [4]: [ss_addr_sk#2, ss_ext_sales_price#3, i_manufact_id#11, ca_address_sk#16] (30) HashAggregate [codegen id : 5] -Input [2]: [ss_ext_sales_price#4, i_manufact_id#13] -Keys [1]: [i_manufact_id#13] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#4))] -Aggregate Attributes [1]: [sum#18] -Results [2]: [i_manufact_id#13, sum#19] +Input [2]: [ss_ext_sales_price#3, i_manufact_id#11] +Keys [1]: [i_manufact_id#11] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [1]: [sum#19] +Results [2]: [i_manufact_id#11, sum#20] (31) Exchange -Input [2]: [i_manufact_id#13, sum#19] -Arguments: hashpartitioning(i_manufact_id#13, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [2]: [i_manufact_id#11, sum#20] +Arguments: hashpartitioning(i_manufact_id#11, 5), ENSURE_REQUIREMENTS, [id=#21] (32) HashAggregate [codegen id : 6] -Input [2]: [i_manufact_id#13, sum#19] -Keys [1]: [i_manufact_id#13] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#4))#21] -Results [2]: [i_manufact_id#13, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#4))#21,17,2) AS total_sales#22] +Input [2]: [i_manufact_id#11, sum#20] +Keys [1]: [i_manufact_id#11] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#22] +Results [2]: [i_manufact_id#11, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#22,17,2) AS total_sales#23] (33) Scan parquet default.catalog_sales -Output [4]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] +Output [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk), IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#27), dynamicpruningexpression(cs_sold_date_sk#27 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct (34) ColumnarToRow [codegen id : 11] -Input [4]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] (35) Filter [codegen id : 11] -Input [4]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Condition : ((isnotnull(cs_sold_date_sk#23) AND isnotnull(cs_bill_addr_sk#24)) AND isnotnull(cs_item_sk#25)) +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] +Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +Output [1]: [d_date_sk#28] (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [cs_sold_date_sk#27] +Right keys [1]: [d_date_sk#28] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, d_date_sk#5] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] -(39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#9] +(39) ReusedExchange [Reuses operator id: 20] +Output [2]: [i_item_sk#29, i_manufact_id#30] (40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#9] +Left keys [1]: [cs_item_sk#25] +Right keys [1]: [i_item_sk#29] Join condition: None (41) Project [codegen id : 11] -Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#9] +Output [3]: [cs_bill_addr_sk#24, cs_ext_sales_price#26, i_manufact_id#30] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#29, i_manufact_id#30] (42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#12, i_manufact_id#13] +Output [1]: [ca_address_sk#31] (43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#12] +Left keys [1]: [cs_bill_addr_sk#24] +Right keys [1]: [ca_address_sk#31] Join condition: None (44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_manufact_id#13] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#12, i_manufact_id#13] +Output [2]: [cs_ext_sales_price#26, i_manufact_id#30] +Input [4]: [cs_bill_addr_sk#24, cs_ext_sales_price#26, i_manufact_id#30, ca_address_sk#31] (45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_manufact_id#13] -Keys [1]: [i_manufact_id#13] +Input [2]: [cs_ext_sales_price#26, i_manufact_id#30] +Keys [1]: [i_manufact_id#30] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#27] -Results [2]: [i_manufact_id#13, sum#28] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_manufact_id#30, sum#33] (46) Exchange -Input [2]: [i_manufact_id#13, sum#28] -Arguments: hashpartitioning(i_manufact_id#13, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [2]: [i_manufact_id#30, sum#33] +Arguments: hashpartitioning(i_manufact_id#30, 5), ENSURE_REQUIREMENTS, [id=#34] (47) HashAggregate [codegen id : 12] -Input [2]: [i_manufact_id#13, sum#28] -Keys [1]: [i_manufact_id#13] +Input [2]: [i_manufact_id#30, sum#33] +Keys [1]: [i_manufact_id#30] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#30] -Results [2]: [i_manufact_id#13, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#30,17,2) AS total_sales#31] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] +Results [2]: [i_manufact_id#30, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] (48) Scan parquet default.web_sales -Output [4]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] +Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] +ReadSchema: struct (49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] (50) Filter [codegen id : 17] -Input [4]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Condition : ((isnotnull(ws_sold_date_sk#32) AND isnotnull(ws_bill_addr_sk#34)) AND isnotnull(ws_item_sk#33)) +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +Output [1]: [d_date_sk#41] (52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#32] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#41] Join condition: None (53) Project [codegen id : 17] -Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Input [5]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, d_date_sk#5] +Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] +Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] -(54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#9] +(54) ReusedExchange [Reuses operator id: 20] +Output [2]: [i_item_sk#42, i_manufact_id#43] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#9] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#42] Join condition: None (56) Project [codegen id : 17] -Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#9] +Output [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, i_manufact_id#43] +Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, i_item_sk#42, i_manufact_id#43] (57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#12, i_manufact_id#13] +Output [1]: [ca_address_sk#44] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#12] +Left keys [1]: [ws_bill_addr_sk#38] +Right keys [1]: [ca_address_sk#44] Join condition: None (59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#35, i_manufact_id#13] -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#12, i_manufact_id#13] +Output [2]: [ws_ext_sales_price#39, i_manufact_id#43] +Input [4]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, i_manufact_id#43, ca_address_sk#44] (60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#35, i_manufact_id#13] -Keys [1]: [i_manufact_id#13] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#36] -Results [2]: [i_manufact_id#13, sum#37] +Input [2]: [ws_ext_sales_price#39, i_manufact_id#43] +Keys [1]: [i_manufact_id#43] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum#45] +Results [2]: [i_manufact_id#43, sum#46] (61) Exchange -Input [2]: [i_manufact_id#13, sum#37] -Arguments: hashpartitioning(i_manufact_id#13, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [2]: [i_manufact_id#43, sum#46] +Arguments: hashpartitioning(i_manufact_id#43, 5), ENSURE_REQUIREMENTS, [id=#47] (62) HashAggregate [codegen id : 18] -Input [2]: [i_manufact_id#13, sum#37] -Keys [1]: [i_manufact_id#13] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#39] -Results [2]: [i_manufact_id#13, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#39,17,2) AS total_sales#40] +Input [2]: [i_manufact_id#43, sum#46] +Keys [1]: [i_manufact_id#43] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] +Results [2]: [i_manufact_id#43, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] (63) Union (64) HashAggregate [codegen id : 19] -Input [2]: [i_manufact_id#13, total_sales#22] -Keys [1]: [i_manufact_id#13] -Functions [1]: [partial_sum(total_sales#22)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [3]: [i_manufact_id#13, sum#43, isEmpty#44] +Input [2]: [i_manufact_id#11, total_sales#23] +Keys [1]: [i_manufact_id#11] +Functions [1]: [partial_sum(total_sales#23)] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [3]: [i_manufact_id#11, sum#52, isEmpty#53] (65) Exchange -Input [3]: [i_manufact_id#13, sum#43, isEmpty#44] -Arguments: hashpartitioning(i_manufact_id#13, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [3]: [i_manufact_id#11, sum#52, isEmpty#53] +Arguments: hashpartitioning(i_manufact_id#11, 5), ENSURE_REQUIREMENTS, [id=#54] (66) HashAggregate [codegen id : 20] -Input [3]: [i_manufact_id#13, sum#43, isEmpty#44] -Keys [1]: [i_manufact_id#13] -Functions [1]: [sum(total_sales#22)] -Aggregate Attributes [1]: [sum(total_sales#22)#46] -Results [2]: [i_manufact_id#13, sum(total_sales#22)#46 AS total_sales#47] +Input [3]: [i_manufact_id#11, sum#52, isEmpty#53] +Keys [1]: [i_manufact_id#11] +Functions [1]: [sum(total_sales#23)] +Aggregate Attributes [1]: [sum(total_sales#23)#55] +Results [2]: [i_manufact_id#11, sum(total_sales#23)#55 AS total_sales#56] (67) TakeOrderedAndProject -Input [2]: [i_manufact_id#13, total_sales#47] -Arguments: 100, [total_sales#47 ASC NULLS FIRST], [i_manufact_id#13, total_sales#47] +Input [2]: [i_manufact_id#11, total_sales#56] +Arguments: 100, [total_sales#56 ASC NULLS FIRST], [i_manufact_id#11, total_sales#56] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 +ReusedExchange (68) + + +(68) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#6] + +Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 + +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 + diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt index 14787f0bbce7b..410def2466e1a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt @@ -14,9 +14,9 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (5) HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_addr_sk,ss_ext_sales_price,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_addr_sk,ss_item_sk] @@ -33,28 +33,28 @@ TakeOrderedAndProject [total_sales,i_manufact_id] Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - Project [ca_address_sk] - Filter [ca_gmt_offset,ca_address_sk] + WholeStageCodegen (3) + BroadcastHashJoin [i_manufact_id,i_manufact_id] + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] + Scan parquet default.item [i_item_sk,i_manufact_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + Project [i_manufact_id] + Filter [i_category] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_category,i_manufact_id] InputAdapter - BroadcastExchange #5 + BroadcastExchange #6 WholeStageCodegen (4) - BroadcastHashJoin [i_manufact_id,i_manufact_id] - Filter [i_item_sk] + Project [ca_address_sk] + Filter [ca_gmt_offset,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [i_manufact_id] - Filter [i_category] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_category,i_manufact_id] + Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] WholeStageCodegen (12) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] InputAdapter @@ -62,9 +62,9 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (11) HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + Project [cs_bill_addr_sk,cs_ext_sales_price,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Filter [cs_sold_date_sk,cs_bill_addr_sk,cs_item_sk] @@ -74,9 +74,9 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter - ReusedExchange [ca_address_sk] #4 + ReusedExchange [i_item_sk,i_manufact_id] #4 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 + ReusedExchange [ca_address_sk] #6 WholeStageCodegen (18) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter @@ -84,9 +84,9 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (17) HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk,ws_bill_addr_sk,ws_item_sk] @@ -96,6 +96,6 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter - ReusedExchange [ca_address_sk] #4 + ReusedExchange [i_item_sk,i_manufact_id] #4 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 + ReusedExchange [ca_address_sk] #6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt index 7465ddae84e8a..a925e29ad6919 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt @@ -1,71 +1,72 @@ == Physical Plan == -* HashAggregate (67) -+- Exchange (66) - +- * HashAggregate (65) - +- * HashAggregate (64) - +- * HashAggregate (63) - +- * HashAggregate (62) - +- * HashAggregate (61) - +- * HashAggregate (60) - +- Exchange (59) - +- * HashAggregate (58) - +- SortMergeJoin LeftSemi (57) - :- SortMergeJoin LeftSemi (39) - : :- * Sort (21) - : : +- Exchange (20) - : : +- * Project (19) - : : +- * SortMergeJoin Inner (18) - : : :- * Sort (12) - : : : +- Exchange (11) - : : : +- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- * Sort (17) - : : +- Exchange (16) - : : +- * Filter (15) - : : +- * ColumnarToRow (14) - : : +- Scan parquet default.customer (13) - : +- * Sort (38) - : +- Exchange (37) - : +- * HashAggregate (36) - : +- Exchange (35) - : +- * HashAggregate (34) - : +- * Project (33) - : +- * SortMergeJoin Inner (32) - : :- * Sort (29) - : : +- Exchange (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Filter (24) - : : : +- * ColumnarToRow (23) - : : : +- Scan parquet default.catalog_sales (22) - : : +- ReusedExchange (25) - : +- * Sort (31) - : +- ReusedExchange (30) - +- * Sort (56) - +- Exchange (55) - +- * HashAggregate (54) - +- Exchange (53) - +- * HashAggregate (52) - +- * Project (51) - +- * SortMergeJoin Inner (50) - :- * Sort (47) - : +- Exchange (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Filter (42) - : : +- * ColumnarToRow (41) - : : +- Scan parquet default.web_sales (40) - : +- ReusedExchange (43) - +- * Sort (49) - +- ReusedExchange (48) +* HashAggregate (68) ++- Exchange (67) + +- * HashAggregate (66) + +- * HashAggregate (65) + +- * HashAggregate (64) + +- * HashAggregate (63) + +- * HashAggregate (62) + +- * HashAggregate (61) + +- Exchange (60) + +- * HashAggregate (59) + +- * Project (58) + +- SortMergeJoin LeftSemi (57) + :- SortMergeJoin LeftSemi (39) + : :- * Sort (21) + : : +- Exchange (20) + : : +- * Project (19) + : : +- * SortMergeJoin Inner (18) + : : :- * Sort (12) + : : : +- Exchange (11) + : : : +- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- * Sort (17) + : : +- Exchange (16) + : : +- * Filter (15) + : : +- * ColumnarToRow (14) + : : +- Scan parquet default.customer (13) + : +- * Sort (38) + : +- Exchange (37) + : +- * HashAggregate (36) + : +- Exchange (35) + : +- * HashAggregate (34) + : +- * Project (33) + : +- * SortMergeJoin Inner (32) + : :- * Sort (29) + : : +- Exchange (28) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Filter (24) + : : : +- * ColumnarToRow (23) + : : : +- Scan parquet default.catalog_sales (22) + : : +- ReusedExchange (25) + : +- * Sort (31) + : +- ReusedExchange (30) + +- * Sort (56) + +- Exchange (55) + +- * HashAggregate (54) + +- Exchange (53) + +- * HashAggregate (52) + +- * Project (51) + +- * SortMergeJoin Inner (50) + :- * Sort (47) + : +- Exchange (46) + : +- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Filter (42) + : : +- * ColumnarToRow (41) + : : +- Scan parquet default.web_sales (40) + : +- ReusedExchange (43) + +- * Sort (49) + +- ReusedExchange (48) (1) Scan parquet default.store_sales @@ -115,7 +116,7 @@ Input [4]: [ss_sold_date_sk#1, ss_customer_sk#2, d_date_sk#3, d_date#4] (11) Exchange Input [2]: [ss_customer_sk#2, d_date#4] -Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#7] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#7] (12) Sort [codegen id : 3] Input [2]: [ss_customer_sk#2, d_date#4] @@ -137,7 +138,7 @@ Condition : isnotnull(c_customer_sk#8) (16) Exchange Input [3]: [c_customer_sk#8, c_first_name#9, c_last_name#10] -Arguments: hashpartitioning(c_customer_sk#8, 5), true, [id=#11] +Arguments: hashpartitioning(c_customer_sk#8, 5), ENSURE_REQUIREMENTS, [id=#11] (17) Sort [codegen id : 5] Input [3]: [c_customer_sk#8, c_first_name#9, c_last_name#10] @@ -154,7 +155,7 @@ Input [5]: [ss_customer_sk#2, d_date#4, c_customer_sk#8, c_first_name#9, c_last_ (20) Exchange Input [3]: [d_date#4, c_first_name#9, c_last_name#10] -Arguments: hashpartitioning(coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#4, 0), isnull(d_date#4), 5), true, [id=#12] +Arguments: hashpartitioning(coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#4, 0), isnull(d_date#4), 5), ENSURE_REQUIREMENTS, [id=#12] (21) Sort [codegen id : 7] Input [3]: [d_date#4, c_first_name#9, c_last_name#10] @@ -188,7 +189,7 @@ Input [4]: [cs_sold_date_sk#13, cs_bill_customer_sk#14, d_date_sk#15, d_date#16] (28) Exchange Input [2]: [cs_bill_customer_sk#14, d_date#16] -Arguments: hashpartitioning(cs_bill_customer_sk#14, 5), true, [id=#17] +Arguments: hashpartitioning(cs_bill_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#17] (29) Sort [codegen id : 10] Input [2]: [cs_bill_customer_sk#14, d_date#16] @@ -219,7 +220,7 @@ Results [3]: [c_last_name#20, c_first_name#19, d_date#16] (35) Exchange Input [3]: [c_last_name#20, c_first_name#19, d_date#16] -Arguments: hashpartitioning(c_last_name#20, c_first_name#19, d_date#16, 5), true, [id=#21] +Arguments: hashpartitioning(c_last_name#20, c_first_name#19, d_date#16, 5), ENSURE_REQUIREMENTS, [id=#21] (36) HashAggregate [codegen id : 14] Input [3]: [c_last_name#20, c_first_name#19, d_date#16] @@ -230,7 +231,7 @@ Results [3]: [c_last_name#20, c_first_name#19, d_date#16] (37) Exchange Input [3]: [c_last_name#20, c_first_name#19, d_date#16] -Arguments: hashpartitioning(coalesce(c_last_name#20, ), isnull(c_last_name#20), coalesce(c_first_name#19, ), isnull(c_first_name#19), coalesce(d_date#16, 0), isnull(d_date#16), 5), true, [id=#22] +Arguments: hashpartitioning(coalesce(c_last_name#20, ), isnull(c_last_name#20), coalesce(c_first_name#19, ), isnull(c_first_name#19), coalesce(d_date#16, 0), isnull(d_date#16), 5), ENSURE_REQUIREMENTS, [id=#22] (38) Sort [codegen id : 15] Input [3]: [c_last_name#20, c_first_name#19, d_date#16] @@ -269,7 +270,7 @@ Input [4]: [ws_sold_date_sk#23, ws_bill_customer_sk#24, d_date_sk#25, d_date#26] (46) Exchange Input [2]: [ws_bill_customer_sk#24, d_date#26] -Arguments: hashpartitioning(ws_bill_customer_sk#24, 5), true, [id=#27] +Arguments: hashpartitioning(ws_bill_customer_sk#24, 5), ENSURE_REQUIREMENTS, [id=#27] (47) Sort [codegen id : 18] Input [2]: [ws_bill_customer_sk#24, d_date#26] @@ -300,7 +301,7 @@ Results [3]: [c_last_name#30, c_first_name#29, d_date#26] (53) Exchange Input [3]: [c_last_name#30, c_first_name#29, d_date#26] -Arguments: hashpartitioning(c_last_name#30, c_first_name#29, d_date#26, 5), true, [id=#31] +Arguments: hashpartitioning(c_last_name#30, c_first_name#29, d_date#26, 5), ENSURE_REQUIREMENTS, [id=#31] (54) HashAggregate [codegen id : 22] Input [3]: [c_last_name#30, c_first_name#29, d_date#26] @@ -311,7 +312,7 @@ Results [3]: [c_last_name#30, c_first_name#29, d_date#26] (55) Exchange Input [3]: [c_last_name#30, c_first_name#29, d_date#26] -Arguments: hashpartitioning(coalesce(c_last_name#30, ), isnull(c_last_name#30), coalesce(c_first_name#29, ), isnull(c_first_name#29), coalesce(d_date#26, 0), isnull(d_date#26), 5), true, [id=#32] +Arguments: hashpartitioning(coalesce(c_last_name#30, ), isnull(c_last_name#30), coalesce(c_first_name#29, ), isnull(c_first_name#29), coalesce(d_date#26, 0), isnull(d_date#26), 5), ENSURE_REQUIREMENTS, [id=#32] (56) Sort [codegen id : 23] Input [3]: [c_last_name#30, c_first_name#29, d_date#26] @@ -322,64 +323,68 @@ Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_f Right keys [6]: [coalesce(c_last_name#30, ), isnull(c_last_name#30), coalesce(c_first_name#29, ), isnull(c_first_name#29), coalesce(d_date#26, 0), isnull(d_date#26)] Join condition: None -(58) HashAggregate [codegen id : 24] +(58) Project [codegen id : 24] +Output [3]: [c_last_name#10, c_first_name#9, d_date#4] Input [3]: [d_date#4, c_first_name#9, c_last_name#10] + +(59) HashAggregate [codegen id : 24] +Input [3]: [c_last_name#10, c_first_name#9, d_date#4] Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#10, c_first_name#9, d_date#4] -(59) Exchange +(60) Exchange Input [3]: [c_last_name#10, c_first_name#9, d_date#4] -Arguments: hashpartitioning(c_last_name#10, c_first_name#9, d_date#4, 5), true, [id=#33] +Arguments: hashpartitioning(c_last_name#10, c_first_name#9, d_date#4, 5), ENSURE_REQUIREMENTS, [id=#33] -(60) HashAggregate [codegen id : 25] +(61) HashAggregate [codegen id : 25] Input [3]: [c_last_name#10, c_first_name#9, d_date#4] Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#10, c_first_name#9, d_date#4] -(61) HashAggregate [codegen id : 25] +(62) HashAggregate [codegen id : 25] Input [3]: [c_last_name#10, c_first_name#9, d_date#4] Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#10, c_first_name#9, d_date#4] -(62) HashAggregate [codegen id : 25] +(63) HashAggregate [codegen id : 25] Input [3]: [c_last_name#10, c_first_name#9, d_date#4] Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#10, c_first_name#9, d_date#4] -(63) HashAggregate [codegen id : 25] +(64) HashAggregate [codegen id : 25] Input [3]: [c_last_name#10, c_first_name#9, d_date#4] Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#10, c_first_name#9, d_date#4] -(64) HashAggregate [codegen id : 25] +(65) HashAggregate [codegen id : 25] Input [3]: [c_last_name#10, c_first_name#9, d_date#4] Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] Functions: [] Aggregate Attributes: [] Results: [] -(65) HashAggregate [codegen id : 25] +(66) HashAggregate [codegen id : 25] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#34] Results [1]: [count#35] -(66) Exchange +(67) Exchange Input [1]: [count#35] -Arguments: SinglePartition, true, [id=#36] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#36] -(67) HashAggregate [codegen id : 26] +(68) HashAggregate [codegen id : 26] Input [1]: [count#35] Keys: [] Functions [1]: [count(1)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt index 8dd59340cf069..015d3c5a81972 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt @@ -13,105 +13,106 @@ WholeStageCodegen (26) Exchange [c_last_name,c_first_name,d_date] #2 WholeStageCodegen (24) HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + InputAdapter SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - WholeStageCodegen (7) - Sort [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #3 - WholeStageCodegen (6) - Project [d_date,c_first_name,c_last_name] - SortMergeJoin [ss_customer_sk,c_customer_sk] + SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + WholeStageCodegen (7) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #3 + WholeStageCodegen (6) + Project [d_date,c_first_name,c_last_name] + SortMergeJoin [ss_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (3) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #4 + WholeStageCodegen (2) + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + WholeStageCodegen (5) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #6 + WholeStageCodegen (4) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + WholeStageCodegen (15) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #7 + WholeStageCodegen (14) + HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - WholeStageCodegen (3) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #4 - WholeStageCodegen (2) - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow + Exchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (13) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (10) + Sort [cs_bill_customer_sk] + InputAdapter + Exchange [cs_bill_customer_sk] #9 + WholeStageCodegen (9) + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_sold_date_sk,cs_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - WholeStageCodegen (5) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #6 - WholeStageCodegen (4) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] - WholeStageCodegen (15) + ReusedExchange [d_date_sk,d_date] #5 + InputAdapter + WholeStageCodegen (12) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 + WholeStageCodegen (23) Sort [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #7 - WholeStageCodegen (14) + Exchange [c_last_name,c_first_name,d_date] #10 + WholeStageCodegen (22) HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (13) + Exchange [c_last_name,c_first_name,d_date] #11 + WholeStageCodegen (21) HashAggregate [c_last_name,c_first_name,d_date] Project [c_last_name,c_first_name,d_date] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (10) - Sort [cs_bill_customer_sk] + WholeStageCodegen (18) + Sort [ws_bill_customer_sk] InputAdapter - Exchange [cs_bill_customer_sk] #9 - WholeStageCodegen (9) - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk,cs_bill_customer_sk] + Exchange [ws_bill_customer_sk] #12 + WholeStageCodegen (17) + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter ReusedExchange [d_date_sk,d_date] #5 InputAdapter - WholeStageCodegen (12) + WholeStageCodegen (20) Sort [c_customer_sk] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 - WholeStageCodegen (23) - Sort [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #10 - WholeStageCodegen (22) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #11 - WholeStageCodegen (21) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (18) - Sort [ws_bill_customer_sk] - InputAdapter - Exchange [ws_bill_customer_sk] #12 - WholeStageCodegen (17) - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 - InputAdapter - WholeStageCodegen (20) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt index 74454cf32afd0..05b4c6b71020e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt @@ -1,58 +1,59 @@ == Physical Plan == -* HashAggregate (54) -+- Exchange (53) - +- * HashAggregate (52) - +- * HashAggregate (51) - +- * HashAggregate (50) - +- * HashAggregate (49) - +- * HashAggregate (48) - +- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * BroadcastHashJoin LeftSemi BuildRight (44) - :- * BroadcastHashJoin LeftSemi BuildRight (30) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.customer (11) - : +- BroadcastExchange (29) - : +- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet default.catalog_sales (17) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - +- BroadcastExchange (43) - +- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Filter (33) - : : +- * ColumnarToRow (32) - : : +- Scan parquet default.web_sales (31) - : +- ReusedExchange (34) - +- ReusedExchange (37) +* HashAggregate (55) ++- Exchange (54) + +- * HashAggregate (53) + +- * HashAggregate (52) + +- * HashAggregate (51) + +- * HashAggregate (50) + +- * HashAggregate (49) + +- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * Project (45) + +- * BroadcastHashJoin LeftSemi BuildRight (44) + :- * BroadcastHashJoin LeftSemi BuildRight (30) + : :- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- BroadcastExchange (14) + : : +- * Filter (13) + : : +- * ColumnarToRow (12) + : : +- Scan parquet default.customer (11) + : +- BroadcastExchange (29) + : +- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * Filter (19) + : : : +- * ColumnarToRow (18) + : : : +- Scan parquet default.catalog_sales (17) + : : +- ReusedExchange (20) + : +- ReusedExchange (23) + +- BroadcastExchange (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * Filter (33) + : : +- * ColumnarToRow (32) + : : +- Scan parquet default.web_sales (31) + : +- ReusedExchange (34) + +- ReusedExchange (37) (1) Scan parquet default.store_sales @@ -174,7 +175,7 @@ Results [3]: [c_last_name#17, c_first_name#16, d_date#14] (27) Exchange Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Arguments: hashpartitioning(c_last_name#17, c_first_name#16, d_date#14, 5), true, [id=#18] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, d_date#14, 5), ENSURE_REQUIREMENTS, [id=#18] (28) HashAggregate [codegen id : 6] Input [3]: [c_last_name#17, c_first_name#16, d_date#14] @@ -239,7 +240,7 @@ Results [3]: [c_last_name#26, c_first_name#25, d_date#23] (41) Exchange Input [3]: [c_last_name#26, c_first_name#25, d_date#23] -Arguments: hashpartitioning(c_last_name#26, c_first_name#25, d_date#23, 5), true, [id=#27] +Arguments: hashpartitioning(c_last_name#26, c_first_name#25, d_date#23, 5), ENSURE_REQUIREMENTS, [id=#27] (42) HashAggregate [codegen id : 10] Input [3]: [c_last_name#26, c_first_name#25, d_date#23] @@ -257,64 +258,68 @@ Left keys [6]: [coalesce(c_last_name#9, ), isnull(c_last_name#9), coalesce(c_fir Right keys [6]: [coalesce(c_last_name#26, ), isnull(c_last_name#26), coalesce(c_first_name#25, ), isnull(c_first_name#25), coalesce(d_date#23, 0), isnull(d_date#23)] Join condition: None -(45) HashAggregate [codegen id : 11] +(45) Project [codegen id : 11] +Output [3]: [c_last_name#9, c_first_name#8, d_date#4] Input [3]: [d_date#4, c_first_name#8, c_last_name#9] + +(46) HashAggregate [codegen id : 11] +Input [3]: [c_last_name#9, c_first_name#8, d_date#4] Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#9, c_first_name#8, d_date#4] -(46) Exchange +(47) Exchange Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Arguments: hashpartitioning(c_last_name#9, c_first_name#8, d_date#4, 5), true, [id=#29] +Arguments: hashpartitioning(c_last_name#9, c_first_name#8, d_date#4, 5), ENSURE_REQUIREMENTS, [id=#29] -(47) HashAggregate [codegen id : 12] +(48) HashAggregate [codegen id : 12] Input [3]: [c_last_name#9, c_first_name#8, d_date#4] Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#9, c_first_name#8, d_date#4] -(48) HashAggregate [codegen id : 12] +(49) HashAggregate [codegen id : 12] Input [3]: [c_last_name#9, c_first_name#8, d_date#4] Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#9, c_first_name#8, d_date#4] -(49) HashAggregate [codegen id : 12] +(50) HashAggregate [codegen id : 12] Input [3]: [c_last_name#9, c_first_name#8, d_date#4] Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#9, c_first_name#8, d_date#4] -(50) HashAggregate [codegen id : 12] +(51) HashAggregate [codegen id : 12] Input [3]: [c_last_name#9, c_first_name#8, d_date#4] Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#9, c_first_name#8, d_date#4] -(51) HashAggregate [codegen id : 12] +(52) HashAggregate [codegen id : 12] Input [3]: [c_last_name#9, c_first_name#8, d_date#4] Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] Functions: [] Aggregate Attributes: [] Results: [] -(52) HashAggregate [codegen id : 12] +(53) HashAggregate [codegen id : 12] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#30] Results [1]: [count#31] -(53) Exchange +(54) Exchange Input [1]: [count#31] -Arguments: SinglePartition, true, [id=#32] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#32] -(54) HashAggregate [codegen id : 13] +(55) HashAggregate [codegen id : 13] Input [1]: [count#31] Keys: [] Functions [1]: [count(1)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt index a5b57a4ac9450..0f32bfbed99d7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt @@ -13,68 +13,69 @@ WholeStageCodegen (13) Exchange [c_last_name,c_first_name,d_date] #2 WholeStageCodegen (11) HashAggregate [c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - Project [d_date,c_first_name,c_last_name] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Filter [c_customer_sk] + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + Project [d_date,c_first_name,c_last_name] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (6) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #6 + WholeStageCodegen (5) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_sold_date_sk,cs_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) + BroadcastExchange #7 + WholeStageCodegen (10) HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (5) + Exchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (9) HashAggregate [c_last_name,c_first_name,d_date] Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk,cs_bill_customer_sk] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter ReusedExchange [d_date_sk,d_date] #3 InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (9) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt index 55bd25c501294..6a0794dc53efd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt @@ -186,14 +186,14 @@ Results [5]: [s_store_id#25, sum#31, sum#32, sum#33, sum#34] (24) Exchange Input [5]: [s_store_id#25, sum#31, sum#32, sum#33, sum#34] -Arguments: hashpartitioning(s_store_id#25, 5), true, [id=#35] +Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, [id=#35] (25) HashAggregate [codegen id : 6] Input [5]: [s_store_id#25, sum#31, sum#32, sum#33, sum#34] Keys [1]: [s_store_id#25] Functions [4]: [sum(UnscaledValue(sales_price#7)), sum(UnscaledValue(return_amt#9)), sum(UnscaledValue(profit#8)), sum(UnscaledValue(net_loss#10))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#7))#36, sum(UnscaledValue(return_amt#9))#37, sum(UnscaledValue(profit#8))#38, sum(UnscaledValue(net_loss#10))#39] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#7))#36,17,2) AS sales#40, MakeDecimal(sum(UnscaledValue(return_amt#9))#37,17,2) AS RETURNS#41, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#8))#38,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#10))#39,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#42, store channel AS channel#43, concat(store, s_store_id#25) AS id#44] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#7))#36,17,2) AS sales#40, MakeDecimal(sum(UnscaledValue(return_amt#9))#37,17,2) AS returns#41, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#8))#38,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#10))#39,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#42, store channel AS channel#43, concat(store, s_store_id#25) AS id#44] (26) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_ext_sales_price#47, cs_net_profit#48] @@ -281,14 +281,14 @@ Results [5]: [cp_catalog_page_id#66, sum#72, sum#73, sum#74, sum#75] (45) Exchange Input [5]: [cp_catalog_page_id#66, sum#72, sum#73, sum#74, sum#75] -Arguments: hashpartitioning(cp_catalog_page_id#66, 5), true, [id=#76] +Arguments: hashpartitioning(cp_catalog_page_id#66, 5), ENSURE_REQUIREMENTS, [id=#76] (46) HashAggregate [codegen id : 12] Input [5]: [cp_catalog_page_id#66, sum#72, sum#73, sum#74, sum#75] Keys [1]: [cp_catalog_page_id#66] Functions [4]: [sum(UnscaledValue(sales_price#51)), sum(UnscaledValue(return_amt#53)), sum(UnscaledValue(profit#52)), sum(UnscaledValue(net_loss#54))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#51))#77, sum(UnscaledValue(return_amt#53))#78, sum(UnscaledValue(profit#52))#79, sum(UnscaledValue(net_loss#54))#80] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#51))#77,17,2) AS sales#81, MakeDecimal(sum(UnscaledValue(return_amt#53))#78,17,2) AS RETURNS#82, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#52))#79,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#54))#80,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#83, catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#66) AS id#85] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#51))#77,17,2) AS sales#81, MakeDecimal(sum(UnscaledValue(return_amt#53))#78,17,2) AS returns#82, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#52))#79,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#54))#80,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#83, catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#66) AS id#85] (47) Scan parquet default.web_sales Output [4]: [ws_sold_date_sk#86, ws_web_site_sk#87, ws_ext_sales_price#88, ws_net_profit#89] @@ -324,7 +324,7 @@ Condition : isnotnull(wr_returned_date_sk#96) (54) Exchange Input [5]: [wr_returned_date_sk#96, wr_item_sk#97, wr_order_number#98, wr_return_amt#99, wr_net_loss#100] -Arguments: hashpartitioning(wr_item_sk#97, wr_order_number#98, 5), true, [id=#101] +Arguments: hashpartitioning(wr_item_sk#97, wr_order_number#98, 5), ENSURE_REQUIREMENTS, [id=#101] (55) Sort [codegen id : 15] Input [5]: [wr_returned_date_sk#96, wr_item_sk#97, wr_order_number#98, wr_return_amt#99, wr_net_loss#100] @@ -346,7 +346,7 @@ Condition : ((isnotnull(ws_item_sk#102) AND isnotnull(ws_order_number#103)) AND (59) Exchange Input [3]: [ws_item_sk#102, ws_web_site_sk#87, ws_order_number#103] -Arguments: hashpartitioning(cast(ws_item_sk#102 as bigint), cast(ws_order_number#103 as bigint), 5), true, [id=#104] +Arguments: hashpartitioning(cast(ws_item_sk#102 as bigint), cast(ws_order_number#103 as bigint), 5), ENSURE_REQUIREMENTS, [id=#104] (60) Sort [codegen id : 17] Input [3]: [ws_item_sk#102, ws_web_site_sk#87, ws_order_number#103] @@ -411,19 +411,19 @@ Results [5]: [web_site_id#112, sum#118, sum#119, sum#120, sum#121] (74) Exchange Input [5]: [web_site_id#112, sum#118, sum#119, sum#120, sum#121] -Arguments: hashpartitioning(web_site_id#112, 5), true, [id=#122] +Arguments: hashpartitioning(web_site_id#112, 5), ENSURE_REQUIREMENTS, [id=#122] (75) HashAggregate [codegen id : 22] Input [5]: [web_site_id#112, sum#118, sum#119, sum#120, sum#121] Keys [1]: [web_site_id#112] Functions [4]: [sum(UnscaledValue(sales_price#92)), sum(UnscaledValue(return_amt#94)), sum(UnscaledValue(profit#93)), sum(UnscaledValue(net_loss#95))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#92))#123, sum(UnscaledValue(return_amt#94))#124, sum(UnscaledValue(profit#93))#125, sum(UnscaledValue(net_loss#95))#126] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#92))#123,17,2) AS sales#127, MakeDecimal(sum(UnscaledValue(return_amt#94))#124,17,2) AS RETURNS#128, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#93))#125,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#95))#126,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#129, web channel AS channel#130, concat(web_site, web_site_id#112) AS id#131] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#92))#123,17,2) AS sales#127, MakeDecimal(sum(UnscaledValue(return_amt#94))#124,17,2) AS returns#128, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#93))#125,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#95))#126,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#129, web channel AS channel#130, concat(web_site, web_site_id#112) AS id#131] (76) Union (77) Expand [codegen id : 23] -Input [5]: [sales#40, RETURNS#41, profit#42, channel#43, id#44] +Input [5]: [sales#40, returns#41, profit#42, channel#43, id#44] Arguments: [List(sales#40, returns#41, profit#42, channel#43, id#44, 0), List(sales#40, returns#41, profit#42, channel#43, null, 1), List(sales#40, returns#41, profit#42, null, null, 3)], [sales#40, returns#41, profit#42, channel#132, id#133, spark_grouping_id#134] (78) HashAggregate [codegen id : 23] @@ -435,7 +435,7 @@ Results [9]: [channel#132, id#133, spark_grouping_id#134, sum#141, isEmpty#142, (79) Exchange Input [9]: [channel#132, id#133, spark_grouping_id#134, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Arguments: hashpartitioning(channel#132, id#133, spark_grouping_id#134, 5), true, [id=#147] +Arguments: hashpartitioning(channel#132, id#133, spark_grouping_id#134, 5), ENSURE_REQUIREMENTS, [id=#147] (80) HashAggregate [codegen id : 24] Input [9]: [channel#132, id#133, spark_grouping_id#134, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt index 80b07a3712d36..25102fb37a80f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter Union WholeStageCodegen (6) - HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,RETURNS,profit,channel,id,sum,sum,sum,sum] + HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] InputAdapter Exchange [s_store_id] #2 WholeStageCodegen (5) @@ -48,7 +48,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter Scan parquet default.store [s_store_sk,s_store_id] WholeStageCodegen (12) - 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] + 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] InputAdapter Exchange [cp_catalog_page_id] #5 WholeStageCodegen (11) @@ -81,7 +81,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter Scan parquet default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (22) - 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] + HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] InputAdapter Exchange [web_site_id] #7 WholeStageCodegen (21) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt index 15f0cda0b5f9f..602013322fde7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt @@ -183,14 +183,14 @@ Results [5]: [s_store_id#25, sum#31, sum#32, sum#33, sum#34] (24) Exchange Input [5]: [s_store_id#25, sum#31, sum#32, sum#33, sum#34] -Arguments: hashpartitioning(s_store_id#25, 5), true, [id=#35] +Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, [id=#35] (25) HashAggregate [codegen id : 6] Input [5]: [s_store_id#25, sum#31, sum#32, sum#33, sum#34] Keys [1]: [s_store_id#25] Functions [4]: [sum(UnscaledValue(sales_price#7)), sum(UnscaledValue(return_amt#9)), sum(UnscaledValue(profit#8)), sum(UnscaledValue(net_loss#10))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#7))#36, sum(UnscaledValue(return_amt#9))#37, sum(UnscaledValue(profit#8))#38, sum(UnscaledValue(net_loss#10))#39] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#7))#36,17,2) AS sales#40, MakeDecimal(sum(UnscaledValue(return_amt#9))#37,17,2) AS RETURNS#41, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#8))#38,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#10))#39,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#42, store channel AS channel#43, concat(store, s_store_id#25) AS id#44] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#7))#36,17,2) AS sales#40, MakeDecimal(sum(UnscaledValue(return_amt#9))#37,17,2) AS returns#41, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#8))#38,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#10))#39,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#42, store channel AS channel#43, concat(store, s_store_id#25) AS id#44] (26) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_ext_sales_price#47, cs_net_profit#48] @@ -278,14 +278,14 @@ Results [5]: [cp_catalog_page_id#66, sum#72, sum#73, sum#74, sum#75] (45) Exchange Input [5]: [cp_catalog_page_id#66, sum#72, sum#73, sum#74, sum#75] -Arguments: hashpartitioning(cp_catalog_page_id#66, 5), true, [id=#76] +Arguments: hashpartitioning(cp_catalog_page_id#66, 5), ENSURE_REQUIREMENTS, [id=#76] (46) HashAggregate [codegen id : 12] Input [5]: [cp_catalog_page_id#66, sum#72, sum#73, sum#74, sum#75] Keys [1]: [cp_catalog_page_id#66] Functions [4]: [sum(UnscaledValue(sales_price#51)), sum(UnscaledValue(return_amt#53)), sum(UnscaledValue(profit#52)), sum(UnscaledValue(net_loss#54))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#51))#77, sum(UnscaledValue(return_amt#53))#78, sum(UnscaledValue(profit#52))#79, sum(UnscaledValue(net_loss#54))#80] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#51))#77,17,2) AS sales#81, MakeDecimal(sum(UnscaledValue(return_amt#53))#78,17,2) AS RETURNS#82, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#52))#79,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#54))#80,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#83, catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#66) AS id#85] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#51))#77,17,2) AS sales#81, MakeDecimal(sum(UnscaledValue(return_amt#53))#78,17,2) AS returns#82, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#52))#79,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#54))#80,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#83, catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#66) AS id#85] (47) Scan parquet default.web_sales Output [4]: [ws_sold_date_sk#86, ws_web_site_sk#87, ws_ext_sales_price#88, ws_net_profit#89] @@ -396,19 +396,19 @@ Results [5]: [web_site_id#111, sum#117, sum#118, sum#119, sum#120] (71) Exchange Input [5]: [web_site_id#111, sum#117, sum#118, sum#119, sum#120] -Arguments: hashpartitioning(web_site_id#111, 5), true, [id=#121] +Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, [id=#121] (72) HashAggregate [codegen id : 19] Input [5]: [web_site_id#111, sum#117, sum#118, sum#119, sum#120] Keys [1]: [web_site_id#111] Functions [4]: [sum(UnscaledValue(sales_price#92)), sum(UnscaledValue(return_amt#94)), sum(UnscaledValue(profit#93)), sum(UnscaledValue(net_loss#95))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#92))#122, sum(UnscaledValue(return_amt#94))#123, sum(UnscaledValue(profit#93))#124, sum(UnscaledValue(net_loss#95))#125] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#92))#122,17,2) AS sales#126, MakeDecimal(sum(UnscaledValue(return_amt#94))#123,17,2) AS RETURNS#127, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#93))#124,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#95))#125,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#128, web channel AS channel#129, concat(web_site, web_site_id#111) AS id#130] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#92))#122,17,2) AS sales#126, MakeDecimal(sum(UnscaledValue(return_amt#94))#123,17,2) AS returns#127, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#93))#124,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#95))#125,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#128, web channel AS channel#129, concat(web_site, web_site_id#111) AS id#130] (73) Union (74) Expand [codegen id : 20] -Input [5]: [sales#40, RETURNS#41, profit#42, channel#43, id#44] +Input [5]: [sales#40, returns#41, profit#42, channel#43, id#44] Arguments: [List(sales#40, returns#41, profit#42, channel#43, id#44, 0), List(sales#40, returns#41, profit#42, channel#43, null, 1), List(sales#40, returns#41, profit#42, null, null, 3)], [sales#40, returns#41, profit#42, channel#131, id#132, spark_grouping_id#133] (75) HashAggregate [codegen id : 20] @@ -420,7 +420,7 @@ Results [9]: [channel#131, id#132, spark_grouping_id#133, sum#140, isEmpty#141, (76) Exchange Input [9]: [channel#131, id#132, spark_grouping_id#133, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] -Arguments: hashpartitioning(channel#131, id#132, spark_grouping_id#133, 5), true, [id=#146] +Arguments: hashpartitioning(channel#131, id#132, spark_grouping_id#133, 5), ENSURE_REQUIREMENTS, [id=#146] (77) HashAggregate [codegen id : 21] Input [9]: [channel#131, id#132, spark_grouping_id#133, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt index 9b7cc3360367c..6f38d3158ef2d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter Union WholeStageCodegen (6) - HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,RETURNS,profit,channel,id,sum,sum,sum,sum] + HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] InputAdapter Exchange [s_store_id] #2 WholeStageCodegen (5) @@ -48,7 +48,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter Scan parquet default.store [s_store_sk,s_store_id] WholeStageCodegen (12) - 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] + 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] InputAdapter Exchange [cp_catalog_page_id] #5 WholeStageCodegen (11) @@ -81,7 +81,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter Scan parquet default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (19) - 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] + HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] InputAdapter Exchange [web_site_id] #7 WholeStageCodegen (18) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/explain.txt index 6492918d3aa13..036de304b72cc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/explain.txt @@ -6,117 +6,117 @@ TakeOrderedAndProject (21) +- * Project (17) +- * BroadcastHashJoin Inner BuildRight (16) :- * Project (10) - : +- * BroadcastHashJoin Inner BuildLeft (9) - : :- BroadcastExchange (5) - : : +- * Project (4) - : : +- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.date_dim (1) - : +- * Filter (8) - : +- * ColumnarToRow (7) - : +- Scan parquet default.store_sales (6) + : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.store_sales (1) + : +- BroadcastExchange (8) + : +- * Project (7) + : +- * Filter (6) + : +- * ColumnarToRow (5) + : +- Scan parquet default.item (4) +- BroadcastExchange (15) +- * Project (14) +- * Filter (13) +- * ColumnarToRow (12) - +- Scan parquet default.item (11) + +- Scan parquet default.date_dim (11) -(1) Scan parquet default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] +(1) Scan parquet default.store_sales +Output [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +(2) ColumnarToRow [codegen id : 3] +Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] -(3) Filter [codegen id : 1] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) +(3) Filter [codegen id : 3] +Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +Condition : (isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_item_sk#2)) -(4) Project [codegen id : 1] -Output [2]: [d_date_sk#1, d_year#2] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +(4) Scan parquet default.item +Output [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] +ReadSchema: struct -(5) BroadcastExchange -Input [2]: [d_date_sk#1, d_year#2] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#4] +(5) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] -(6) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct +(6) Filter [codegen id : 1] +Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] +Condition : ((isnotnull(i_manager_id#7) AND (i_manager_id#7 = 1)) AND isnotnull(i_item_sk#4)) -(7) ColumnarToRow -Input [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] +(7) Project [codegen id : 1] +Output [3]: [i_item_sk#4, i_brand_id#5, i_brand#6] +Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] -(8) Filter -Input [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] -Condition : (isnotnull(ss_sold_date_sk#5) AND isnotnull(ss_item_sk#6)) +(8) BroadcastExchange +Input [3]: [i_item_sk#4, i_brand_id#5, i_brand#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] (9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#5] +Left keys [1]: [ss_item_sk#2] +Right keys [1]: [i_item_sk#4] Join condition: None (10) Project [codegen id : 3] -Output [3]: [d_year#2, ss_item_sk#6, ss_ext_sales_price#7] -Input [5]: [d_date_sk#1, d_year#2, ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] +Output [4]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_brand_id#5, i_brand#6] +Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3, i_item_sk#4, i_brand_id#5, i_brand#6] -(11) Scan parquet default.item -Output [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] +(11) Scan parquet default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct (12) ColumnarToRow [codegen id : 2] -Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (13) Filter [codegen id : 2] -Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] -Condition : ((isnotnull(i_manager_id#11) AND (i_manager_id#11 = 1)) AND isnotnull(i_item_sk#8)) +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : ((((isnotnull(d_moy#11) AND isnotnull(d_year#10)) AND (d_moy#11 = 11)) AND (d_year#10 = 2000)) AND isnotnull(d_date_sk#9)) (14) Project [codegen id : 2] -Output [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] -Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] +Output [2]: [d_date_sk#9, d_year#10] +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (15) BroadcastExchange -Input [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] +Input [2]: [d_date_sk#9, d_year#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] (16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#6] -Right keys [1]: [i_item_sk#8] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#9] Join condition: None (17) Project [codegen id : 3] -Output [4]: [d_year#2, ss_ext_sales_price#7, i_brand_id#9, i_brand#10] -Input [6]: [d_year#2, ss_item_sk#6, ss_ext_sales_price#7, i_item_sk#8, i_brand_id#9, i_brand#10] +Output [4]: [d_year#10, ss_ext_sales_price#3, i_brand_id#5, i_brand#6] +Input [6]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_brand_id#5, i_brand#6, d_date_sk#9, d_year#10] (18) HashAggregate [codegen id : 3] -Input [4]: [d_year#2, ss_ext_sales_price#7, i_brand_id#9, i_brand#10] -Keys [3]: [d_year#2, i_brand#10, i_brand_id#9] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] +Input [4]: [d_year#10, ss_ext_sales_price#3, i_brand_id#5, i_brand#6] +Keys [3]: [d_year#10, i_brand#6, i_brand_id#5] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum#13] -Results [4]: [d_year#2, i_brand#10, i_brand_id#9, sum#14] +Results [4]: [d_year#10, i_brand#6, i_brand_id#5, sum#14] (19) Exchange -Input [4]: [d_year#2, i_brand#10, i_brand_id#9, sum#14] -Arguments: hashpartitioning(d_year#2, i_brand#10, i_brand_id#9, 5), ENSURE_REQUIREMENTS, [id=#15] +Input [4]: [d_year#10, i_brand#6, i_brand_id#5, sum#14] +Arguments: hashpartitioning(d_year#10, i_brand#6, i_brand_id#5, 5), ENSURE_REQUIREMENTS, [id=#15] (20) HashAggregate [codegen id : 4] -Input [4]: [d_year#2, i_brand#10, i_brand_id#9, sum#14] -Keys [3]: [d_year#2, i_brand#10, i_brand_id#9] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#7))#16] -Results [4]: [d_year#2, i_brand_id#9 AS brand_id#17, i_brand#10 AS brand#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#7))#16,17,2) AS ext_price#19] +Input [4]: [d_year#10, i_brand#6, i_brand_id#5, sum#14] +Keys [3]: [d_year#10, i_brand#6, i_brand_id#5] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#16] +Results [4]: [d_year#10, i_brand_id#5 AS brand_id#17, i_brand#6 AS brand#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#16,17,2) AS ext_price#19] (21) TakeOrderedAndProject -Input [4]: [d_year#2, brand_id#17, brand#18, ext_price#19] -Arguments: 100, [d_year#2 ASC NULLS FIRST, ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#2, brand_id#17, brand#18, ext_price#19] +Input [4]: [d_year#10, brand_id#17, brand#18, ext_price#19] +Arguments: 100, [d_year#10 ASC NULLS FIRST, ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#10, brand_id#17, brand#18, ext_price#19] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/simplified.txt index f4aaf3df75135..8ed500d84390c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/simplified.txt @@ -6,26 +6,26 @@ TakeOrderedAndProject [d_year,ext_price,brand_id,brand] WholeStageCodegen (3) HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] Project [d_year,ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [d_year,ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_sold_date_sk,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Project [d_date_sk,d_year] - Filter [d_moy,d_year,d_date_sk] + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - Filter [ss_sold_date_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [i_item_sk,i_brand_id,i_brand] - Filter [i_manager_id,i_item_sk] + Project [d_date_sk,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/explain.txt index b8d8aa358d532..cca43a4232a8a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/explain.txt @@ -6,115 +6,115 @@ TakeOrderedAndProject (21) +- * Project (17) +- * BroadcastHashJoin Inner BuildRight (16) :- * Project (10) - : +- * BroadcastHashJoin Inner BuildLeft (9) - : :- BroadcastExchange (5) - : : +- * Project (4) - : : +- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.date_dim (1) - : +- * Filter (8) - : +- * ColumnarToRow (7) - : +- Scan parquet default.store_sales (6) + : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.store_sales (1) + : +- BroadcastExchange (8) + : +- * Project (7) + : +- * Filter (6) + : +- * ColumnarToRow (5) + : +- Scan parquet default.item (4) +- BroadcastExchange (15) +- * Project (14) +- * Filter (13) +- * ColumnarToRow (12) - +- Scan parquet default.item (11) + +- Scan parquet default.date_dim (11) -(1) Scan parquet default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] +(1) Scan parquet default.store_sales +Output [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +(2) ColumnarToRow [codegen id : 3] +Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] -(3) Filter [codegen id : 1] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1999)) AND isnotnull(d_date_sk#1)) +(3) Filter [codegen id : 3] +Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +Condition : (isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_item_sk#2)) -(4) Project [codegen id : 1] -Output [1]: [d_date_sk#1] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +(4) Scan parquet default.item +Output [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,28), IsNotNull(i_item_sk)] +ReadSchema: struct -(5) BroadcastExchange -Input [1]: [d_date_sk#1] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#4] +(5) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] -(6) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct +(6) Filter [codegen id : 1] +Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] +Condition : ((isnotnull(i_manager_id#7) AND (i_manager_id#7 = 28)) AND isnotnull(i_item_sk#4)) -(7) ColumnarToRow -Input [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] +(7) Project [codegen id : 1] +Output [3]: [i_item_sk#4, i_brand_id#5, i_brand#6] +Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] -(8) Filter -Input [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] -Condition : (isnotnull(ss_sold_date_sk#5) AND isnotnull(ss_item_sk#6)) +(8) BroadcastExchange +Input [3]: [i_item_sk#4, i_brand_id#5, i_brand#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] (9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#5] +Left keys [1]: [ss_item_sk#2] +Right keys [1]: [i_item_sk#4] Join condition: None (10) Project [codegen id : 3] -Output [2]: [ss_item_sk#6, ss_ext_sales_price#7] -Input [4]: [d_date_sk#1, ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] +Output [4]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_brand_id#5, i_brand#6] +Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3, i_item_sk#4, i_brand_id#5, i_brand#6] -(11) Scan parquet default.item -Output [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] +(11) Scan parquet default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,28), IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +ReadSchema: struct (12) ColumnarToRow [codegen id : 2] -Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (13) Filter [codegen id : 2] -Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] -Condition : ((isnotnull(i_manager_id#11) AND (i_manager_id#11 = 28)) AND isnotnull(i_item_sk#8)) +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : ((((isnotnull(d_moy#11) AND isnotnull(d_year#10)) AND (d_moy#11 = 11)) AND (d_year#10 = 1999)) AND isnotnull(d_date_sk#9)) (14) Project [codegen id : 2] -Output [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] -Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] +Output [1]: [d_date_sk#9] +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (15) BroadcastExchange -Input [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] +Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] (16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#6] -Right keys [1]: [i_item_sk#8] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#9] Join condition: None (17) Project [codegen id : 3] -Output [3]: [ss_ext_sales_price#7, i_brand_id#9, i_brand#10] -Input [5]: [ss_item_sk#6, ss_ext_sales_price#7, i_item_sk#8, i_brand_id#9, i_brand#10] +Output [3]: [ss_ext_sales_price#3, i_brand_id#5, i_brand#6] +Input [5]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_brand_id#5, i_brand#6, d_date_sk#9] (18) HashAggregate [codegen id : 3] -Input [3]: [ss_ext_sales_price#7, i_brand_id#9, i_brand#10] -Keys [2]: [i_brand#10, i_brand_id#9] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] +Input [3]: [ss_ext_sales_price#3, i_brand_id#5, i_brand#6] +Keys [2]: [i_brand#6, i_brand_id#5] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum#13] -Results [3]: [i_brand#10, i_brand_id#9, sum#14] +Results [3]: [i_brand#6, i_brand_id#5, sum#14] (19) Exchange -Input [3]: [i_brand#10, i_brand_id#9, sum#14] -Arguments: hashpartitioning(i_brand#10, i_brand_id#9, 5), ENSURE_REQUIREMENTS, [id=#15] +Input [3]: [i_brand#6, i_brand_id#5, sum#14] +Arguments: hashpartitioning(i_brand#6, i_brand_id#5, 5), ENSURE_REQUIREMENTS, [id=#15] (20) HashAggregate [codegen id : 4] -Input [3]: [i_brand#10, i_brand_id#9, sum#14] -Keys [2]: [i_brand#10, i_brand_id#9] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#7))#16] -Results [3]: [i_brand_id#9 AS brand_id#17, i_brand#10 AS brand#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#7))#16,17,2) AS ext_price#19] +Input [3]: [i_brand#6, i_brand_id#5, sum#14] +Keys [2]: [i_brand#6, i_brand_id#5] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#16] +Results [3]: [i_brand_id#5 AS brand_id#17, i_brand#6 AS brand#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#16,17,2) AS ext_price#19] (21) TakeOrderedAndProject Input [3]: [brand_id#17, brand#18, ext_price#19] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/simplified.txt index 4f375c80678e8..b0d0e0d809441 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/simplified.txt @@ -6,26 +6,26 @@ TakeOrderedAndProject [ext_price,brand_id,brand] WholeStageCodegen (3) HashAggregate [i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_sold_date_sk,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - Filter [ss_sold_date_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [i_item_sk,i_brand_id,i_brand] - Filter [i_manager_id,i_item_sk] + Project [d_date_sk] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt index 3007b11a1a860..3a61d77f674f0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt @@ -11,60 +11,60 @@ TakeOrderedAndProject (79) : +- * BroadcastHashJoin LeftOuter BuildRight (65) : :- * Project (60) : : +- * SortMergeJoin Inner (59) - : : :- * Sort (34) - : : : +- Exchange (33) - : : : +- * Project (32) - : : : +- * SortMergeJoin Inner (31) - : : : :- * Sort (25) - : : : : +- Exchange (24) - : : : : +- * Project (23) - : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : :- * Project (17) - : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet default.catalog_sales (1) - : : : : : : +- BroadcastExchange (8) - : : : : : : +- * Project (7) - : : : : : : +- * Filter (6) - : : : : : : +- * ColumnarToRow (5) - : : : : : : +- Scan parquet default.household_demographics (4) - : : : : : +- BroadcastExchange (15) - : : : : : +- * Project (14) - : : : : : +- * Filter (13) - : : : : : +- * ColumnarToRow (12) - : : : : : +- Scan parquet default.customer_demographics (11) - : : : : +- BroadcastExchange (21) - : : : : +- * Filter (20) - : : : : +- * ColumnarToRow (19) - : : : : +- Scan parquet default.date_dim (18) - : : : +- * Sort (30) - : : : +- Exchange (29) - : : : +- * Filter (28) - : : : +- * ColumnarToRow (27) - : : : +- Scan parquet default.item (26) + : : :- * Sort (47) + : : : +- Exchange (46) + : : : +- * Project (45) + : : : +- * BroadcastHashJoin Inner BuildRight (44) + : : : :- * Project (32) + : : : : +- * SortMergeJoin Inner (31) + : : : : :- * Sort (25) + : : : : : +- Exchange (24) + : : : : : +- * Project (23) + : : : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : : : :- * Project (17) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : : : : :- * Project (10) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : : : :- * Filter (3) + : : : : : : : : +- * ColumnarToRow (2) + : : : : : : : : +- Scan parquet default.catalog_sales (1) + : : : : : : : +- BroadcastExchange (8) + : : : : : : : +- * Project (7) + : : : : : : : +- * Filter (6) + : : : : : : : +- * ColumnarToRow (5) + : : : : : : : +- Scan parquet default.household_demographics (4) + : : : : : : +- BroadcastExchange (15) + : : : : : : +- * Project (14) + : : : : : : +- * Filter (13) + : : : : : : +- * ColumnarToRow (12) + : : : : : : +- Scan parquet default.customer_demographics (11) + : : : : : +- BroadcastExchange (21) + : : : : : +- * Filter (20) + : : : : : +- * ColumnarToRow (19) + : : : : : +- Scan parquet default.date_dim (18) + : : : : +- * Sort (30) + : : : : +- Exchange (29) + : : : : +- * Filter (28) + : : : : +- * ColumnarToRow (27) + : : : : +- Scan parquet default.item (26) + : : : +- BroadcastExchange (43) + : : : +- * Project (42) + : : : +- * BroadcastHashJoin Inner BuildLeft (41) + : : : :- BroadcastExchange (37) + : : : : +- * Project (36) + : : : : +- * Filter (35) + : : : : +- * ColumnarToRow (34) + : : : : +- Scan parquet default.date_dim (33) + : : : +- * Filter (40) + : : : +- * ColumnarToRow (39) + : : : +- Scan parquet default.date_dim (38) : : +- * Sort (58) : : +- Exchange (57) : : +- * Project (56) : : +- * BroadcastHashJoin Inner BuildRight (55) - : : :- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildLeft (49) - : : : :- BroadcastExchange (45) - : : : : +- * Project (44) - : : : : +- * BroadcastHashJoin Inner BuildLeft (43) - : : : : :- BroadcastExchange (39) - : : : : : +- * Project (38) - : : : : : +- * Filter (37) - : : : : : +- * ColumnarToRow (36) - : : : : : +- Scan parquet default.date_dim (35) - : : : : +- * Filter (42) - : : : : +- * ColumnarToRow (41) - : : : : +- Scan parquet default.date_dim (40) - : : : +- * Filter (48) - : : : +- * ColumnarToRow (47) - : : : +- Scan parquet default.inventory (46) + : : :- * Filter (50) + : : : +- * ColumnarToRow (49) + : : : +- Scan parquet default.inventory (48) : : +- BroadcastExchange (54) : : +- * Filter (53) : : +- * ColumnarToRow (52) @@ -98,7 +98,7 @@ Condition : (((((isnotnull(cs_quantity#8) AND isnotnull(cs_item_sk#5)) AND isnot Output [2]: [hd_demo_sk#9, hd_buy_potential#10] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000 ), IsNotNull(hd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -106,7 +106,7 @@ Input [2]: [hd_demo_sk#9, hd_buy_potential#10] (6) Filter [codegen id : 1] Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = >10000)) AND isnotnull(hd_demo_sk#9)) +Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = >10000 )) AND isnotnull(hd_demo_sk#9)) (7) Project [codegen id : 1] Output [1]: [hd_demo_sk#9] @@ -213,95 +213,95 @@ Arguments: hashpartitioning(i_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#21] Input [2]: [i_item_sk#19, i_item_desc#20] Arguments: [i_item_sk#19 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin [codegen id : 8] +(31) SortMergeJoin [codegen id : 10] Left keys [1]: [cs_item_sk#5] Right keys [1]: [i_item_sk#19] Join condition: None -(32) Project [codegen id : 8] +(32) Project [codegen id : 10] Output [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] Input [8]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_sk#19, i_item_desc#20] -(33) Exchange -Input [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] -Arguments: hashpartitioning(cs_item_sk#5, cs_sold_date_sk#1, 5), ENSURE_REQUIREMENTS, [id=#22] - -(34) Sort [codegen id : 9] -Input [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] -Arguments: [cs_item_sk#5 ASC NULLS FIRST, cs_sold_date_sk#1 ASC NULLS FIRST], false, 0 - -(35) Scan parquet default.date_dim -Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] +(33) Scan parquet default.date_dim +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct -(36) ColumnarToRow [codegen id : 10] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] +(34) ColumnarToRow [codegen id : 8] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -(37) Filter [codegen id : 10] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] -Condition : ((((isnotnull(d_year#26) AND (d_year#26 = 1999)) AND isnotnull(d_date_sk#23)) AND isnotnull(d_week_seq#25)) AND isnotnull(d_date#24)) +(35) Filter [codegen id : 8] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) -(38) Project [codegen id : 10] -Output [3]: [d_date_sk#23, d_date#24, d_week_seq#25] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] +(36) Project [codegen id : 8] +Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -(39) BroadcastExchange -Input [3]: [d_date_sk#23, d_date#24, d_week_seq#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#27] +(37) BroadcastExchange +Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#26] -(40) Scan parquet default.date_dim -Output [2]: [d_date_sk#28, d_week_seq#29] +(38) Scan parquet default.date_dim +Output [2]: [d_date_sk#27, d_week_seq#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(41) ColumnarToRow -Input [2]: [d_date_sk#28, d_week_seq#29] +(39) ColumnarToRow +Input [2]: [d_date_sk#27, d_week_seq#28] -(42) Filter -Input [2]: [d_date_sk#28, d_week_seq#29] -Condition : (isnotnull(d_week_seq#29) AND isnotnull(d_date_sk#28)) +(40) Filter +Input [2]: [d_date_sk#27, d_week_seq#28] +Condition : (isnotnull(d_week_seq#28) AND isnotnull(d_date_sk#27)) -(43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_week_seq#25] -Right keys [1]: [d_week_seq#29] +(41) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [d_week_seq#24] +Right keys [1]: [d_week_seq#28] Join condition: None -(44) Project [codegen id : 11] -Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28] -Input [5]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28, d_week_seq#29] +(42) Project [codegen id : 9] +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] +Input [5]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27, d_week_seq#28] + +(43) BroadcastExchange +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] + +(44) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_sold_date_sk#1] +Right keys [1]: [d_date_sk#22] +Join condition: (d_date#16 > d_date#23 + 5 days) + +(45) Project [codegen id : 10] +Output [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Input [11]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20, d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] -(45) BroadcastExchange -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[3, int, true] as bigint)),false), [id=#30] +(46) Exchange +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Arguments: hashpartitioning(cs_item_sk#5, d_date_sk#27, 5), ENSURE_REQUIREMENTS, [id=#30] -(46) Scan parquet default.inventory +(47) Sort [codegen id : 11] +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Arguments: [cs_item_sk#5 ASC NULLS FIRST, d_date_sk#27 ASC NULLS FIRST], false, 0 + +(48) Scan parquet default.inventory Output [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] Batched: true Location [not included in comparison]/{warehouse_dir}/inventory] PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk), IsNotNull(inv_date_sk)] ReadSchema: struct -(47) ColumnarToRow +(49) ColumnarToRow [codegen id : 13] Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] -(48) Filter +(50) Filter [codegen id : 13] Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] Condition : (((isnotnull(inv_quantity_on_hand#34) AND isnotnull(inv_item_sk#32)) AND isnotnull(inv_warehouse_sk#33)) AND isnotnull(inv_date_sk#31)) -(49) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [d_date_sk#28] -Right keys [1]: [inv_date_sk#31] -Join condition: None - -(50) Project [codegen id : 13] -Output [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] -Input [8]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28, inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] - (51) Scan parquet default.warehouse Output [2]: [w_warehouse_sk#35, w_warehouse_name#36] Batched: true @@ -326,25 +326,25 @@ Right keys [1]: [w_warehouse_sk#35] Join condition: None (56) Project [codegen id : 13] -Output [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Input [8]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, w_warehouse_sk#35, w_warehouse_name#36] +Output [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Input [6]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, w_warehouse_sk#35, w_warehouse_name#36] (57) Exchange -Input [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Arguments: hashpartitioning(inv_item_sk#32, d_date_sk#23, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Arguments: hashpartitioning(inv_item_sk#32, inv_date_sk#31, 5), ENSURE_REQUIREMENTS, [id=#38] (58) Sort [codegen id : 14] -Input [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Arguments: [inv_item_sk#32 ASC NULLS FIRST, d_date_sk#23 ASC NULLS FIRST], false, 0 +Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Arguments: [inv_item_sk#32 ASC NULLS FIRST, inv_date_sk#31 ASC NULLS FIRST], false, 0 (59) SortMergeJoin [codegen id : 16] -Left keys [2]: [cs_item_sk#5, cs_sold_date_sk#1] -Right keys [2]: [inv_item_sk#32, d_date_sk#23] -Join condition: ((inv_quantity_on_hand#34 < cs_quantity#8) AND (d_date#16 > d_date#24 + 5 days)) +Left keys [2]: [cs_item_sk#5, d_date_sk#27] +Right keys [2]: [inv_item_sk#32, inv_date_sk#31] +Join condition: (inv_quantity_on_hand#34 < cs_quantity#8) (60) Project [codegen id : 16] -Output [6]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] -Input [13]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20, d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Output [6]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Input [11]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27, inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] (61) Scan parquet default.promotion Output [1]: [p_promo_sk#39] @@ -370,15 +370,15 @@ Right keys [1]: [p_promo_sk#39] Join condition: None (66) Project [codegen id : 16] -Output [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] -Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25, p_promo_sk#39] +Output [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24, p_promo_sk#39] (67) Exchange -Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] Arguments: hashpartitioning(cs_item_sk#5, cs_order_number#7, 5), ENSURE_REQUIREMENTS, [id=#41] (68) Sort [codegen id : 17] -Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] Arguments: [cs_item_sk#5 ASC NULLS FIRST, cs_order_number#7 ASC NULLS FIRST], false, 0 (69) Scan parquet default.catalog_returns @@ -409,28 +409,28 @@ Right keys [2]: [cr_item_sk#42, cr_order_number#43] Join condition: None (75) Project [codegen id : 20] -Output [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#25] -Input [7]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25, cr_item_sk#42, cr_order_number#43] +Output [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Input [7]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24, cr_item_sk#42, cr_order_number#43] (76) HashAggregate [codegen id : 20] -Input [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#25] -Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25] +Input [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#45] -Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] +Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] (77) Exchange -Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] -Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#36, d_week_seq#25, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] +Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#36, d_week_seq#24, 5), ENSURE_REQUIREMENTS, [id=#47] (78) HashAggregate [codegen id : 21] -Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] -Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25] +Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] +Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#48] -Results [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count(1)#48 AS no_promo#49, count(1)#48 AS promo#50, count(1)#48 AS total_cnt#51] +Results [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count(1)#48 AS no_promo#49, count(1)#48 AS promo#50, count(1)#48 AS total_cnt#51] (79) TakeOrderedAndProject -Input [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, no_promo#49, promo#50, total_cnt#51] -Arguments: 100, [total_cnt#51 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#36 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, no_promo#49, promo#50, total_cnt#51] +Input [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, no_promo#49, promo#50, total_cnt#51] +Arguments: 100, [total_cnt#51 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#36 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, no_promo#49, promo#50, total_cnt#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt index b88505ad7b9bc..918508787c4b0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt @@ -16,95 +16,95 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] BroadcastHashJoin [cs_promo_sk,p_promo_sk] Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - SortMergeJoin [cs_item_sk,cs_sold_date_sk,inv_item_sk,d_date_sk,inv_quantity_on_hand,cs_quantity,d_date,d_date] + SortMergeJoin [cs_item_sk,d_date_sk,inv_item_sk,inv_date_sk,inv_quantity_on_hand,cs_quantity] InputAdapter - WholeStageCodegen (9) - Sort [cs_item_sk,cs_sold_date_sk] + WholeStageCodegen (11) + Sort [cs_item_sk,d_date_sk] InputAdapter - Exchange [cs_item_sk,cs_sold_date_sk] #3 - WholeStageCodegen (8) - Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date,i_item_desc] - SortMergeJoin [cs_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (5) - Sort [cs_item_sk] - InputAdapter - Exchange [cs_item_sk] #4 - WholeStageCodegen (4) - Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] - ColumnarToRow + Exchange [cs_item_sk,d_date_sk] #3 + WholeStageCodegen (10) + Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_week_seq,d_date_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date] + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date,i_item_desc] + SortMergeJoin [cs_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (5) + Sort [cs_item_sk] + InputAdapter + Exchange [cs_item_sk] #4 + WholeStageCodegen (4) + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastExchange #5 + WholeStageCodegen (1) + Project [hd_demo_sk] + Filter [hd_buy_potential,hd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk] + BroadcastExchange #6 + WholeStageCodegen (2) + Project [cd_demo_sk] + Filter [cd_marital_status,cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [cd_demo_sk] - Filter [cd_marital_status,cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + BroadcastExchange #7 + WholeStageCodegen (3) + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (7) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #8 + WholeStageCodegen (6) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_desc] InputAdapter - WholeStageCodegen (7) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #8 - WholeStageCodegen (6) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_desc] + BroadcastExchange #9 + WholeStageCodegen (9) + Project [d_date_sk,d_date,d_week_seq,d_date_sk] + BroadcastHashJoin [d_week_seq,d_week_seq] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + Project [d_date_sk,d_date,d_week_seq] + Filter [d_year,d_date_sk,d_week_seq,d_date] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + Filter [d_week_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter WholeStageCodegen (14) - Sort [inv_item_sk,d_date_sk] + Sort [inv_item_sk,inv_date_sk] InputAdapter - Exchange [inv_item_sk,d_date_sk] #9 + Exchange [inv_item_sk,inv_date_sk] #11 WholeStageCodegen (13) - Project [d_date_sk,d_date,d_week_seq,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] + Project [inv_date_sk,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [d_date_sk,d_date,d_week_seq,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - BroadcastHashJoin [d_date_sk,inv_date_sk] + Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #10 - WholeStageCodegen (11) - Project [d_date_sk,d_date,d_week_seq,d_date_sk] - BroadcastHashJoin [d_week_seq,d_week_seq] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (10) - Project [d_date_sk,d_date,d_week_seq] - Filter [d_year,d_date_sk,d_week_seq,d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - Filter [d_week_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq] - Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #12 WholeStageCodegen (12) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt index 6813696266ac5..f89e4b8d093f1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt @@ -1,343 +1,343 @@ == Physical Plan == TakeOrderedAndProject (61) +- * Project (60) - +- * SortMergeJoin Inner (59) - :- * Sort (47) - : +- Exchange (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Filter (23) - : : +- * HashAggregate (22) - : : +- Exchange (21) - : : +- * HashAggregate (20) - : : +- * Project (19) - : : +- * SortMergeJoin Inner (18) - : : :- * Sort (12) - : : : +- Exchange (11) - : : : +- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.catalog_returns (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- * Sort (17) - : : +- Exchange (16) - : : +- * Filter (15) - : : +- * ColumnarToRow (14) - : : +- Scan parquet default.customer_address (13) - : +- BroadcastExchange (43) - : +- * Filter (42) - : +- * HashAggregate (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- * HashAggregate (38) - : +- Exchange (37) - : +- * HashAggregate (36) - : +- * Project (35) - : +- * SortMergeJoin Inner (34) - : :- * Sort (31) - : : +- Exchange (30) - : : +- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * Filter (26) - : : : +- * ColumnarToRow (25) - : : : +- Scan parquet default.catalog_returns (24) - : : +- ReusedExchange (27) - : +- * Sort (33) - : +- ReusedExchange (32) - +- * Sort (58) - +- Exchange (57) - +- * Project (56) - +- * BroadcastHashJoin Inner BuildRight (55) - :- * Filter (50) - : +- * ColumnarToRow (49) - : +- Scan parquet default.customer (48) - +- BroadcastExchange (54) - +- * Filter (53) - +- * ColumnarToRow (52) - +- Scan parquet default.customer_address (51) - - -(1) Scan parquet default.catalog_returns -Output [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] + +- * BroadcastHashJoin Inner BuildRight (59) + :- * Project (38) + : +- * SortMergeJoin Inner (37) + : :- * Sort (11) + : : +- Exchange (10) + : : +- * Project (9) + : : +- * BroadcastHashJoin Inner BuildRight (8) + : : :- * Filter (3) + : : : +- * ColumnarToRow (2) + : : : +- Scan parquet default.customer (1) + : : +- BroadcastExchange (7) + : : +- * Filter (6) + : : +- * ColumnarToRow (5) + : : +- Scan parquet default.customer_address (4) + : +- * Sort (36) + : +- Exchange (35) + : +- * Filter (34) + : +- * HashAggregate (33) + : +- Exchange (32) + : +- * HashAggregate (31) + : +- * Project (30) + : +- * SortMergeJoin Inner (29) + : :- * Sort (23) + : : +- Exchange (22) + : : +- * Project (21) + : : +- * BroadcastHashJoin Inner BuildRight (20) + : : :- * Filter (14) + : : : +- * ColumnarToRow (13) + : : : +- Scan parquet default.catalog_returns (12) + : : +- BroadcastExchange (19) + : : +- * Project (18) + : : +- * Filter (17) + : : +- * ColumnarToRow (16) + : : +- Scan parquet default.date_dim (15) + : +- * Sort (28) + : +- Exchange (27) + : +- * Filter (26) + : +- * ColumnarToRow (25) + : +- Scan parquet default.customer_address (24) + +- BroadcastExchange (58) + +- * Filter (57) + +- * HashAggregate (56) + +- Exchange (55) + +- * HashAggregate (54) + +- * HashAggregate (53) + +- Exchange (52) + +- * HashAggregate (51) + +- * Project (50) + +- * SortMergeJoin Inner (49) + :- * Sort (46) + : +- Exchange (45) + : +- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) + : :- * Filter (41) + : : +- * ColumnarToRow (40) + : : +- Scan parquet default.catalog_returns (39) + : +- ReusedExchange (42) + +- * Sort (48) + +- ReusedExchange (47) + + +(1) Scan parquet default.customer +Output [6]: [c_customer_sk#1, c_customer_id#2, c_current_addr_sk#3, c_salutation#4, c_first_name#5, c_last_name#6] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 2] -Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Input [6]: [c_customer_sk#1, c_customer_id#2, c_current_addr_sk#3, c_salutation#4, c_first_name#5, c_last_name#6] (3) Filter [codegen id : 2] -Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -Condition : ((isnotnull(cr_returned_date_sk#1) AND isnotnull(cr_returning_addr_sk#3)) AND isnotnull(cr_returning_customer_sk#2)) +Input [6]: [c_customer_sk#1, c_customer_id#2, c_current_addr_sk#3, c_salutation#4, c_first_name#5, c_last_name#6] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_current_addr_sk#3)) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_year#6] +(4) Scan parquet default.customer_address +Output [12]: [ca_address_sk#7, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] +ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_year#6] +Input [12]: [ca_address_sk#7, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18] (6) Filter [codegen id : 1] -Input [2]: [d_date_sk#5, d_year#6] -Condition : ((isnotnull(d_year#6) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) - -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#5] -Input [2]: [d_date_sk#5, d_year#6] +Input [12]: [ca_address_sk#7, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18] +Condition : ((isnotnull(ca_state#14) AND (ca_state#14 = GA)) AND isnotnull(ca_address_sk#7)) -(8) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#7] +(7) BroadcastExchange +Input [12]: [ca_address_sk#7, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#19] -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [cr_returned_date_sk#1] -Right keys [1]: [d_date_sk#5] +(8) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [c_current_addr_sk#3] +Right keys [1]: [ca_address_sk#7] Join condition: None -(10) Project [codegen id : 2] -Output [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -Input [5]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4, d_date_sk#5] +(9) Project [codegen id : 2] +Output [16]: [c_customer_sk#1, c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18] +Input [18]: [c_customer_sk#1, c_customer_id#2, c_current_addr_sk#3, c_salutation#4, c_first_name#5, c_last_name#6, ca_address_sk#7, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18] -(11) Exchange -Input [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -Arguments: hashpartitioning(cr_returning_addr_sk#3, 5), ENSURE_REQUIREMENTS, [id=#8] +(10) Exchange +Input [16]: [c_customer_sk#1, c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18] +Arguments: hashpartitioning(c_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#20] -(12) Sort [codegen id : 3] -Input [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -Arguments: [cr_returning_addr_sk#3 ASC NULLS FIRST], false, 0 +(11) Sort [codegen id : 3] +Input [16]: [c_customer_sk#1, c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18] +Arguments: [c_customer_sk#1 ASC NULLS FIRST], false, 0 -(13) Scan parquet default.customer_address -Output [2]: [ca_address_sk#9, ca_state#10] +(12) Scan parquet default.catalog_returns +Output [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_state)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] +ReadSchema: struct -(14) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#9, ca_state#10] +(13) ColumnarToRow [codegen id : 5] +Input [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -(15) Filter [codegen id : 4] -Input [2]: [ca_address_sk#9, ca_state#10] -Condition : (isnotnull(ca_address_sk#9) AND isnotnull(ca_state#10)) +(14) Filter [codegen id : 5] +Input [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +Condition : ((isnotnull(cr_returned_date_sk#21) AND isnotnull(cr_returning_addr_sk#23)) AND isnotnull(cr_returning_customer_sk#22)) -(16) Exchange -Input [2]: [ca_address_sk#9, ca_state#10] -Arguments: hashpartitioning(ca_address_sk#9, 5), ENSURE_REQUIREMENTS, [id=#11] +(15) Scan parquet default.date_dim +Output [2]: [d_date_sk#25, d_year#26] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct -(17) Sort [codegen id : 5] -Input [2]: [ca_address_sk#9, ca_state#10] -Arguments: [ca_address_sk#9 ASC NULLS FIRST], false, 0 +(16) ColumnarToRow [codegen id : 4] +Input [2]: [d_date_sk#25, d_year#26] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [cr_returning_addr_sk#3] -Right keys [1]: [ca_address_sk#9] -Join condition: None +(17) Filter [codegen id : 4] +Input [2]: [d_date_sk#25, d_year#26] +Condition : ((isnotnull(d_year#26) AND (d_year#26 = 2000)) AND isnotnull(d_date_sk#25)) -(19) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#2, cr_return_amt_inc_tax#4, ca_state#10] -Input [5]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4, ca_address_sk#9, ca_state#10] - -(20) HashAggregate [codegen id : 6] -Input [3]: [cr_returning_customer_sk#2, cr_return_amt_inc_tax#4, ca_state#10] -Keys [2]: [cr_returning_customer_sk#2, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#4))] -Aggregate Attributes [1]: [sum#12] -Results [3]: [cr_returning_customer_sk#2, ca_state#10, sum#13] - -(21) Exchange -Input [3]: [cr_returning_customer_sk#2, ca_state#10, sum#13] -Arguments: hashpartitioning(cr_returning_customer_sk#2, ca_state#10, 5), ENSURE_REQUIREMENTS, [id=#14] - -(22) HashAggregate [codegen id : 15] -Input [3]: [cr_returning_customer_sk#2, ca_state#10, sum#13] -Keys [2]: [cr_returning_customer_sk#2, ca_state#10] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#4))#15] -Results [3]: [cr_returning_customer_sk#2 AS ctr_customer_sk#16, ca_state#10 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#4))#15,17,2) AS ctr_total_return#18] - -(23) Filter [codegen id : 15] -Input [3]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18] -Condition : isnotnull(ctr_total_return#18) - -(24) Scan parquet default.catalog_returns -Output [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_addr_sk)] -ReadSchema: struct +(18) Project [codegen id : 4] +Output [1]: [d_date_sk#25] +Input [2]: [d_date_sk#25, d_year#26] -(25) ColumnarToRow [codegen id : 8] -Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +(19) BroadcastExchange +Input [1]: [d_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] -(26) Filter [codegen id : 8] -Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -Condition : (isnotnull(cr_returned_date_sk#1) AND isnotnull(cr_returning_addr_sk#3)) +(20) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cr_returned_date_sk#21] +Right keys [1]: [d_date_sk#25] +Join condition: None -(27) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +(21) Project [codegen id : 5] +Output [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +Input [5]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24, d_date_sk#25] -(28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cr_returned_date_sk#1] -Right keys [1]: [d_date_sk#5] -Join condition: None +(22) Exchange +Input [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +Arguments: hashpartitioning(cr_returning_addr_sk#23, 5), ENSURE_REQUIREMENTS, [id=#28] -(29) Project [codegen id : 8] -Output [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -Input [5]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4, d_date_sk#5] +(23) Sort [codegen id : 6] +Input [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +Arguments: [cr_returning_addr_sk#23 ASC NULLS FIRST], false, 0 -(30) Exchange -Input [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -Arguments: hashpartitioning(cr_returning_addr_sk#3, 5), ENSURE_REQUIREMENTS, [id=#19] +(24) Scan parquet default.customer_address +Output [2]: [ca_address_sk#7, ca_state#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_state)] +ReadSchema: struct -(31) Sort [codegen id : 9] -Input [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -Arguments: [cr_returning_addr_sk#3 ASC NULLS FIRST], false, 0 +(25) ColumnarToRow [codegen id : 7] +Input [2]: [ca_address_sk#7, ca_state#14] -(32) ReusedExchange [Reuses operator id: 16] -Output [2]: [ca_address_sk#9, ca_state#10] +(26) Filter [codegen id : 7] +Input [2]: [ca_address_sk#7, ca_state#14] +Condition : (isnotnull(ca_address_sk#7) AND isnotnull(ca_state#14)) -(33) Sort [codegen id : 11] -Input [2]: [ca_address_sk#9, ca_state#10] -Arguments: [ca_address_sk#9 ASC NULLS FIRST], false, 0 +(27) Exchange +Input [2]: [ca_address_sk#7, ca_state#14] +Arguments: hashpartitioning(ca_address_sk#7, 5), ENSURE_REQUIREMENTS, [id=#29] -(34) SortMergeJoin [codegen id : 12] -Left keys [1]: [cr_returning_addr_sk#3] -Right keys [1]: [ca_address_sk#9] -Join condition: None +(28) Sort [codegen id : 8] +Input [2]: [ca_address_sk#7, ca_state#14] +Arguments: [ca_address_sk#7 ASC NULLS FIRST], false, 0 -(35) Project [codegen id : 12] -Output [3]: [cr_returning_customer_sk#2, cr_return_amt_inc_tax#4, ca_state#10] -Input [5]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4, ca_address_sk#9, ca_state#10] - -(36) HashAggregate [codegen id : 12] -Input [3]: [cr_returning_customer_sk#2, cr_return_amt_inc_tax#4, ca_state#10] -Keys [2]: [cr_returning_customer_sk#2, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#4))] -Aggregate Attributes [1]: [sum#20] -Results [3]: [cr_returning_customer_sk#2, ca_state#10, sum#21] - -(37) Exchange -Input [3]: [cr_returning_customer_sk#2, ca_state#10, sum#21] -Arguments: hashpartitioning(cr_returning_customer_sk#2, ca_state#10, 5), ENSURE_REQUIREMENTS, [id=#22] - -(38) HashAggregate [codegen id : 13] -Input [3]: [cr_returning_customer_sk#2, ca_state#10, sum#21] -Keys [2]: [cr_returning_customer_sk#2, ca_state#10] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#4))#23] -Results [2]: [ca_state#10 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#4))#23,17,2) AS ctr_total_return#18] - -(39) HashAggregate [codegen id : 13] -Input [2]: [ctr_state#17, ctr_total_return#18] -Keys [1]: [ctr_state#17] -Functions [1]: [partial_avg(ctr_total_return#18)] -Aggregate Attributes [2]: [sum#24, count#25] -Results [3]: [ctr_state#17, sum#26, count#27] - -(40) Exchange -Input [3]: [ctr_state#17, sum#26, count#27] -Arguments: hashpartitioning(ctr_state#17, 5), ENSURE_REQUIREMENTS, [id=#28] - -(41) HashAggregate [codegen id : 14] -Input [3]: [ctr_state#17, sum#26, count#27] -Keys [1]: [ctr_state#17] -Functions [1]: [avg(ctr_total_return#18)] -Aggregate Attributes [1]: [avg(ctr_total_return#18)#29] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#18)#29) * 1.200000), DecimalType(24,7), true) AS (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30, ctr_state#17 AS ctr_state#17#31] - -(42) Filter [codegen id : 14] -Input [2]: [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30, ctr_state#17#31] -Condition : isnotnull((CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30) - -(43) BroadcastExchange -Input [2]: [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30, ctr_state#17#31] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#32] - -(44) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ctr_state#17] -Right keys [1]: [ctr_state#17#31] -Join condition: (cast(ctr_total_return#18 as decimal(24,7)) > (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30) - -(45) Project [codegen id : 15] -Output [2]: [ctr_customer_sk#16, ctr_total_return#18] -Input [5]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18, (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30, ctr_state#17#31] - -(46) Exchange -Input [2]: [ctr_customer_sk#16, ctr_total_return#18] -Arguments: hashpartitioning(ctr_customer_sk#16, 5), ENSURE_REQUIREMENTS, [id=#33] - -(47) Sort [codegen id : 16] -Input [2]: [ctr_customer_sk#16, ctr_total_return#18] -Arguments: [ctr_customer_sk#16 ASC NULLS FIRST], false, 0 - -(48) Scan parquet default.customer -Output [6]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct +(29) SortMergeJoin [codegen id : 9] +Left keys [1]: [cr_returning_addr_sk#23] +Right keys [1]: [ca_address_sk#7] +Join condition: None -(49) ColumnarToRow [codegen id : 18] -Input [6]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39] +(30) Project [codegen id : 9] +Output [3]: [cr_returning_customer_sk#22, cr_return_amt_inc_tax#24, ca_state#14] +Input [5]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24, ca_address_sk#7, ca_state#14] + +(31) HashAggregate [codegen id : 9] +Input [3]: [cr_returning_customer_sk#22, cr_return_amt_inc_tax#24, ca_state#14] +Keys [2]: [cr_returning_customer_sk#22, ca_state#14] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#24))] +Aggregate Attributes [1]: [sum#30] +Results [3]: [cr_returning_customer_sk#22, ca_state#14, sum#31] + +(32) Exchange +Input [3]: [cr_returning_customer_sk#22, ca_state#14, sum#31] +Arguments: hashpartitioning(cr_returning_customer_sk#22, ca_state#14, 5), ENSURE_REQUIREMENTS, [id=#32] + +(33) HashAggregate [codegen id : 10] +Input [3]: [cr_returning_customer_sk#22, ca_state#14, sum#31] +Keys [2]: [cr_returning_customer_sk#22, ca_state#14] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#24))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#24))#33] +Results [3]: [cr_returning_customer_sk#22 AS ctr_customer_sk#34, ca_state#14 AS ctr_state#35, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#24))#33,17,2) AS ctr_total_return#36] + +(34) Filter [codegen id : 10] +Input [3]: [ctr_customer_sk#34, ctr_state#35, ctr_total_return#36] +Condition : isnotnull(ctr_total_return#36) + +(35) Exchange +Input [3]: [ctr_customer_sk#34, ctr_state#35, ctr_total_return#36] +Arguments: hashpartitioning(ctr_customer_sk#34, 5), ENSURE_REQUIREMENTS, [id=#37] + +(36) Sort [codegen id : 11] +Input [3]: [ctr_customer_sk#34, ctr_state#35, ctr_total_return#36] +Arguments: [ctr_customer_sk#34 ASC NULLS FIRST], false, 0 + +(37) SortMergeJoin [codegen id : 20] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ctr_customer_sk#34] +Join condition: None -(50) Filter [codegen id : 18] -Input [6]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39] -Condition : (isnotnull(c_customer_sk#34) AND isnotnull(c_current_addr_sk#36)) +(38) Project [codegen id : 20] +Output [17]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_state#35, ctr_total_return#36] +Input [19]: [c_customer_sk#1, c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_customer_sk#34, ctr_state#35, ctr_total_return#36] -(51) Scan parquet default.customer_address -Output [12]: [ca_address_sk#9, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +(39) Scan parquet default.catalog_returns +Output [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_addr_sk)] +ReadSchema: struct -(52) ColumnarToRow [codegen id : 17] -Input [12]: [ca_address_sk#9, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +(40) ColumnarToRow [codegen id : 13] +Input [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -(53) Filter [codegen id : 17] -Input [12]: [ca_address_sk#9, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] -Condition : ((isnotnull(ca_state#10) AND (ca_state#10 = GA)) AND isnotnull(ca_address_sk#9)) +(41) Filter [codegen id : 13] +Input [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +Condition : (isnotnull(cr_returned_date_sk#21) AND isnotnull(cr_returning_addr_sk#23)) -(54) BroadcastExchange -Input [12]: [ca_address_sk#9, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#50] +(42) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#25] -(55) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_addr_sk#36] -Right keys [1]: [ca_address_sk#9] +(43) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [cr_returned_date_sk#21] +Right keys [1]: [d_date_sk#25] Join condition: None -(56) Project [codegen id : 18] -Output [16]: [c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] -Input [18]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, ca_address_sk#9, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +(44) Project [codegen id : 13] +Output [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +Input [5]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24, d_date_sk#25] -(57) Exchange -Input [16]: [c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] -Arguments: hashpartitioning(c_customer_sk#34, 5), ENSURE_REQUIREMENTS, [id=#51] +(45) Exchange +Input [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +Arguments: hashpartitioning(cr_returning_addr_sk#23, 5), ENSURE_REQUIREMENTS, [id=#38] -(58) Sort [codegen id : 19] -Input [16]: [c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] -Arguments: [c_customer_sk#34 ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 14] +Input [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +Arguments: [cr_returning_addr_sk#23 ASC NULLS FIRST], false, 0 -(59) SortMergeJoin [codegen id : 20] -Left keys [1]: [ctr_customer_sk#16] -Right keys [1]: [c_customer_sk#34] +(47) ReusedExchange [Reuses operator id: 27] +Output [2]: [ca_address_sk#7, ca_state#14] + +(48) Sort [codegen id : 16] +Input [2]: [ca_address_sk#7, ca_state#14] +Arguments: [ca_address_sk#7 ASC NULLS FIRST], false, 0 + +(49) SortMergeJoin [codegen id : 17] +Left keys [1]: [cr_returning_addr_sk#23] +Right keys [1]: [ca_address_sk#7] Join condition: None +(50) Project [codegen id : 17] +Output [3]: [cr_returning_customer_sk#22, cr_return_amt_inc_tax#24, ca_state#14] +Input [5]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24, ca_address_sk#7, ca_state#14] + +(51) HashAggregate [codegen id : 17] +Input [3]: [cr_returning_customer_sk#22, cr_return_amt_inc_tax#24, ca_state#14] +Keys [2]: [cr_returning_customer_sk#22, ca_state#14] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#24))] +Aggregate Attributes [1]: [sum#39] +Results [3]: [cr_returning_customer_sk#22, ca_state#14, sum#40] + +(52) Exchange +Input [3]: [cr_returning_customer_sk#22, ca_state#14, sum#40] +Arguments: hashpartitioning(cr_returning_customer_sk#22, ca_state#14, 5), ENSURE_REQUIREMENTS, [id=#41] + +(53) HashAggregate [codegen id : 18] +Input [3]: [cr_returning_customer_sk#22, ca_state#14, sum#40] +Keys [2]: [cr_returning_customer_sk#22, ca_state#14] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#24))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#24))#42] +Results [2]: [ca_state#14 AS ctr_state#35, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#24))#42,17,2) AS ctr_total_return#36] + +(54) HashAggregate [codegen id : 18] +Input [2]: [ctr_state#35, ctr_total_return#36] +Keys [1]: [ctr_state#35] +Functions [1]: [partial_avg(ctr_total_return#36)] +Aggregate Attributes [2]: [sum#43, count#44] +Results [3]: [ctr_state#35, sum#45, count#46] + +(55) Exchange +Input [3]: [ctr_state#35, sum#45, count#46] +Arguments: hashpartitioning(ctr_state#35, 5), ENSURE_REQUIREMENTS, [id=#47] + +(56) HashAggregate [codegen id : 19] +Input [3]: [ctr_state#35, sum#45, count#46] +Keys [1]: [ctr_state#35] +Functions [1]: [avg(ctr_total_return#36)] +Aggregate Attributes [1]: [avg(ctr_total_return#36)#48] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#36)#48) * 1.200000), DecimalType(24,7), true) AS (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49, ctr_state#35 AS ctr_state#35#50] + +(57) Filter [codegen id : 19] +Input [2]: [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49, ctr_state#35#50] +Condition : isnotnull((CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49) + +(58) BroadcastExchange +Input [2]: [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49, ctr_state#35#50] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#51] + +(59) BroadcastHashJoin [codegen id : 20] +Left keys [1]: [ctr_state#35] +Right keys [1]: [ctr_state#35#50] +Join condition: (cast(ctr_total_return#36 as decimal(24,7)) > (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49) + (60) Project [codegen id : 20] -Output [16]: [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49, ctr_total_return#18] -Input [18]: [ctr_customer_sk#16, ctr_total_return#18, c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +Output [16]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_total_return#36] +Input [19]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_state#35, ctr_total_return#36, (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49, ctr_state#35#50] (61) TakeOrderedAndProject -Input [16]: [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49, ctr_total_return#18] -Arguments: 100, [c_customer_id#35 ASC NULLS FIRST, c_salutation#37 ASC NULLS FIRST, c_first_name#38 ASC NULLS FIRST, c_last_name#39 ASC NULLS FIRST, ca_street_number#40 ASC NULLS FIRST, ca_street_name#41 ASC NULLS FIRST, ca_street_type#42 ASC NULLS FIRST, ca_suite_number#43 ASC NULLS FIRST, ca_city#44 ASC NULLS FIRST, ca_county#45 ASC NULLS FIRST, ca_state#10 ASC NULLS FIRST, ca_zip#46 ASC NULLS FIRST, ca_country#47 ASC NULLS FIRST, ca_gmt_offset#48 ASC NULLS FIRST, ca_location_type#49 ASC NULLS FIRST, ctr_total_return#18 ASC NULLS FIRST], [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49, ctr_total_return#18] +Input [16]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_total_return#36] +Arguments: 100, [c_customer_id#2 ASC NULLS FIRST, c_salutation#4 ASC NULLS FIRST, c_first_name#5 ASC NULLS FIRST, c_last_name#6 ASC NULLS FIRST, ca_street_number#8 ASC NULLS FIRST, ca_street_name#9 ASC NULLS FIRST, ca_street_type#10 ASC NULLS FIRST, ca_suite_number#11 ASC NULLS FIRST, ca_city#12 ASC NULLS FIRST, ca_county#13 ASC NULLS FIRST, ca_state#14 ASC NULLS FIRST, ca_zip#15 ASC NULLS FIRST, ca_country#16 ASC NULLS FIRST, ca_gmt_offset#17 ASC NULLS FIRST, ca_location_type#18 ASC NULLS FIRST, ctr_total_return#36 ASC NULLS FIRST], [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_total_return#36] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt index 99677b6e39736..c603ab5194286 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt @@ -1,29 +1,48 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] WholeStageCodegen (20) Project [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] - SortMergeJoin [ctr_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (16) - Sort [ctr_customer_sk] - InputAdapter - Exchange [ctr_customer_sk] #1 - WholeStageCodegen (15) - Project [ctr_customer_sk,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] + BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] + Project [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_state,ctr_total_return] + SortMergeJoin [c_customer_sk,ctr_customer_sk] + InputAdapter + WholeStageCodegen (3) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #1 + WholeStageCodegen (2) + Project [c_customer_sk,c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Filter [c_customer_sk,c_current_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + Filter [ca_state,ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet 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] + InputAdapter + WholeStageCodegen (11) + Sort [ctr_customer_sk] + InputAdapter + Exchange [ctr_customer_sk] #3 + WholeStageCodegen (10) Filter [ctr_total_return] HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] InputAdapter - Exchange [cr_returning_customer_sk,ca_state] #2 - WholeStageCodegen (6) + Exchange [cr_returning_customer_sk,ca_state] #4 + WholeStageCodegen (9) HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] SortMergeJoin [cr_returning_addr_sk,ca_address_sk] InputAdapter - WholeStageCodegen (3) + WholeStageCodegen (6) Sort [cr_returning_addr_sk] InputAdapter - Exchange [cr_returning_addr_sk] #3 - WholeStageCodegen (2) + Exchange [cr_returning_addr_sk] #5 + WholeStageCodegen (5) Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] BroadcastHashJoin [cr_returned_date_sk,d_date_sk] Filter [cr_returned_date_sk,cr_returning_addr_sk,cr_returning_customer_sk] @@ -31,74 +50,55 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st InputAdapter Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) + BroadcastExchange #6 + WholeStageCodegen (4) Project [d_date_sk] Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter - WholeStageCodegen (5) + WholeStageCodegen (8) Sort [ca_address_sk] InputAdapter - Exchange [ca_address_sk] #5 - WholeStageCodegen (4) + Exchange [ca_address_sk] #7 + WholeStageCodegen (7) Filter [ca_address_sk,ca_state] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) - Filter [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] - HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),ctr_state,sum,count] - InputAdapter - Exchange [ctr_state] #7 - WholeStageCodegen (13) - HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] - HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] - InputAdapter - Exchange [cr_returning_customer_sk,ca_state] #8 - WholeStageCodegen (12) - HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] - Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - SortMergeJoin [cr_returning_addr_sk,ca_address_sk] - InputAdapter - WholeStageCodegen (9) - Sort [cr_returning_addr_sk] - InputAdapter - Exchange [cr_returning_addr_sk] #9 - WholeStageCodegen (8) - Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Filter [cr_returned_date_sk,cr_returning_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - WholeStageCodegen (11) - Sort [ca_address_sk] - InputAdapter - ReusedExchange [ca_address_sk,ca_state] #5 InputAdapter - WholeStageCodegen (19) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #10 - WholeStageCodegen (18) - Project [c_customer_sk,c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Filter [c_customer_sk,c_current_addr_sk] - ColumnarToRow + BroadcastExchange #8 + WholeStageCodegen (19) + Filter [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] + HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),ctr_state,sum,count] + InputAdapter + Exchange [ctr_state] #9 + WholeStageCodegen (18) + HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] + HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] InputAdapter - Scan parquet default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (17) - Filter [ca_state,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet 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] + Exchange [cr_returning_customer_sk,ca_state] #10 + WholeStageCodegen (17) + HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] + Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + SortMergeJoin [cr_returning_addr_sk,ca_address_sk] + InputAdapter + WholeStageCodegen (14) + Sort [cr_returning_addr_sk] + InputAdapter + Exchange [cr_returning_addr_sk] #11 + WholeStageCodegen (13) + Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + Filter [cr_returned_date_sk,cr_returning_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + WholeStageCodegen (16) + Sort [ca_address_sk] + InputAdapter + ReusedExchange [ca_address_sk,ca_state] #7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt index 377bd36f520eb..e635f0297d69f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt @@ -1,71 +1,72 @@ == Physical Plan == -* HashAggregate (67) -+- Exchange (66) - +- * HashAggregate (65) - +- * HashAggregate (64) - +- * HashAggregate (63) - +- * HashAggregate (62) - +- * HashAggregate (61) - +- * HashAggregate (60) - +- Exchange (59) - +- * HashAggregate (58) - +- SortMergeJoin LeftAnti (57) - :- SortMergeJoin LeftAnti (39) - : :- * Sort (21) - : : +- Exchange (20) - : : +- * Project (19) - : : +- * SortMergeJoin Inner (18) - : : :- * Sort (12) - : : : +- Exchange (11) - : : : +- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- * Sort (17) - : : +- Exchange (16) - : : +- * Filter (15) - : : +- * ColumnarToRow (14) - : : +- Scan parquet default.customer (13) - : +- * Sort (38) - : +- Exchange (37) - : +- * HashAggregate (36) - : +- Exchange (35) - : +- * HashAggregate (34) - : +- * Project (33) - : +- * SortMergeJoin Inner (32) - : :- * Sort (29) - : : +- Exchange (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Filter (24) - : : : +- * ColumnarToRow (23) - : : : +- Scan parquet default.catalog_sales (22) - : : +- ReusedExchange (25) - : +- * Sort (31) - : +- ReusedExchange (30) - +- * Sort (56) - +- Exchange (55) - +- * HashAggregate (54) - +- Exchange (53) - +- * HashAggregate (52) - +- * Project (51) - +- * SortMergeJoin Inner (50) - :- * Sort (47) - : +- Exchange (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Filter (42) - : : +- * ColumnarToRow (41) - : : +- Scan parquet default.web_sales (40) - : +- ReusedExchange (43) - +- * Sort (49) - +- ReusedExchange (48) +* HashAggregate (68) ++- Exchange (67) + +- * HashAggregate (66) + +- * HashAggregate (65) + +- * HashAggregate (64) + +- * HashAggregate (63) + +- * HashAggregate (62) + +- * HashAggregate (61) + +- Exchange (60) + +- * HashAggregate (59) + +- * Project (58) + +- SortMergeJoin LeftAnti (57) + :- SortMergeJoin LeftAnti (39) + : :- * Sort (21) + : : +- Exchange (20) + : : +- * Project (19) + : : +- * SortMergeJoin Inner (18) + : : :- * Sort (12) + : : : +- Exchange (11) + : : : +- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- * Sort (17) + : : +- Exchange (16) + : : +- * Filter (15) + : : +- * ColumnarToRow (14) + : : +- Scan parquet default.customer (13) + : +- * Sort (38) + : +- Exchange (37) + : +- * HashAggregate (36) + : +- Exchange (35) + : +- * HashAggregate (34) + : +- * Project (33) + : +- * SortMergeJoin Inner (32) + : :- * Sort (29) + : : +- Exchange (28) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Filter (24) + : : : +- * ColumnarToRow (23) + : : : +- Scan parquet default.catalog_sales (22) + : : +- ReusedExchange (25) + : +- * Sort (31) + : +- ReusedExchange (30) + +- * Sort (56) + +- Exchange (55) + +- * HashAggregate (54) + +- Exchange (53) + +- * HashAggregate (52) + +- * Project (51) + +- * SortMergeJoin Inner (50) + :- * Sort (47) + : +- Exchange (46) + : +- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Filter (42) + : : +- * ColumnarToRow (41) + : : +- Scan parquet default.web_sales (40) + : +- ReusedExchange (43) + +- * Sort (49) + +- ReusedExchange (48) (1) Scan parquet default.store_sales @@ -115,7 +116,7 @@ Input [4]: [ss_sold_date_sk#1, ss_customer_sk#2, d_date_sk#3, d_date#4] (11) Exchange Input [2]: [ss_customer_sk#2, d_date#4] -Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#7] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#7] (12) Sort [codegen id : 3] Input [2]: [ss_customer_sk#2, d_date#4] @@ -137,7 +138,7 @@ Condition : isnotnull(c_customer_sk#8) (16) Exchange Input [3]: [c_customer_sk#8, c_first_name#9, c_last_name#10] -Arguments: hashpartitioning(c_customer_sk#8, 5), true, [id=#11] +Arguments: hashpartitioning(c_customer_sk#8, 5), ENSURE_REQUIREMENTS, [id=#11] (17) Sort [codegen id : 5] Input [3]: [c_customer_sk#8, c_first_name#9, c_last_name#10] @@ -154,7 +155,7 @@ Input [5]: [ss_customer_sk#2, d_date#4, c_customer_sk#8, c_first_name#9, c_last_ (20) Exchange Input [3]: [d_date#4, c_first_name#9, c_last_name#10] -Arguments: hashpartitioning(coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#4, 0), isnull(d_date#4), 5), true, [id=#12] +Arguments: hashpartitioning(coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#4, 0), isnull(d_date#4), 5), ENSURE_REQUIREMENTS, [id=#12] (21) Sort [codegen id : 7] Input [3]: [d_date#4, c_first_name#9, c_last_name#10] @@ -188,7 +189,7 @@ Input [4]: [cs_sold_date_sk#13, cs_bill_customer_sk#14, d_date_sk#15, d_date#16] (28) Exchange Input [2]: [cs_bill_customer_sk#14, d_date#16] -Arguments: hashpartitioning(cs_bill_customer_sk#14, 5), true, [id=#17] +Arguments: hashpartitioning(cs_bill_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#17] (29) Sort [codegen id : 10] Input [2]: [cs_bill_customer_sk#14, d_date#16] @@ -219,7 +220,7 @@ Results [3]: [c_last_name#20, c_first_name#19, d_date#16] (35) Exchange Input [3]: [c_last_name#20, c_first_name#19, d_date#16] -Arguments: hashpartitioning(c_last_name#20, c_first_name#19, d_date#16, 5), true, [id=#21] +Arguments: hashpartitioning(c_last_name#20, c_first_name#19, d_date#16, 5), ENSURE_REQUIREMENTS, [id=#21] (36) HashAggregate [codegen id : 14] Input [3]: [c_last_name#20, c_first_name#19, d_date#16] @@ -230,7 +231,7 @@ Results [3]: [c_last_name#20, c_first_name#19, d_date#16] (37) Exchange Input [3]: [c_last_name#20, c_first_name#19, d_date#16] -Arguments: hashpartitioning(coalesce(c_last_name#20, ), isnull(c_last_name#20), coalesce(c_first_name#19, ), isnull(c_first_name#19), coalesce(d_date#16, 0), isnull(d_date#16), 5), true, [id=#22] +Arguments: hashpartitioning(coalesce(c_last_name#20, ), isnull(c_last_name#20), coalesce(c_first_name#19, ), isnull(c_first_name#19), coalesce(d_date#16, 0), isnull(d_date#16), 5), ENSURE_REQUIREMENTS, [id=#22] (38) Sort [codegen id : 15] Input [3]: [c_last_name#20, c_first_name#19, d_date#16] @@ -269,7 +270,7 @@ Input [4]: [ws_sold_date_sk#23, ws_bill_customer_sk#24, d_date_sk#25, d_date#26] (46) Exchange Input [2]: [ws_bill_customer_sk#24, d_date#26] -Arguments: hashpartitioning(ws_bill_customer_sk#24, 5), true, [id=#27] +Arguments: hashpartitioning(ws_bill_customer_sk#24, 5), ENSURE_REQUIREMENTS, [id=#27] (47) Sort [codegen id : 18] Input [2]: [ws_bill_customer_sk#24, d_date#26] @@ -300,7 +301,7 @@ Results [3]: [c_last_name#30, c_first_name#29, d_date#26] (53) Exchange Input [3]: [c_last_name#30, c_first_name#29, d_date#26] -Arguments: hashpartitioning(c_last_name#30, c_first_name#29, d_date#26, 5), true, [id=#31] +Arguments: hashpartitioning(c_last_name#30, c_first_name#29, d_date#26, 5), ENSURE_REQUIREMENTS, [id=#31] (54) HashAggregate [codegen id : 22] Input [3]: [c_last_name#30, c_first_name#29, d_date#26] @@ -311,7 +312,7 @@ Results [3]: [c_last_name#30, c_first_name#29, d_date#26] (55) Exchange Input [3]: [c_last_name#30, c_first_name#29, d_date#26] -Arguments: hashpartitioning(coalesce(c_last_name#30, ), isnull(c_last_name#30), coalesce(c_first_name#29, ), isnull(c_first_name#29), coalesce(d_date#26, 0), isnull(d_date#26), 5), true, [id=#32] +Arguments: hashpartitioning(coalesce(c_last_name#30, ), isnull(c_last_name#30), coalesce(c_first_name#29, ), isnull(c_first_name#29), coalesce(d_date#26, 0), isnull(d_date#26), 5), ENSURE_REQUIREMENTS, [id=#32] (56) Sort [codegen id : 23] Input [3]: [c_last_name#30, c_first_name#29, d_date#26] @@ -322,64 +323,68 @@ Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_f Right keys [6]: [coalesce(c_last_name#30, ), isnull(c_last_name#30), coalesce(c_first_name#29, ), isnull(c_first_name#29), coalesce(d_date#26, 0), isnull(d_date#26)] Join condition: None -(58) HashAggregate [codegen id : 24] +(58) Project [codegen id : 24] +Output [3]: [c_last_name#10, c_first_name#9, d_date#4] Input [3]: [d_date#4, c_first_name#9, c_last_name#10] + +(59) HashAggregate [codegen id : 24] +Input [3]: [c_last_name#10, c_first_name#9, d_date#4] Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#10, c_first_name#9, d_date#4] -(59) Exchange +(60) Exchange Input [3]: [c_last_name#10, c_first_name#9, d_date#4] -Arguments: hashpartitioning(c_last_name#10, c_first_name#9, d_date#4, 5), true, [id=#33] +Arguments: hashpartitioning(c_last_name#10, c_first_name#9, d_date#4, 5), ENSURE_REQUIREMENTS, [id=#33] -(60) HashAggregate [codegen id : 25] +(61) HashAggregate [codegen id : 25] Input [3]: [c_last_name#10, c_first_name#9, d_date#4] Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#10, c_first_name#9, d_date#4] -(61) HashAggregate [codegen id : 25] +(62) HashAggregate [codegen id : 25] Input [3]: [c_last_name#10, c_first_name#9, d_date#4] Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#10, c_first_name#9, d_date#4] -(62) HashAggregate [codegen id : 25] +(63) HashAggregate [codegen id : 25] Input [3]: [c_last_name#10, c_first_name#9, d_date#4] Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#10, c_first_name#9, d_date#4] -(63) HashAggregate [codegen id : 25] +(64) HashAggregate [codegen id : 25] Input [3]: [c_last_name#10, c_first_name#9, d_date#4] Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#10, c_first_name#9, d_date#4] -(64) HashAggregate [codegen id : 25] +(65) HashAggregate [codegen id : 25] Input [3]: [c_last_name#10, c_first_name#9, d_date#4] Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] Functions: [] Aggregate Attributes: [] Results: [] -(65) HashAggregate [codegen id : 25] +(66) HashAggregate [codegen id : 25] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#34] Results [1]: [count#35] -(66) Exchange +(67) Exchange Input [1]: [count#35] -Arguments: SinglePartition, true, [id=#36] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#36] -(67) HashAggregate [codegen id : 26] +(68) HashAggregate [codegen id : 26] Input [1]: [count#35] Keys: [] Functions [1]: [count(1)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/simplified.txt index 8dd59340cf069..015d3c5a81972 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/simplified.txt @@ -13,105 +13,106 @@ WholeStageCodegen (26) Exchange [c_last_name,c_first_name,d_date] #2 WholeStageCodegen (24) HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + InputAdapter SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - WholeStageCodegen (7) - Sort [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #3 - WholeStageCodegen (6) - Project [d_date,c_first_name,c_last_name] - SortMergeJoin [ss_customer_sk,c_customer_sk] + SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + WholeStageCodegen (7) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #3 + WholeStageCodegen (6) + Project [d_date,c_first_name,c_last_name] + SortMergeJoin [ss_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (3) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #4 + WholeStageCodegen (2) + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + WholeStageCodegen (5) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #6 + WholeStageCodegen (4) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + WholeStageCodegen (15) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #7 + WholeStageCodegen (14) + HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - WholeStageCodegen (3) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #4 - WholeStageCodegen (2) - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow + Exchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (13) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (10) + Sort [cs_bill_customer_sk] + InputAdapter + Exchange [cs_bill_customer_sk] #9 + WholeStageCodegen (9) + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_sold_date_sk,cs_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - WholeStageCodegen (5) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #6 - WholeStageCodegen (4) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] - WholeStageCodegen (15) + ReusedExchange [d_date_sk,d_date] #5 + InputAdapter + WholeStageCodegen (12) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 + WholeStageCodegen (23) Sort [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #7 - WholeStageCodegen (14) + Exchange [c_last_name,c_first_name,d_date] #10 + WholeStageCodegen (22) HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (13) + Exchange [c_last_name,c_first_name,d_date] #11 + WholeStageCodegen (21) HashAggregate [c_last_name,c_first_name,d_date] Project [c_last_name,c_first_name,d_date] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (10) - Sort [cs_bill_customer_sk] + WholeStageCodegen (18) + Sort [ws_bill_customer_sk] InputAdapter - Exchange [cs_bill_customer_sk] #9 - WholeStageCodegen (9) - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk,cs_bill_customer_sk] + Exchange [ws_bill_customer_sk] #12 + WholeStageCodegen (17) + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter ReusedExchange [d_date_sk,d_date] #5 InputAdapter - WholeStageCodegen (12) + WholeStageCodegen (20) Sort [c_customer_sk] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 - WholeStageCodegen (23) - Sort [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #10 - WholeStageCodegen (22) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #11 - WholeStageCodegen (21) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (18) - Sort [ws_bill_customer_sk] - InputAdapter - Exchange [ws_bill_customer_sk] #12 - WholeStageCodegen (17) - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 - InputAdapter - WholeStageCodegen (20) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt index 3d59a670b7e8b..3f52e6de41fb0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt @@ -1,58 +1,59 @@ == Physical Plan == -* HashAggregate (54) -+- Exchange (53) - +- * HashAggregate (52) - +- * HashAggregate (51) - +- * HashAggregate (50) - +- * HashAggregate (49) - +- * HashAggregate (48) - +- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * BroadcastHashJoin LeftAnti BuildRight (44) - :- * BroadcastHashJoin LeftAnti BuildRight (30) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.customer (11) - : +- BroadcastExchange (29) - : +- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet default.catalog_sales (17) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - +- BroadcastExchange (43) - +- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Filter (33) - : : +- * ColumnarToRow (32) - : : +- Scan parquet default.web_sales (31) - : +- ReusedExchange (34) - +- ReusedExchange (37) +* HashAggregate (55) ++- Exchange (54) + +- * HashAggregate (53) + +- * HashAggregate (52) + +- * HashAggregate (51) + +- * HashAggregate (50) + +- * HashAggregate (49) + +- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * Project (45) + +- * BroadcastHashJoin LeftAnti BuildRight (44) + :- * BroadcastHashJoin LeftAnti BuildRight (30) + : :- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- BroadcastExchange (14) + : : +- * Filter (13) + : : +- * ColumnarToRow (12) + : : +- Scan parquet default.customer (11) + : +- BroadcastExchange (29) + : +- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * Filter (19) + : : : +- * ColumnarToRow (18) + : : : +- Scan parquet default.catalog_sales (17) + : : +- ReusedExchange (20) + : +- ReusedExchange (23) + +- BroadcastExchange (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * Filter (33) + : : +- * ColumnarToRow (32) + : : +- Scan parquet default.web_sales (31) + : +- ReusedExchange (34) + +- ReusedExchange (37) (1) Scan parquet default.store_sales @@ -174,7 +175,7 @@ Results [3]: [c_last_name#17, c_first_name#16, d_date#14] (27) Exchange Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Arguments: hashpartitioning(c_last_name#17, c_first_name#16, d_date#14, 5), true, [id=#18] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, d_date#14, 5), ENSURE_REQUIREMENTS, [id=#18] (28) HashAggregate [codegen id : 6] Input [3]: [c_last_name#17, c_first_name#16, d_date#14] @@ -239,7 +240,7 @@ Results [3]: [c_last_name#26, c_first_name#25, d_date#23] (41) Exchange Input [3]: [c_last_name#26, c_first_name#25, d_date#23] -Arguments: hashpartitioning(c_last_name#26, c_first_name#25, d_date#23, 5), true, [id=#27] +Arguments: hashpartitioning(c_last_name#26, c_first_name#25, d_date#23, 5), ENSURE_REQUIREMENTS, [id=#27] (42) HashAggregate [codegen id : 10] Input [3]: [c_last_name#26, c_first_name#25, d_date#23] @@ -257,64 +258,68 @@ Left keys [6]: [coalesce(c_last_name#9, ), isnull(c_last_name#9), coalesce(c_fir Right keys [6]: [coalesce(c_last_name#26, ), isnull(c_last_name#26), coalesce(c_first_name#25, ), isnull(c_first_name#25), coalesce(d_date#23, 0), isnull(d_date#23)] Join condition: None -(45) HashAggregate [codegen id : 11] +(45) Project [codegen id : 11] +Output [3]: [c_last_name#9, c_first_name#8, d_date#4] Input [3]: [d_date#4, c_first_name#8, c_last_name#9] + +(46) HashAggregate [codegen id : 11] +Input [3]: [c_last_name#9, c_first_name#8, d_date#4] Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#9, c_first_name#8, d_date#4] -(46) Exchange +(47) Exchange Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Arguments: hashpartitioning(c_last_name#9, c_first_name#8, d_date#4, 5), true, [id=#29] +Arguments: hashpartitioning(c_last_name#9, c_first_name#8, d_date#4, 5), ENSURE_REQUIREMENTS, [id=#29] -(47) HashAggregate [codegen id : 12] +(48) HashAggregate [codegen id : 12] Input [3]: [c_last_name#9, c_first_name#8, d_date#4] Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#9, c_first_name#8, d_date#4] -(48) HashAggregate [codegen id : 12] +(49) HashAggregate [codegen id : 12] Input [3]: [c_last_name#9, c_first_name#8, d_date#4] Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#9, c_first_name#8, d_date#4] -(49) HashAggregate [codegen id : 12] +(50) HashAggregate [codegen id : 12] Input [3]: [c_last_name#9, c_first_name#8, d_date#4] Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#9, c_first_name#8, d_date#4] -(50) HashAggregate [codegen id : 12] +(51) HashAggregate [codegen id : 12] Input [3]: [c_last_name#9, c_first_name#8, d_date#4] Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#9, c_first_name#8, d_date#4] -(51) HashAggregate [codegen id : 12] +(52) HashAggregate [codegen id : 12] Input [3]: [c_last_name#9, c_first_name#8, d_date#4] Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] Functions: [] Aggregate Attributes: [] Results: [] -(52) HashAggregate [codegen id : 12] +(53) HashAggregate [codegen id : 12] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#30] Results [1]: [count#31] -(53) Exchange +(54) Exchange Input [1]: [count#31] -Arguments: SinglePartition, true, [id=#32] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#32] -(54) HashAggregate [codegen id : 13] +(55) HashAggregate [codegen id : 13] Input [1]: [count#31] Keys: [] Functions [1]: [count(1)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt index a5b57a4ac9450..0f32bfbed99d7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt @@ -13,68 +13,69 @@ WholeStageCodegen (13) Exchange [c_last_name,c_first_name,d_date] #2 WholeStageCodegen (11) HashAggregate [c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - Project [d_date,c_first_name,c_last_name] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Filter [c_customer_sk] + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + Project [d_date,c_first_name,c_last_name] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (6) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #6 + WholeStageCodegen (5) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_sold_date_sk,cs_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) + BroadcastExchange #7 + WholeStageCodegen (10) HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (5) + Exchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (9) HashAggregate [c_last_name,c_first_name,d_date] Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk,cs_bill_customer_sk] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter ReusedExchange [d_date_sk,d_date] #3 InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (9) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt index 6bcbe470cec50..16589f89deb6e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt @@ -8,206 +8,206 @@ +- * BroadcastHashJoin Inner BuildRight (41) :- * Project (36) : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * Project (23) - : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : :- * Project (16) - : : : : +- * BroadcastHashJoin Inner BuildLeft (15) - : : : : :- BroadcastExchange (11) - : : : : : +- * Project (10) - : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : :- * Filter (3) - : : : : : : +- * ColumnarToRow (2) - : : : : : : +- Scan parquet default.catalog_returns (1) - : : : : : +- BroadcastExchange (8) - : : : : : +- * Project (7) - : : : : : +- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet default.date_dim (4) - : : : : +- * Filter (14) - : : : : +- * ColumnarToRow (13) - : : : : +- Scan parquet default.customer (12) - : : : +- BroadcastExchange (21) - : : : +- * Project (20) - : : : +- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet default.household_demographics (17) - : : +- BroadcastExchange (28) - : : +- * Project (27) - : : +- * Filter (26) - : : +- * ColumnarToRow (25) - : : +- Scan parquet default.customer_address (24) + : :- * Project (23) + : : +- * BroadcastHashJoin Inner BuildRight (22) + : : :- * Project (16) + : : : +- * BroadcastHashJoin Inner BuildRight (15) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet default.customer (1) + : : : : +- BroadcastExchange (7) + : : : : +- * Filter (6) + : : : : +- * ColumnarToRow (5) + : : : : +- Scan parquet default.customer_demographics (4) + : : : +- BroadcastExchange (14) + : : : +- * Project (13) + : : : +- * Filter (12) + : : : +- * ColumnarToRow (11) + : : : +- Scan parquet default.household_demographics (10) + : : +- BroadcastExchange (21) + : : +- * Project (20) + : : +- * Filter (19) + : : +- * ColumnarToRow (18) + : : +- Scan parquet default.customer_address (17) : +- BroadcastExchange (34) - : +- * Filter (33) - : +- * ColumnarToRow (32) - : +- Scan parquet default.customer_demographics (31) + : +- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Filter (26) + : : +- * ColumnarToRow (25) + : : +- Scan parquet default.catalog_returns (24) + : +- BroadcastExchange (31) + : +- * Project (30) + : +- * Filter (29) + : +- * ColumnarToRow (28) + : +- Scan parquet default.date_dim (27) +- BroadcastExchange (40) +- * Filter (39) +- * ColumnarToRow (38) +- Scan parquet default.call_center (37) -(1) Scan parquet default.catalog_returns -Output [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4] +(1) Scan parquet default.customer +Output [4]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] -Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4] +(2) ColumnarToRow [codegen id : 7] +Input [4]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4] -(3) Filter [codegen id : 2] -Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4] -Condition : ((isnotnull(cr_call_center_sk#3) AND isnotnull(cr_returned_date_sk#1)) AND isnotnull(cr_returning_customer_sk#2)) +(3) Filter [codegen id : 7] +Input [4]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4] +Condition : (((isnotnull(c_customer_sk#1) AND isnotnull(c_current_addr_sk#4)) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_current_hdemo_sk#3)) -(4) Scan parquet default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_moy#7] +(4) Scan parquet default.customer_demographics +Output [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown )),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree ))), IsNotNull(cd_demo_sk)] +ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#6, d_moy#7] +Input [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] (6) Filter [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#6, d_moy#7] -Condition : ((((isnotnull(d_year#6) AND isnotnull(d_moy#7)) AND (d_year#6 = 1998)) AND (d_moy#7 = 11)) AND isnotnull(d_date_sk#5)) +Input [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] +Condition : ((((cd_marital_status#6 = M) AND (cd_education_status#7 = Unknown )) OR ((cd_marital_status#6 = W) AND (cd_education_status#7 = Advanced Degree ))) AND isnotnull(cd_demo_sk#5)) -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#5] -Input [3]: [d_date_sk#5, d_year#6, d_moy#7] +(7) BroadcastExchange +Input [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#8] -(8) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] - -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [cr_returned_date_sk#1] -Right keys [1]: [d_date_sk#5] +(8) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#5] Join condition: None -(10) Project [codegen id : 2] -Output [3]: [cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4] -Input [5]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4, d_date_sk#5] - -(11) BroadcastExchange -Input [3]: [cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#9] +(9) Project [codegen id : 7] +Output [5]: [c_customer_sk#1, c_current_hdemo_sk#3, c_current_addr_sk#4, cd_marital_status#6, cd_education_status#7] +Input [7]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] -(12) Scan parquet default.customer -Output [4]: [c_customer_sk#10, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] +(10) Scan parquet default.household_demographics +Output [2]: [hd_demo_sk#9, hd_buy_potential#10] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [IsNotNull(hd_buy_potential), StringStartsWith(hd_buy_potential,Unknown), IsNotNull(hd_demo_sk)] +ReadSchema: struct + +(11) ColumnarToRow [codegen id : 2] +Input [2]: [hd_demo_sk#9, hd_buy_potential#10] + +(12) Filter [codegen id : 2] +Input [2]: [hd_demo_sk#9, hd_buy_potential#10] +Condition : ((isnotnull(hd_buy_potential#10) AND StartsWith(hd_buy_potential#10, Unknown)) AND isnotnull(hd_demo_sk#9)) -(13) ColumnarToRow -Input [4]: [c_customer_sk#10, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] +(13) Project [codegen id : 2] +Output [1]: [hd_demo_sk#9] +Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -(14) Filter -Input [4]: [c_customer_sk#10, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] -Condition : (((isnotnull(c_customer_sk#10) AND isnotnull(c_current_addr_sk#13)) AND isnotnull(c_current_cdemo_sk#11)) AND isnotnull(c_current_hdemo_sk#12)) +(14) BroadcastExchange +Input [1]: [hd_demo_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] (15) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cr_returning_customer_sk#2] -Right keys [1]: [c_customer_sk#10] +Left keys [1]: [c_current_hdemo_sk#3] +Right keys [1]: [hd_demo_sk#9] Join condition: None (16) Project [codegen id : 7] -Output [5]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] -Input [7]: [cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4, c_customer_sk#10, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] +Output [4]: [c_customer_sk#1, c_current_addr_sk#4, cd_marital_status#6, cd_education_status#7] +Input [6]: [c_customer_sk#1, c_current_hdemo_sk#3, c_current_addr_sk#4, cd_marital_status#6, cd_education_status#7, hd_demo_sk#9] -(17) Scan parquet default.household_demographics -Output [2]: [hd_demo_sk#14, hd_buy_potential#15] +(17) Scan parquet default.customer_address +Output [2]: [ca_address_sk#12, ca_gmt_offset#13] Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), StringStartsWith(hd_buy_potential,Unknown), IsNotNull(hd_demo_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-7.00), IsNotNull(ca_address_sk)] +ReadSchema: struct (18) ColumnarToRow [codegen id : 3] -Input [2]: [hd_demo_sk#14, hd_buy_potential#15] +Input [2]: [ca_address_sk#12, ca_gmt_offset#13] (19) Filter [codegen id : 3] -Input [2]: [hd_demo_sk#14, hd_buy_potential#15] -Condition : ((isnotnull(hd_buy_potential#15) AND StartsWith(hd_buy_potential#15, Unknown)) AND isnotnull(hd_demo_sk#14)) +Input [2]: [ca_address_sk#12, ca_gmt_offset#13] +Condition : ((isnotnull(ca_gmt_offset#13) AND (ca_gmt_offset#13 = -7.00)) AND isnotnull(ca_address_sk#12)) (20) Project [codegen id : 3] -Output [1]: [hd_demo_sk#14] -Input [2]: [hd_demo_sk#14, hd_buy_potential#15] +Output [1]: [ca_address_sk#12] +Input [2]: [ca_address_sk#12, ca_gmt_offset#13] (21) BroadcastExchange -Input [1]: [hd_demo_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +Input [1]: [ca_address_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] (22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_hdemo_sk#12] -Right keys [1]: [hd_demo_sk#14] +Left keys [1]: [c_current_addr_sk#4] +Right keys [1]: [ca_address_sk#12] Join condition: None (23) Project [codegen id : 7] -Output [4]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11, c_current_addr_sk#13] -Input [6]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13, hd_demo_sk#14] +Output [3]: [c_customer_sk#1, cd_marital_status#6, cd_education_status#7] +Input [5]: [c_customer_sk#1, c_current_addr_sk#4, cd_marital_status#6, cd_education_status#7, ca_address_sk#12] -(24) Scan parquet default.customer_address -Output [2]: [ca_address_sk#17, ca_gmt_offset#18] +(24) Scan parquet default.catalog_returns +Output [4]: [cr_returned_date_sk#15, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-7.00), IsNotNull(ca_address_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_customer_sk)] +ReadSchema: struct -(25) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#17, ca_gmt_offset#18] +(25) ColumnarToRow [codegen id : 5] +Input [4]: [cr_returned_date_sk#15, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] -(26) Filter [codegen id : 4] -Input [2]: [ca_address_sk#17, ca_gmt_offset#18] -Condition : ((isnotnull(ca_gmt_offset#18) AND (ca_gmt_offset#18 = -7.00)) AND isnotnull(ca_address_sk#17)) +(26) Filter [codegen id : 5] +Input [4]: [cr_returned_date_sk#15, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] +Condition : ((isnotnull(cr_call_center_sk#17) AND isnotnull(cr_returned_date_sk#15)) AND isnotnull(cr_returning_customer_sk#16)) -(27) Project [codegen id : 4] -Output [1]: [ca_address_sk#17] -Input [2]: [ca_address_sk#17, ca_gmt_offset#18] +(27) Scan parquet default.date_dim +Output [3]: [d_date_sk#19, d_year#20, d_moy#21] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] +ReadSchema: struct -(28) BroadcastExchange -Input [1]: [ca_address_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +(28) ColumnarToRow [codegen id : 4] +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -(29) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#13] -Right keys [1]: [ca_address_sk#17] -Join condition: None +(29) Filter [codegen id : 4] +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] +Condition : ((((isnotnull(d_year#20) AND isnotnull(d_moy#21)) AND (d_year#20 = 1998)) AND (d_moy#21 = 11)) AND isnotnull(d_date_sk#19)) -(30) Project [codegen id : 7] -Output [3]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11] -Input [5]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11, c_current_addr_sk#13, ca_address_sk#17] +(30) Project [codegen id : 4] +Output [1]: [d_date_sk#19] +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -(31) Scan parquet default.customer_demographics -Output [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown)),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree))), IsNotNull(cd_demo_sk)] -ReadSchema: struct +(31) BroadcastExchange +Input [1]: [d_date_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] -(32) ColumnarToRow [codegen id : 5] -Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] +(32) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cr_returned_date_sk#15] +Right keys [1]: [d_date_sk#19] +Join condition: None -(33) Filter [codegen id : 5] -Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Condition : ((((cd_marital_status#21 = M) AND (cd_education_status#22 = Unknown)) OR ((cd_marital_status#21 = W) AND (cd_education_status#22 = Advanced Degree))) AND isnotnull(cd_demo_sk#20)) +(33) Project [codegen id : 5] +Output [3]: [cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] +Input [5]: [cr_returned_date_sk#15, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18, d_date_sk#19] (34) BroadcastExchange -Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] +Input [3]: [cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (35) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#11] -Right keys [1]: [cd_demo_sk#20] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [cr_returning_customer_sk#16] Join condition: None (36) Project [codegen id : 7] -Output [4]: [cr_call_center_sk#3, cr_net_loss#4, cd_marital_status#21, cd_education_status#22] -Input [6]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11, cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] +Output [4]: [cd_marital_status#6, cd_education_status#7, cr_call_center_sk#17, cr_net_loss#18] +Input [6]: [c_customer_sk#1, cd_marital_status#6, cd_education_status#7, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] (37) Scan parquet default.call_center Output [4]: [cc_call_center_sk#24, cc_call_center_id#25, cc_name#26, cc_manager#27] @@ -228,31 +228,31 @@ Input [4]: [cc_call_center_sk#24, cc_call_center_id#25, cc_name#26, cc_manager#2 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] (41) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cr_call_center_sk#3] +Left keys [1]: [cr_call_center_sk#17] Right keys [1]: [cc_call_center_sk#24] Join condition: None (42) Project [codegen id : 7] -Output [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cr_net_loss#4, cd_marital_status#21, cd_education_status#22] -Input [8]: [cr_call_center_sk#3, cr_net_loss#4, cd_marital_status#21, cd_education_status#22, cc_call_center_sk#24, cc_call_center_id#25, cc_name#26, cc_manager#27] +Output [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cr_net_loss#18, cd_marital_status#6, cd_education_status#7] +Input [8]: [cd_marital_status#6, cd_education_status#7, cr_call_center_sk#17, cr_net_loss#18, cc_call_center_sk#24, cc_call_center_id#25, cc_name#26, cc_manager#27] (43) HashAggregate [codegen id : 7] -Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cr_net_loss#4, cd_marital_status#21, cd_education_status#22] -Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22] -Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#4))] +Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cr_net_loss#18, cd_marital_status#6, cd_education_status#7] +Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7] +Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#18))] Aggregate Attributes [1]: [sum#29] -Results [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22, sum#30] +Results [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7, sum#30] (44) Exchange -Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22, sum#30] -Arguments: hashpartitioning(cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7, sum#30] +Arguments: hashpartitioning(cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7, 5), ENSURE_REQUIREMENTS, [id=#31] (45) HashAggregate [codegen id : 8] -Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22, sum#30] -Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22] -Functions [1]: [sum(UnscaledValue(cr_net_loss#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#4))#32] -Results [4]: [cc_call_center_id#25 AS Call_Center#33, cc_name#26 AS Call_Center_Name#34, cc_manager#27 AS Manager#35, MakeDecimal(sum(UnscaledValue(cr_net_loss#4))#32,17,2) AS Returns_Loss#36] +Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7, sum#30] +Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7] +Functions [1]: [sum(UnscaledValue(cr_net_loss#18))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#18))#32] +Results [4]: [cc_call_center_id#25 AS Call_Center#33, cc_name#26 AS Call_Center_Name#34, cc_manager#27 AS Manager#35, MakeDecimal(sum(UnscaledValue(cr_net_loss#18))#32,17,2) AS Returns_Loss#36] (46) Exchange Input [4]: [Call_Center#33, Call_Center_Name#34, Manager#35, Returns_Loss#36] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt index 6c8d629feed3e..87beb3b565cc1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt @@ -10,58 +10,58 @@ WholeStageCodegen (9) HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] [sum,sum] Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] BroadcastHashJoin [cr_call_center_sk,cc_call_center_sk] - Project [cr_call_center_sk,cr_net_loss,cd_marital_status,cd_education_status] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cr_call_center_sk,cr_net_loss,c_current_cdemo_sk] + Project [cd_marital_status,cd_education_status,cr_call_center_sk,cr_net_loss] + BroadcastHashJoin [c_customer_sk,cr_returning_customer_sk] + Project [c_customer_sk,cd_marital_status,cd_education_status] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cr_call_center_sk,cr_net_loss,c_current_cdemo_sk,c_current_addr_sk] + Project [c_customer_sk,c_current_addr_sk,cd_marital_status,cd_education_status] BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [cr_call_center_sk,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - BroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Filter [cr_call_center_sk,cr_returned_date_sk,cr_returning_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Project [c_customer_sk,c_current_hdemo_sk,c_current_addr_sk,cd_marital_status,cd_education_status] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] Filter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [cd_marital_status,cd_education_status,cd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) + BroadcastExchange #4 + WholeStageCodegen (2) Project [hd_demo_sk] Filter [hd_buy_potential,hd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) + BroadcastExchange #5 + WholeStageCodegen (3) Project [ca_address_sk] Filter [ca_gmt_offset,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] InputAdapter - BroadcastExchange #7 + BroadcastExchange #6 WholeStageCodegen (5) - Filter [cd_marital_status,cd_education_status,cd_demo_sk] - ColumnarToRow + Project [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + Filter [cr_call_center_sk,cr_returned_date_sk,cr_returning_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + BroadcastExchange #7 + WholeStageCodegen (4) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt index 30dabdd2d5523..c005ef0f9ff1a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt @@ -10,24 +10,24 @@ +- Exchange (21) +- * HashAggregate (20) +- * Project (19) - +- * SortMergeJoin Inner (18) - :- * Sort (12) - : +- Exchange (11) - : +- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.store_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.date_dim (4) - +- * Sort (17) - +- Exchange (16) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (12) + : +- * SortMergeJoin Inner (11) + : :- * Sort (5) + : : +- Exchange (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.store_sales (1) + : +- * Sort (10) + : +- Exchange (9) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.item (6) + +- BroadcastExchange (17) + +- * Project (16) +- * Filter (15) +- * ColumnarToRow (14) - +- Scan parquet default.item (13) + +- Scan parquet default.date_dim (13) (1) Scan parquet default.store_sales @@ -37,126 +37,126 @@ Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct +(4) Exchange +Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +Arguments: hashpartitioning(ss_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#4] -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] +(5) Sort [codegen id : 2] +Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] -Condition : (((isnotnull(d_date#5) AND (d_date#5 >= 10644)) AND (d_date#5 <= 10674)) AND isnotnull(d_date_sk#4)) +(6) Scan parquet default.item +Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_date#5] +(7) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] +(8) Filter [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None +(9) Exchange +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: hashpartitioning(i_item_sk#5, 5), ENSURE_REQUIREMENTS, [id=#11] -(10) Project [codegen id : 2] -Output [2]: [ss_item_sk#2, ss_ext_sales_price#3] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3, d_date_sk#4] +(10) Sort [codegen id : 4] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5 ASC NULLS FIRST], false, 0 -(11) Exchange -Input [2]: [ss_item_sk#2, ss_ext_sales_price#3] -Arguments: hashpartitioning(ss_item_sk#2, 5), true, [id=#7] +(11) SortMergeJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#2] +Right keys [1]: [i_item_sk#5] +Join condition: None -(12) Sort [codegen id : 3] -Input [2]: [ss_item_sk#2, ss_ext_sales_price#3] -Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 +(12) Project [codegen id : 6] +Output [7]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) Scan parquet default.item -Output [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(13) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_date#13] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(14) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(14) ColumnarToRow [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] -(15) Filter [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Condition : (i_category#13 IN (Sports,Books,Home) AND isnotnull(i_item_sk#8)) +(15) Filter [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) -(16) Exchange -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: hashpartitioning(i_item_sk#8, 5), true, [id=#14] +(16) Project [codegen id : 5] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_date#13] -(17) Sort [codegen id : 5] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: [i_item_sk#8 ASC NULLS FIRST], false, 0 +(17) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#8] +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#12] Join condition: None (19) Project [codegen id : 6] -Output [6]: [ss_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Input [8]: [ss_item_sk#2, ss_ext_sales_price#3, i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +Output [6]: [ss_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [8]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#12] (20) HashAggregate [codegen id : 6] -Input [6]: [ss_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [ss_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] (21) Exchange -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Arguments: hashpartitioning(i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, 5), true, [id=#17] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [id=#17] (22) HashAggregate [codegen id : 7] -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#18] -Results [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS _w1#21, i_item_id#9] +Results [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS _w1#21, i_item_id#6] (23) Exchange -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: hashpartitioning(i_class#12, 5), true, [id=#22] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [id=#22] (24) Sort [codegen id : 8] -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: [i_class#12 ASC NULLS FIRST], false, 0 +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 (25) Window -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: [sum(_w1#21) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#12] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: [sum(_w1#21) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#9] (26) Project [codegen id : 9] -Output [7]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24, i_item_id#9] -Input [9]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9, _we0#23] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24, i_item_id#6] +Input [9]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6, _we0#23] (27) Exchange -Input [7]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24, i_item_id#9] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST, i_item_desc#10 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST, 5), true, [id=#25] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24, i_item_id#6] +Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#25] (28) Sort [codegen id : 10] -Input [7]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24, i_item_id#9] -Arguments: [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST, i_item_desc#10 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], true, 0 +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24, i_item_id#6] +Arguments: [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], true, 0 (29) Project [codegen id : 10] -Output [6]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] -Input [7]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24, i_item_id#9] +Output [6]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24, i_item_id#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/simplified.txt index a180ed2b060b9..c6ce597f3c1bf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/simplified.txt @@ -18,34 +18,34 @@ WholeStageCodegen (10) WholeStageCodegen (6) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - SortMergeJoin [ss_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (3) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #4 - WholeStageCodegen (2) - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + SortMergeJoin [ss_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #4 + WholeStageCodegen (1) Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #5 + WholeStageCodegen (3) + Filter [i_category,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter - WholeStageCodegen (5) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #6 - WholeStageCodegen (4) - Filter [i_category,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastExchange #6 + WholeStageCodegen (5) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt index c1bf12b7c2c5a..60d89b18f8fc9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt @@ -8,24 +8,24 @@ TakeOrderedAndProject (27) +- Exchange (21) +- * HashAggregate (20) +- * Project (19) - +- * SortMergeJoin Inner (18) - :- * Sort (12) - : +- Exchange (11) - : +- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.web_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.date_dim (4) - +- * Sort (17) - +- Exchange (16) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (12) + : +- * SortMergeJoin Inner (11) + : :- * Sort (5) + : : +- Exchange (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.web_sales (1) + : +- * Sort (10) + : +- Exchange (9) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.item (6) + +- BroadcastExchange (17) + +- * Project (16) +- * Filter (15) +- * ColumnarToRow (14) - +- Scan parquet default.item (13) + +- Scan parquet default.date_dim (13) (1) Scan parquet default.web_sales @@ -35,118 +35,118 @@ Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] Condition : (isnotnull(ws_item_sk#2) AND isnotnull(ws_sold_date_sk#1)) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct +(4) Exchange +Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] +Arguments: hashpartitioning(ws_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#4] -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] +(5) Sort [codegen id : 2] +Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] +Arguments: [ws_item_sk#2 ASC NULLS FIRST], false, 0 -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] -Condition : (((isnotnull(d_date#5) AND (d_date#5 >= 10644)) AND (d_date#5 <= 10674)) AND isnotnull(d_date_sk#4)) +(6) Scan parquet default.item +Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_date#5] +(7) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] +(8) Filter [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None +(9) Exchange +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: hashpartitioning(i_item_sk#5, 5), ENSURE_REQUIREMENTS, [id=#11] -(10) Project [codegen id : 2] -Output [2]: [ws_item_sk#2, ws_ext_sales_price#3] -Input [4]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3, d_date_sk#4] +(10) Sort [codegen id : 4] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5 ASC NULLS FIRST], false, 0 -(11) Exchange -Input [2]: [ws_item_sk#2, ws_ext_sales_price#3] -Arguments: hashpartitioning(ws_item_sk#2, 5), true, [id=#7] +(11) SortMergeJoin [codegen id : 6] +Left keys [1]: [ws_item_sk#2] +Right keys [1]: [i_item_sk#5] +Join condition: None -(12) Sort [codegen id : 3] -Input [2]: [ws_item_sk#2, ws_ext_sales_price#3] -Arguments: [ws_item_sk#2 ASC NULLS FIRST], false, 0 +(12) Project [codegen id : 6] +Output [7]: [ws_sold_date_sk#1, ws_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [9]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) Scan parquet default.item -Output [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(13) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_date#13] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(14) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(14) ColumnarToRow [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] -(15) Filter [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Condition : (i_category#13 IN (Sports,Books,Home) AND isnotnull(i_item_sk#8)) +(15) Filter [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) -(16) Exchange -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: hashpartitioning(i_item_sk#8, 5), true, [id=#14] +(16) Project [codegen id : 5] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_date#13] -(17) Sort [codegen id : 5] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: [i_item_sk#8 ASC NULLS FIRST], false, 0 +(17) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#2] -Right keys [1]: [i_item_sk#8] +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#1] +Right keys [1]: [d_date_sk#12] Join condition: None (19) Project [codegen id : 6] -Output [6]: [ws_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Input [8]: [ws_item_sk#2, ws_ext_sales_price#3, i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +Output [6]: [ws_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [8]: [ws_sold_date_sk#1, ws_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#12] (20) HashAggregate [codegen id : 6] -Input [6]: [ws_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [ws_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#3))] Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] (21) Exchange -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Arguments: hashpartitioning(i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, 5), true, [id=#17] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [id=#17] (22) HashAggregate [codegen id : 7] -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#3))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#3))#18] -Results [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS _w1#21] +Results [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS _w1#21] (23) Exchange -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: hashpartitioning(i_class#12, 5), true, [id=#22] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [id=#22] (24) Sort [codegen id : 8] -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: [i_class#12 ASC NULLS FIRST], false, 0 +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 (25) Window -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: [sum(_w1#21) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#12] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: [sum(_w1#21) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#9] (26) Project [codegen id : 9] -Output [7]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24] -Input [9]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, _we0#23] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24] +Input [9]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, _we0#23] (27) TakeOrderedAndProject -Input [7]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST, i_item_desc#10 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/simplified.txt index d8db515f84f68..cfb9973ef6983 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/simplified.txt @@ -14,34 +14,34 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (6) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - SortMergeJoin [ws_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (3) - Sort [ws_item_sk] - InputAdapter - Exchange [ws_item_sk] #3 - WholeStageCodegen (2) - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + SortMergeJoin [ws_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ws_item_sk] + InputAdapter + Exchange [ws_item_sk] #3 + WholeStageCodegen (1) Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_ext_sales_price] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #4 + WholeStageCodegen (3) + Filter [i_category,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter - WholeStageCodegen (5) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #5 - WholeStageCodegen (4) - Filter [i_category,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastExchange #5 + WholeStageCodegen (5) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/explain.txt index f6c5258701525..35e3304de7082 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/explain.txt @@ -34,24 +34,24 @@ TakeOrderedAndProject (160) : +- * Sort (46) : +- Exchange (45) : +- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) - : :- * Project (38) - : : +- * SortMergeJoin Inner (37) - : : :- * Sort (31) - : : : +- Exchange (30) - : : : +- * Project (29) - : : : +- * Filter (28) - : : : +- * ColumnarToRow (27) - : : : +- Scan parquet default.customer (26) - : : +- * Sort (36) - : : +- Exchange (35) - : : +- * Filter (34) - : : +- * ColumnarToRow (33) - : : +- Scan parquet default.customer_demographics (32) - : +- BroadcastExchange (42) - : +- * Filter (41) - : +- * ColumnarToRow (40) - : +- Scan parquet default.customer_address (39) + : +- * SortMergeJoin Inner (43) + : :- * Sort (37) + : : +- Exchange (36) + : : +- * Project (35) + : : +- * BroadcastHashJoin Inner BuildRight (34) + : : :- * Project (29) + : : : +- * Filter (28) + : : : +- * ColumnarToRow (27) + : : : +- Scan parquet default.customer (26) + : : +- BroadcastExchange (33) + : : +- * Filter (32) + : : +- * ColumnarToRow (31) + : : +- Scan parquet default.customer_address (30) + : +- * Sort (42) + : +- Exchange (41) + : +- * Filter (40) + : +- * ColumnarToRow (39) + : +- Scan parquet default.customer_demographics (38) :- * HashAggregate (76) : +- Exchange (75) : +- * HashAggregate (74) @@ -162,716 +162,733 @@ TakeOrderedAndProject (160) (1) Scan parquet default.catalog_sales -Output [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] +Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 4] -Input [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] (3) Filter [codegen id : 4] -Input [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] -Condition : (((isnotnull(cs_bill_cdemo_sk#3) AND isnotnull(cs_bill_customer_sk#2)) AND isnotnull(cs_sold_date_sk#1)) AND isnotnull(cs_item_sk#4)) +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) (4) Scan parquet default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] +Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] +Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] (6) Filter [codegen id : 1] -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] -Condition : ((((isnotnull(cd_gender#11) AND isnotnull(cd_education_status#12)) AND (cd_gender#11 = M)) AND (cd_education_status#12 = College)) AND isnotnull(cd_demo_sk#10)) +Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] +Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = M)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#11)) (7) Project [codegen id : 1] -Output [2]: [cd_demo_sk#10, cd_dep_count#13] -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] +Output [2]: [cd_demo_sk#11, cd_dep_count#14] +Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] (8) BroadcastExchange -Input [2]: [cd_demo_sk#10, cd_dep_count#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Input [2]: [cd_demo_sk#11, cd_dep_count#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] (9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_bill_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (10) Project [codegen id : 4] -Output [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [11]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_demo_sk#10, cd_dep_count#13] +Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] (11) Scan parquet default.date_dim -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#16, d_year#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#15, d_year#16] +Input [2]: [d_date_sk#16, d_year#17] (13) Filter [codegen id : 2] -Input [2]: [d_date_sk#15, d_year#16] -Condition : ((isnotnull(d_year#16) AND (d_year#16 = 2001)) AND isnotnull(d_date_sk#15)) +Input [2]: [d_date_sk#16, d_year#17] +Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) (14) Project [codegen id : 2] -Output [1]: [d_date_sk#15] -Input [2]: [d_date_sk#15, d_year#16] +Output [1]: [d_date_sk#16] +Input [2]: [d_date_sk#16, d_year#17] (15) BroadcastExchange -Input [1]: [d_date_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Input [1]: [d_date_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#16] Join condition: None (17) Project [codegen id : 4] -Output [8]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [10]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, d_date_sk#15] +Output [8]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14] +Input [10]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, d_date_sk#16] (18) Scan parquet default.item -Output [2]: [i_item_sk#18, i_item_id#19] +Output [2]: [i_item_sk#19, i_item_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#18, i_item_id#19] +Input [2]: [i_item_sk#19, i_item_id#20] (20) Filter [codegen id : 3] -Input [2]: [i_item_sk#18, i_item_id#19] -Condition : isnotnull(i_item_sk#18) +Input [2]: [i_item_sk#19, i_item_id#20] +Condition : isnotnull(i_item_sk#19) (21) BroadcastExchange -Input [2]: [i_item_sk#18, i_item_id#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] +Input [2]: [i_item_sk#19, i_item_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] (22) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#18] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#19] Join condition: None (23) Project [codegen id : 4] -Output [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] -Input [10]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_sk#18, i_item_id#19] +Output [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20] +Input [10]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_sk#19, i_item_id#20] (24) Exchange -Input [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] -Arguments: hashpartitioning(cs_bill_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#22] (25) Sort [codegen id : 5] -Input [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] -Arguments: [cs_bill_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20] +Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (26) Scan parquet default.customer -Output [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [9,5,12,4,1,10]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 6] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +(27) ColumnarToRow [codegen id : 7] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] -(28) Filter [codegen id : 6] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] -Condition : (((c_birth_month#25 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#22)) AND isnotnull(c_current_cdemo_sk#23)) AND isnotnull(c_current_addr_sk#24)) +(28) Filter [codegen id : 7] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Condition : (((c_birth_month#26 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#23)) AND isnotnull(c_current_cdemo_sk#24)) AND isnotnull(c_current_addr_sk#25)) -(29) Project [codegen id : 6] -Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +(29) Project [codegen id : 7] +Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] -(30) Exchange -Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] -Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), ENSURE_REQUIREMENTS, [id=#27] - -(31) Sort [codegen id : 7] -Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] -Arguments: [c_current_cdemo_sk#23 ASC NULLS FIRST], false, 0 - -(32) Scan parquet default.customer_demographics -Output [1]: [cd_demo_sk#28] +(30) Scan parquet default.customer_address +Output [4]: [ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(33) ColumnarToRow [codegen id : 8] -Input [1]: [cd_demo_sk#28] +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] +ReadSchema: struct -(34) Filter [codegen id : 8] -Input [1]: [cd_demo_sk#28] -Condition : isnotnull(cd_demo_sk#28) +(31) ColumnarToRow [codegen id : 6] +Input [4]: [ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] -(35) Exchange -Input [1]: [cd_demo_sk#28] -Arguments: hashpartitioning(cd_demo_sk#28, 5), ENSURE_REQUIREMENTS, [id=#29] +(32) Filter [codegen id : 6] +Input [4]: [ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] +Condition : (ca_state#30 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#28)) -(36) Sort [codegen id : 9] -Input [1]: [cd_demo_sk#28] -Arguments: [cd_demo_sk#28 ASC NULLS FIRST], false, 0 +(33) BroadcastExchange +Input [4]: [ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] -(37) SortMergeJoin [codegen id : 11] -Left keys [1]: [c_current_cdemo_sk#23] -Right keys [1]: [cd_demo_sk#28] +(34) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_addr_sk#25] +Right keys [1]: [ca_address_sk#28] Join condition: None -(38) Project [codegen id : 11] -Output [3]: [c_customer_sk#22, c_current_addr_sk#24, c_birth_year#26] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, cd_demo_sk#28] +(35) Project [codegen id : 7] +Output [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Input [8]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27, ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] + +(36) Exchange +Input [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Arguments: hashpartitioning(c_current_cdemo_sk#24, 5), ENSURE_REQUIREMENTS, [id=#33] -(39) Scan parquet default.customer_address -Output [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +(37) Sort [codegen id : 8] +Input [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Arguments: [c_current_cdemo_sk#24 ASC NULLS FIRST], false, 0 + +(38) Scan parquet default.customer_demographics +Output [1]: [cd_demo_sk#34] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk)] +ReadSchema: struct + +(39) ColumnarToRow [codegen id : 9] +Input [1]: [cd_demo_sk#34] -(40) ColumnarToRow [codegen id : 10] -Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +(40) Filter [codegen id : 9] +Input [1]: [cd_demo_sk#34] +Condition : isnotnull(cd_demo_sk#34) -(41) Filter [codegen id : 10] -Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] -Condition : (ca_state#32 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#30)) +(41) Exchange +Input [1]: [cd_demo_sk#34] +Arguments: hashpartitioning(cd_demo_sk#34, 5), ENSURE_REQUIREMENTS, [id=#35] -(42) BroadcastExchange -Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +(42) Sort [codegen id : 10] +Input [1]: [cd_demo_sk#34] +Arguments: [cd_demo_sk#34 ASC NULLS FIRST], false, 0 -(43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#30] +(43) SortMergeJoin [codegen id : 11] +Left keys [1]: [c_current_cdemo_sk#24] +Right keys [1]: [cd_demo_sk#34] Join condition: None (44) Project [codegen id : 11] -Output [5]: [c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] -Input [7]: [c_customer_sk#22, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +Output [5]: [c_customer_sk#23, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Input [7]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31, cd_demo_sk#34] (45) Exchange -Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] -Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [5]: [c_customer_sk#23, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, [id=#36] (46) Sort [codegen id : 12] -Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] -Arguments: [c_customer_sk#22 ASC NULLS FIRST], false, 0 +Input [5]: [c_customer_sk#23, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Arguments: [c_customer_sk#23 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 13] -Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#22] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#23] Join condition: None (48) Project [codegen id : 13] -Output [11]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] -Input [13]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] +Output [11]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29, cast(cs_quantity#4 as decimal(12,2)) AS agg1#37, cast(cs_list_price#5 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#41, cast(c_birth_year#27 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#43] +Input [13]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20, c_customer_sk#23, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] (49) HashAggregate [codegen id : 13] -Input [11]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] -Keys [4]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31] -Functions [7]: [partial_avg(agg1#36), partial_avg(agg2#37), partial_avg(agg3#38), partial_avg(agg4#39), partial_avg(agg5#40), partial_avg(agg6#41), partial_avg(agg7#42)] -Aggregate Attributes [14]: [sum#43, count#44, sum#45, count#46, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56] -Results [18]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Input [11]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [4]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29] +Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] +Aggregate Attributes [14]: [sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55, sum#56, count#57] +Results [18]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29, sum#58, count#59, sum#60, count#61, sum#62, count#63, sum#64, count#65, sum#66, count#67, sum#68, count#69, sum#70, count#71] (50) Exchange -Input [18]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Arguments: hashpartitioning(i_item_id#19, ca_country#33, ca_state#32, ca_county#31, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [18]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29, sum#58, count#59, sum#60, count#61, sum#62, count#63, sum#64, count#65, sum#66, count#67, sum#68, count#69, sum#70, count#71] +Arguments: hashpartitioning(i_item_id#20, ca_country#31, ca_state#30, ca_county#29, 5), ENSURE_REQUIREMENTS, [id=#72] (51) HashAggregate [codegen id : 14] -Input [18]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Keys [4]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31] -Functions [7]: [avg(agg1#36), avg(agg2#37), avg(agg3#38), avg(agg4#39), avg(agg5#40), avg(agg6#41), avg(agg7#42)] -Aggregate Attributes [7]: [avg(agg1#36)#72, avg(agg2#37)#73, avg(agg3#38)#74, avg(agg4#39)#75, avg(agg5#40)#76, avg(agg6#41)#77, avg(agg7#42)#78] -Results [11]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, avg(agg1#36)#72 AS agg1#79, avg(agg2#37)#73 AS agg2#80, avg(agg3#38)#74 AS agg3#81, avg(agg4#39)#75 AS agg4#82, avg(agg5#40)#76 AS agg5#83, avg(agg6#41)#77 AS agg6#84, avg(agg7#42)#78 AS agg7#85] +Input [18]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29, sum#58, count#59, sum#60, count#61, sum#62, count#63, sum#64, count#65, sum#66, count#67, sum#68, count#69, sum#70, count#71] +Keys [4]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29] +Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] +Aggregate Attributes [7]: [avg(agg1#37)#73, avg(agg2#38)#74, avg(agg3#39)#75, avg(agg4#40)#76, avg(agg5#41)#77, avg(agg6#42)#78, avg(agg7#43)#79] +Results [11]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29, avg(agg1#37)#73 AS agg1#80, avg(agg2#38)#74 AS agg2#81, avg(agg3#39)#75 AS agg3#82, avg(agg4#40)#76 AS agg4#83, avg(agg5#41)#77 AS agg5#84, avg(agg6#42)#78 AS agg6#85, avg(agg7#43)#79 AS agg7#86] -(52) ReusedExchange [Reuses operator id: 24] -Output [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] +(52) ReusedExchange [Reuses operator id: unknown] +Output [8]: [cs_bill_customer_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#93, i_item_id#94] (53) Sort [codegen id : 19] -Input [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] -Arguments: [cs_bill_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [8]: [cs_bill_customer_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#93, i_item_id#94] +Arguments: [cs_bill_customer_sk#87 ASC NULLS FIRST], false, 0 (54) Scan parquet default.customer -Output [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Output [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [9,5,12,4,1,10]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 21] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] (56) Filter [codegen id : 21] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] -Condition : (((c_birth_month#25 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#22)) AND isnotnull(c_current_cdemo_sk#23)) AND isnotnull(c_current_addr_sk#24)) +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] +Condition : (((c_birth_month#98 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#95)) AND isnotnull(c_current_cdemo_sk#96)) AND isnotnull(c_current_addr_sk#97)) (57) Project [codegen id : 21] -Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Output [4]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#99] +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] (58) Scan parquet default.customer_address -Output [3]: [ca_address_sk#30, ca_state#32, ca_country#33] +Output [3]: [ca_address_sk#100, ca_state#101, ca_country#102] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 20] -Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] +Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] (60) Filter [codegen id : 20] -Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] -Condition : (ca_state#32 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#30)) +Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] +Condition : (ca_state#101 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#100)) (61) BroadcastExchange -Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#86] +Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#103] (62) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#30] +Left keys [1]: [c_current_addr_sk#97] +Right keys [1]: [ca_address_sk#100] Join condition: None (63) Project [codegen id : 21] -Output [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#32, ca_country#33] -Input [7]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#30, ca_state#32, ca_country#33] +Output [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102] +Input [7]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#99, ca_address_sk#100, ca_state#101, ca_country#102] (64) Exchange -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#32, ca_country#33] -Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), ENSURE_REQUIREMENTS, [id=#87] +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102] +Arguments: hashpartitioning(c_current_cdemo_sk#96, 5), ENSURE_REQUIREMENTS, [id=#104] (65) Sort [codegen id : 22] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#32, ca_country#33] -Arguments: [c_current_cdemo_sk#23 ASC NULLS FIRST], false, 0 +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102] +Arguments: [c_current_cdemo_sk#96 ASC NULLS FIRST], false, 0 -(66) ReusedExchange [Reuses operator id: 35] -Output [1]: [cd_demo_sk#88] +(66) ReusedExchange [Reuses operator id: 41] +Output [1]: [cd_demo_sk#105] (67) Sort [codegen id : 24] -Input [1]: [cd_demo_sk#88] -Arguments: [cd_demo_sk#88 ASC NULLS FIRST], false, 0 +Input [1]: [cd_demo_sk#105] +Arguments: [cd_demo_sk#105 ASC NULLS FIRST], false, 0 (68) SortMergeJoin [codegen id : 25] -Left keys [1]: [c_current_cdemo_sk#23] -Right keys [1]: [cd_demo_sk#88] +Left keys [1]: [c_current_cdemo_sk#96] +Right keys [1]: [cd_demo_sk#105] Join condition: None (69) Project [codegen id : 25] -Output [4]: [c_customer_sk#22, c_birth_year#26, ca_state#32, ca_country#33] -Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#32, ca_country#33, cd_demo_sk#88] +Output [4]: [c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] +Input [6]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102, cd_demo_sk#105] (70) Exchange -Input [4]: [c_customer_sk#22, c_birth_year#26, ca_state#32, ca_country#33] -Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#89] +Input [4]: [c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] +Arguments: hashpartitioning(c_customer_sk#95, 5), ENSURE_REQUIREMENTS, [id=#106] (71) Sort [codegen id : 26] -Input [4]: [c_customer_sk#22, c_birth_year#26, ca_state#32, ca_country#33] -Arguments: [c_customer_sk#22 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] +Arguments: [c_customer_sk#95 ASC NULLS FIRST], false, 0 (72) SortMergeJoin [codegen id : 27] -Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#22] +Left keys [1]: [cs_bill_customer_sk#87] +Right keys [1]: [c_customer_sk#95] Join condition: None (73) Project [codegen id : 27] -Output [10]: [i_item_id#19, ca_country#33, ca_state#32, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] -Input [12]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_state#32, ca_country#33] +Output [10]: [i_item_id#94, ca_country#102, ca_state#101, cast(cs_quantity#88 as decimal(12,2)) AS agg1#37, cast(cs_list_price#89 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#91 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#90 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#92 as decimal(12,2)) AS agg5#41, cast(c_birth_year#99 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#93 as decimal(12,2)) AS agg7#43] +Input [12]: [cs_bill_customer_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#93, i_item_id#94, c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] (74) HashAggregate [codegen id : 27] -Input [10]: [i_item_id#19, ca_country#33, ca_state#32, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] -Keys [3]: [i_item_id#19, ca_country#33, ca_state#32] -Functions [7]: [partial_avg(agg1#36), partial_avg(agg2#37), partial_avg(agg3#38), partial_avg(agg4#39), partial_avg(agg5#40), partial_avg(agg6#41), partial_avg(agg7#42)] -Aggregate Attributes [14]: [sum#90, count#91, sum#92, count#93, sum#94, count#95, sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103] -Results [17]: [i_item_id#19, ca_country#33, ca_state#32, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] +Input [10]: [i_item_id#94, ca_country#102, ca_state#101, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [3]: [i_item_id#94, ca_country#102, ca_state#101] +Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] +Aggregate Attributes [14]: [sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118, sum#119, count#120] +Results [17]: [i_item_id#94, ca_country#102, ca_state#101, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] (75) Exchange -Input [17]: [i_item_id#19, ca_country#33, ca_state#32, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] -Arguments: hashpartitioning(i_item_id#19, ca_country#33, ca_state#32, 5), ENSURE_REQUIREMENTS, [id=#118] +Input [17]: [i_item_id#94, ca_country#102, ca_state#101, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] +Arguments: hashpartitioning(i_item_id#94, ca_country#102, ca_state#101, 5), ENSURE_REQUIREMENTS, [id=#135] (76) HashAggregate [codegen id : 28] -Input [17]: [i_item_id#19, ca_country#33, ca_state#32, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] -Keys [3]: [i_item_id#19, ca_country#33, ca_state#32] -Functions [7]: [avg(agg1#36), avg(agg2#37), avg(agg3#38), avg(agg4#39), avg(agg5#40), avg(agg6#41), avg(agg7#42)] -Aggregate Attributes [7]: [avg(agg1#36)#119, avg(agg2#37)#120, avg(agg3#38)#121, avg(agg4#39)#122, avg(agg5#40)#123, avg(agg6#41)#124, avg(agg7#42)#125] -Results [11]: [i_item_id#19, ca_country#33, ca_state#32, null AS county#126, avg(agg1#36)#119 AS agg1#127, avg(agg2#37)#120 AS agg2#128, avg(agg3#38)#121 AS agg3#129, avg(agg4#39)#122 AS agg4#130, avg(agg5#40)#123 AS agg5#131, avg(agg6#41)#124 AS agg6#132, avg(agg7#42)#125 AS agg7#133] +Input [17]: [i_item_id#94, ca_country#102, ca_state#101, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] +Keys [3]: [i_item_id#94, ca_country#102, ca_state#101] +Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] +Aggregate Attributes [7]: [avg(agg1#37)#136, avg(agg2#38)#137, avg(agg3#39)#138, avg(agg4#40)#139, avg(agg5#41)#140, avg(agg6#42)#141, avg(agg7#43)#142] +Results [11]: [i_item_id#94, ca_country#102, ca_state#101, null AS county#143, avg(agg1#37)#136 AS agg1#144, avg(agg2#38)#137 AS agg2#145, avg(agg3#39)#138 AS agg3#146, avg(agg4#40)#139 AS agg4#147, avg(agg5#41)#140 AS agg5#148, avg(agg6#42)#141 AS agg6#149, avg(agg7#43)#142 AS agg7#150] -(77) ReusedExchange [Reuses operator id: 24] -Output [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] +(77) ReusedExchange [Reuses operator id: unknown] +Output [8]: [cs_bill_customer_sk#151, cs_quantity#152, cs_list_price#153, cs_sales_price#154, cs_coupon_amt#155, cs_net_profit#156, cd_dep_count#157, i_item_id#158] (78) Sort [codegen id : 33] -Input [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] -Arguments: [cs_bill_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [8]: [cs_bill_customer_sk#151, cs_quantity#152, cs_list_price#153, cs_sales_price#154, cs_coupon_amt#155, cs_net_profit#156, cd_dep_count#157, i_item_id#158] +Arguments: [cs_bill_customer_sk#151 ASC NULLS FIRST], false, 0 (79) Scan parquet default.customer -Output [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Output [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [9,5,12,4,1,10]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (80) ColumnarToRow [codegen id : 35] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] (81) Filter [codegen id : 35] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] -Condition : (((c_birth_month#25 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#22)) AND isnotnull(c_current_cdemo_sk#23)) AND isnotnull(c_current_addr_sk#24)) +Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] +Condition : (((c_birth_month#162 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#159)) AND isnotnull(c_current_cdemo_sk#160)) AND isnotnull(c_current_addr_sk#161)) (82) Project [codegen id : 35] -Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Output [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_year#163] +Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] (83) Scan parquet default.customer_address -Output [3]: [ca_address_sk#30, ca_state#32, ca_country#33] +Output [3]: [ca_address_sk#164, ca_state#165, ca_country#166] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (84) ColumnarToRow [codegen id : 34] -Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] +Input [3]: [ca_address_sk#164, ca_state#165, ca_country#166] (85) Filter [codegen id : 34] -Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] -Condition : (ca_state#32 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#30)) +Input [3]: [ca_address_sk#164, ca_state#165, ca_country#166] +Condition : (ca_state#165 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#164)) (86) Project [codegen id : 34] -Output [2]: [ca_address_sk#30, ca_country#33] -Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] +Output [2]: [ca_address_sk#164, ca_country#166] +Input [3]: [ca_address_sk#164, ca_state#165, ca_country#166] (87) BroadcastExchange -Input [2]: [ca_address_sk#30, ca_country#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#134] +Input [2]: [ca_address_sk#164, ca_country#166] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#167] (88) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#30] +Left keys [1]: [c_current_addr_sk#161] +Right keys [1]: [ca_address_sk#164] Join condition: None (89) Project [codegen id : 35] -Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#33] -Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#30, ca_country#33] +Output [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166] +Input [6]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_year#163, ca_address_sk#164, ca_country#166] (90) Exchange -Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#33] -Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), ENSURE_REQUIREMENTS, [id=#135] +Input [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166] +Arguments: hashpartitioning(c_current_cdemo_sk#160, 5), ENSURE_REQUIREMENTS, [id=#168] (91) Sort [codegen id : 36] -Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#33] -Arguments: [c_current_cdemo_sk#23 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166] +Arguments: [c_current_cdemo_sk#160 ASC NULLS FIRST], false, 0 -(92) ReusedExchange [Reuses operator id: 35] -Output [1]: [cd_demo_sk#136] +(92) ReusedExchange [Reuses operator id: 41] +Output [1]: [cd_demo_sk#169] (93) Sort [codegen id : 38] -Input [1]: [cd_demo_sk#136] -Arguments: [cd_demo_sk#136 ASC NULLS FIRST], false, 0 +Input [1]: [cd_demo_sk#169] +Arguments: [cd_demo_sk#169 ASC NULLS FIRST], false, 0 (94) SortMergeJoin [codegen id : 39] -Left keys [1]: [c_current_cdemo_sk#23] -Right keys [1]: [cd_demo_sk#136] +Left keys [1]: [c_current_cdemo_sk#160] +Right keys [1]: [cd_demo_sk#169] Join condition: None (95) Project [codegen id : 39] -Output [3]: [c_customer_sk#22, c_birth_year#26, ca_country#33] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#33, cd_demo_sk#136] +Output [3]: [c_customer_sk#159, c_birth_year#163, ca_country#166] +Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166, cd_demo_sk#169] (96) Exchange -Input [3]: [c_customer_sk#22, c_birth_year#26, ca_country#33] -Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#137] +Input [3]: [c_customer_sk#159, c_birth_year#163, ca_country#166] +Arguments: hashpartitioning(c_customer_sk#159, 5), ENSURE_REQUIREMENTS, [id=#170] (97) Sort [codegen id : 40] -Input [3]: [c_customer_sk#22, c_birth_year#26, ca_country#33] -Arguments: [c_customer_sk#22 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#159, c_birth_year#163, ca_country#166] +Arguments: [c_customer_sk#159 ASC NULLS FIRST], false, 0 (98) SortMergeJoin [codegen id : 41] -Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#22] +Left keys [1]: [cs_bill_customer_sk#151] +Right keys [1]: [c_customer_sk#159] Join condition: None (99) Project [codegen id : 41] -Output [9]: [i_item_id#19, ca_country#33, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] -Input [11]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_country#33] +Output [9]: [i_item_id#158, ca_country#166, cast(cs_quantity#152 as decimal(12,2)) AS agg1#37, cast(cs_list_price#153 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#155 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#154 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#156 as decimal(12,2)) AS agg5#41, cast(c_birth_year#163 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#157 as decimal(12,2)) AS agg7#43] +Input [11]: [cs_bill_customer_sk#151, cs_quantity#152, cs_list_price#153, cs_sales_price#154, cs_coupon_amt#155, cs_net_profit#156, cd_dep_count#157, i_item_id#158, c_customer_sk#159, c_birth_year#163, ca_country#166] (100) HashAggregate [codegen id : 41] -Input [9]: [i_item_id#19, ca_country#33, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] -Keys [2]: [i_item_id#19, ca_country#33] -Functions [7]: [partial_avg(agg1#36), partial_avg(agg2#37), partial_avg(agg3#38), partial_avg(agg4#39), partial_avg(agg5#40), partial_avg(agg6#41), partial_avg(agg7#42)] -Aggregate Attributes [14]: [sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149, sum#150, count#151] -Results [16]: [i_item_id#19, ca_country#33, sum#152, count#153, sum#154, count#155, sum#156, count#157, sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165] +Input [9]: [i_item_id#158, ca_country#166, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [2]: [i_item_id#158, ca_country#166] +Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] +Aggregate Attributes [14]: [sum#171, count#172, sum#173, count#174, sum#175, count#176, sum#177, count#178, sum#179, count#180, sum#181, count#182, sum#183, count#184] +Results [16]: [i_item_id#158, ca_country#166, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198] (101) Exchange -Input [16]: [i_item_id#19, ca_country#33, sum#152, count#153, sum#154, count#155, sum#156, count#157, sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165] -Arguments: hashpartitioning(i_item_id#19, ca_country#33, 5), ENSURE_REQUIREMENTS, [id=#166] +Input [16]: [i_item_id#158, ca_country#166, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198] +Arguments: hashpartitioning(i_item_id#158, ca_country#166, 5), ENSURE_REQUIREMENTS, [id=#199] (102) HashAggregate [codegen id : 42] -Input [16]: [i_item_id#19, ca_country#33, sum#152, count#153, sum#154, count#155, sum#156, count#157, sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165] -Keys [2]: [i_item_id#19, ca_country#33] -Functions [7]: [avg(agg1#36), avg(agg2#37), avg(agg3#38), avg(agg4#39), avg(agg5#40), avg(agg6#41), avg(agg7#42)] -Aggregate Attributes [7]: [avg(agg1#36)#167, avg(agg2#37)#168, avg(agg3#38)#169, avg(agg4#39)#170, avg(agg5#40)#171, avg(agg6#41)#172, avg(agg7#42)#173] -Results [11]: [i_item_id#19, ca_country#33, null AS ca_state#174, null AS county#175, avg(agg1#36)#167 AS agg1#176, avg(agg2#37)#168 AS agg2#177, avg(agg3#38)#169 AS agg3#178, avg(agg4#39)#170 AS agg4#179, avg(agg5#40)#171 AS agg5#180, avg(agg6#41)#172 AS agg6#181, avg(agg7#42)#173 AS agg7#182] +Input [16]: [i_item_id#158, ca_country#166, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198] +Keys [2]: [i_item_id#158, ca_country#166] +Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] +Aggregate Attributes [7]: [avg(agg1#37)#200, avg(agg2#38)#201, avg(agg3#39)#202, avg(agg4#40)#203, avg(agg5#41)#204, avg(agg6#42)#205, avg(agg7#43)#206] +Results [11]: [i_item_id#158, ca_country#166, null AS ca_state#207, null AS county#208, avg(agg1#37)#200 AS agg1#209, avg(agg2#38)#201 AS agg2#210, avg(agg3#39)#202 AS agg3#211, avg(agg4#40)#203 AS agg4#212, avg(agg5#41)#204 AS agg5#213, avg(agg6#42)#205 AS agg6#214, avg(agg7#43)#206 AS agg7#215] (103) Scan parquet default.catalog_sales -Output [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] +Output [9]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#224), dynamicpruningexpression(cs_sold_date_sk#224 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct (104) ColumnarToRow [codegen id : 49] -Input [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] +Input [9]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224] (105) Filter [codegen id : 49] -Input [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] -Condition : (((isnotnull(cs_bill_cdemo_sk#3) AND isnotnull(cs_bill_customer_sk#2)) AND isnotnull(cs_sold_date_sk#1)) AND isnotnull(cs_item_sk#4)) +Input [9]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224] +Condition : ((isnotnull(cs_bill_cdemo_sk#217) AND isnotnull(cs_bill_customer_sk#216)) AND isnotnull(cs_item_sk#218)) (106) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#10, cd_dep_count#13] +Output [2]: [cd_demo_sk#225, cd_dep_count#226] (107) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_bill_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [cs_bill_cdemo_sk#217] +Right keys [1]: [cd_demo_sk#225] Join condition: None (108) Project [codegen id : 49] -Output [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [11]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_demo_sk#10, cd_dep_count#13] +Output [9]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224, cd_dep_count#226] +Input [11]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224, cd_demo_sk#225, cd_dep_count#226] (109) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#227] (110) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [cs_sold_date_sk#224] +Right keys [1]: [d_date_sk#227] Join condition: None (111) Project [codegen id : 49] -Output [8]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [10]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, d_date_sk#15] +Output [8]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226] +Input [10]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224, cd_dep_count#226, d_date_sk#227] (112) Scan parquet default.customer -Output [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Output [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [9,5,12,4,1,10]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (113) ColumnarToRow [codegen id : 46] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] (114) Filter [codegen id : 46] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] -Condition : (((c_birth_month#25 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#22)) AND isnotnull(c_current_cdemo_sk#23)) AND isnotnull(c_current_addr_sk#24)) +Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] +Condition : (((c_birth_month#231 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#228)) AND isnotnull(c_current_cdemo_sk#229)) AND isnotnull(c_current_addr_sk#230)) (115) Project [codegen id : 46] -Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Output [4]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_year#232] +Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] (116) Scan parquet default.customer_address -Output [2]: [ca_address_sk#30, ca_state#32] +Output [2]: [ca_address_sk#233, ca_state#234] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (117) ColumnarToRow [codegen id : 45] -Input [2]: [ca_address_sk#30, ca_state#32] +Input [2]: [ca_address_sk#233, ca_state#234] (118) Filter [codegen id : 45] -Input [2]: [ca_address_sk#30, ca_state#32] -Condition : (ca_state#32 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#30)) +Input [2]: [ca_address_sk#233, ca_state#234] +Condition : (ca_state#234 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#233)) (119) Project [codegen id : 45] -Output [1]: [ca_address_sk#30] -Input [2]: [ca_address_sk#30, ca_state#32] +Output [1]: [ca_address_sk#233] +Input [2]: [ca_address_sk#233, ca_state#234] (120) BroadcastExchange -Input [1]: [ca_address_sk#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#183] +Input [1]: [ca_address_sk#233] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#235] (121) BroadcastHashJoin [codegen id : 46] -Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#30] +Left keys [1]: [c_current_addr_sk#230] +Right keys [1]: [ca_address_sk#233] Join condition: None (122) Project [codegen id : 46] -Output [3]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#30] +Output [3]: [c_customer_sk#228, c_current_cdemo_sk#229, c_birth_year#232] +Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_year#232, ca_address_sk#233] (123) BroadcastExchange -Input [3]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#184] +Input [3]: [c_customer_sk#228, c_current_cdemo_sk#229, c_birth_year#232] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#236] (124) Scan parquet default.customer_demographics -Output [1]: [cd_demo_sk#185] +Output [1]: [cd_demo_sk#237] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (125) ColumnarToRow -Input [1]: [cd_demo_sk#185] +Input [1]: [cd_demo_sk#237] (126) Filter -Input [1]: [cd_demo_sk#185] -Condition : isnotnull(cd_demo_sk#185) +Input [1]: [cd_demo_sk#237] +Condition : isnotnull(cd_demo_sk#237) (127) BroadcastHashJoin [codegen id : 47] -Left keys [1]: [c_current_cdemo_sk#23] -Right keys [1]: [cd_demo_sk#185] +Left keys [1]: [c_current_cdemo_sk#229] +Right keys [1]: [cd_demo_sk#237] Join condition: None (128) Project [codegen id : 47] -Output [2]: [c_customer_sk#22, c_birth_year#26] -Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, cd_demo_sk#185] +Output [2]: [c_customer_sk#228, c_birth_year#232] +Input [4]: [c_customer_sk#228, c_current_cdemo_sk#229, c_birth_year#232, cd_demo_sk#237] (129) BroadcastExchange -Input [2]: [c_customer_sk#22, c_birth_year#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#186] +Input [2]: [c_customer_sk#228, c_birth_year#232] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#238] (130) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#22] +Left keys [1]: [cs_bill_customer_sk#216] +Right keys [1]: [c_customer_sk#228] Join condition: None (131) Project [codegen id : 49] -Output [8]: [cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26] -Input [10]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_customer_sk#22, c_birth_year#26] +Output [8]: [cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226, c_birth_year#232] +Input [10]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226, c_customer_sk#228, c_birth_year#232] (132) ReusedExchange [Reuses operator id: 21] -Output [2]: [i_item_sk#18, i_item_id#19] +Output [2]: [i_item_sk#239, i_item_id#240] (133) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#18] +Left keys [1]: [cs_item_sk#218] +Right keys [1]: [i_item_sk#239] Join condition: None (134) Project [codegen id : 49] -Output [8]: [i_item_id#19, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] -Input [10]: [cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_sk#18, i_item_id#19] +Output [8]: [i_item_id#240, cast(cs_quantity#219 as decimal(12,2)) AS agg1#37, cast(cs_list_price#220 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#222 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#221 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#223 as decimal(12,2)) AS agg5#41, cast(c_birth_year#232 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#226 as decimal(12,2)) AS agg7#43] +Input [10]: [cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226, c_birth_year#232, i_item_sk#239, i_item_id#240] (135) HashAggregate [codegen id : 49] -Input [8]: [i_item_id#19, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] -Keys [1]: [i_item_id#19] -Functions [7]: [partial_avg(agg1#36), partial_avg(agg2#37), partial_avg(agg3#38), partial_avg(agg4#39), partial_avg(agg5#40), partial_avg(agg6#41), partial_avg(agg7#42)] -Aggregate Attributes [14]: [sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198, sum#199, count#200] -Results [15]: [i_item_id#19, sum#201, count#202, sum#203, count#204, sum#205, count#206, sum#207, count#208, sum#209, count#210, sum#211, count#212, sum#213, count#214] +Input [8]: [i_item_id#240, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [1]: [i_item_id#240] +Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] +Aggregate Attributes [14]: [sum#241, count#242, sum#243, count#244, sum#245, count#246, sum#247, count#248, sum#249, count#250, sum#251, count#252, sum#253, count#254] +Results [15]: [i_item_id#240, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266, sum#267, count#268] (136) Exchange -Input [15]: [i_item_id#19, sum#201, count#202, sum#203, count#204, sum#205, count#206, sum#207, count#208, sum#209, count#210, sum#211, count#212, sum#213, count#214] -Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, [id=#215] +Input [15]: [i_item_id#240, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266, sum#267, count#268] +Arguments: hashpartitioning(i_item_id#240, 5), ENSURE_REQUIREMENTS, [id=#269] (137) HashAggregate [codegen id : 50] -Input [15]: [i_item_id#19, sum#201, count#202, sum#203, count#204, sum#205, count#206, sum#207, count#208, sum#209, count#210, sum#211, count#212, sum#213, count#214] -Keys [1]: [i_item_id#19] -Functions [7]: [avg(agg1#36), avg(agg2#37), avg(agg3#38), avg(agg4#39), avg(agg5#40), avg(agg6#41), avg(agg7#42)] -Aggregate Attributes [7]: [avg(agg1#36)#216, avg(agg2#37)#217, avg(agg3#38)#218, avg(agg4#39)#219, avg(agg5#40)#220, avg(agg6#41)#221, avg(agg7#42)#222] -Results [11]: [i_item_id#19, null AS ca_country#223, null AS ca_state#224, null AS county#225, avg(agg1#36)#216 AS agg1#226, avg(agg2#37)#217 AS agg2#227, avg(agg3#38)#218 AS agg3#228, avg(agg4#39)#219 AS agg4#229, avg(agg5#40)#220 AS agg5#230, avg(agg6#41)#221 AS agg6#231, avg(agg7#42)#222 AS agg7#232] +Input [15]: [i_item_id#240, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266, sum#267, count#268] +Keys [1]: [i_item_id#240] +Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] +Aggregate Attributes [7]: [avg(agg1#37)#270, avg(agg2#38)#271, avg(agg3#39)#272, avg(agg4#40)#273, avg(agg5#41)#274, avg(agg6#42)#275, avg(agg7#43)#276] +Results [11]: [i_item_id#240, null AS ca_country#277, null AS ca_state#278, null AS county#279, avg(agg1#37)#270 AS agg1#280, avg(agg2#38)#271 AS agg2#281, avg(agg3#39)#272 AS agg3#282, avg(agg4#40)#273 AS agg4#283, avg(agg5#41)#274 AS agg5#284, avg(agg6#42)#275 AS agg6#285, avg(agg7#43)#276 AS agg7#286] (138) Scan parquet default.catalog_sales -Output [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] +Output [9]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#295), dynamicpruningexpression(cs_sold_date_sk#295 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct (139) ColumnarToRow [codegen id : 57] -Input [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] +Input [9]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295] (140) Filter [codegen id : 57] -Input [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] -Condition : (((isnotnull(cs_bill_cdemo_sk#3) AND isnotnull(cs_bill_customer_sk#2)) AND isnotnull(cs_sold_date_sk#1)) AND isnotnull(cs_item_sk#4)) +Input [9]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295] +Condition : ((isnotnull(cs_bill_cdemo_sk#288) AND isnotnull(cs_bill_customer_sk#287)) AND isnotnull(cs_item_sk#289)) (141) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#10, cd_dep_count#13] +Output [2]: [cd_demo_sk#296, cd_dep_count#297] (142) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_bill_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [cs_bill_cdemo_sk#288] +Right keys [1]: [cd_demo_sk#296] Join condition: None (143) Project [codegen id : 57] -Output [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [11]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_demo_sk#10, cd_dep_count#13] +Output [9]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295, cd_dep_count#297] +Input [11]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295, cd_demo_sk#296, cd_dep_count#297] (144) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#298] (145) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [cs_sold_date_sk#295] +Right keys [1]: [d_date_sk#298] Join condition: None (146) Project [codegen id : 57] -Output [8]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [10]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, d_date_sk#15] +Output [8]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297] +Input [10]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295, cd_dep_count#297, d_date_sk#298] (147) Scan parquet default.item -Output [1]: [i_item_sk#18] +Output [1]: [i_item_sk#299] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (148) ColumnarToRow [codegen id : 53] -Input [1]: [i_item_sk#18] +Input [1]: [i_item_sk#299] (149) Filter [codegen id : 53] -Input [1]: [i_item_sk#18] -Condition : isnotnull(i_item_sk#18) +Input [1]: [i_item_sk#299] +Condition : isnotnull(i_item_sk#299) (150) BroadcastExchange -Input [1]: [i_item_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#233] +Input [1]: [i_item_sk#299] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#300] (151) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#18] +Left keys [1]: [cs_item_sk#289] +Right keys [1]: [i_item_sk#299] Join condition: None (152) Project [codegen id : 57] -Output [7]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [9]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_sk#18] +Output [7]: [cs_bill_customer_sk#287, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297] +Input [9]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297, i_item_sk#299] (153) ReusedExchange [Reuses operator id: 129] -Output [2]: [c_customer_sk#22, c_birth_year#26] +Output [2]: [c_customer_sk#301, c_birth_year#302] (154) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#22] +Left keys [1]: [cs_bill_customer_sk#287] +Right keys [1]: [c_customer_sk#301] Join condition: None (155) Project [codegen id : 57] -Output [7]: [cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] -Input [9]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_customer_sk#22, c_birth_year#26] +Output [7]: [cast(cs_quantity#290 as decimal(12,2)) AS agg1#37, cast(cs_list_price#291 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#293 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#292 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#294 as decimal(12,2)) AS agg5#41, cast(c_birth_year#302 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#297 as decimal(12,2)) AS agg7#43] +Input [9]: [cs_bill_customer_sk#287, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297, c_customer_sk#301, c_birth_year#302] (156) HashAggregate [codegen id : 57] -Input [7]: [agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] +Input [7]: [agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] Keys: [] -Functions [7]: [partial_avg(agg1#36), partial_avg(agg2#37), partial_avg(agg3#38), partial_avg(agg4#39), partial_avg(agg5#40), partial_avg(agg6#41), partial_avg(agg7#42)] -Aggregate Attributes [14]: [sum#234, count#235, sum#236, count#237, sum#238, count#239, sum#240, count#241, sum#242, count#243, sum#244, count#245, sum#246, count#247] -Results [14]: [sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255, sum#256, count#257, sum#258, count#259, sum#260, count#261] +Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] +Aggregate Attributes [14]: [sum#303, count#304, sum#305, count#306, sum#307, count#308, sum#309, count#310, sum#311, count#312, sum#313, count#314, sum#315, count#316] +Results [14]: [sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328, sum#329, count#330] (157) Exchange -Input [14]: [sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255, sum#256, count#257, sum#258, count#259, sum#260, count#261] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#262] +Input [14]: [sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328, sum#329, count#330] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#331] (158) HashAggregate [codegen id : 58] -Input [14]: [sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255, sum#256, count#257, sum#258, count#259, sum#260, count#261] +Input [14]: [sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328, sum#329, count#330] Keys: [] -Functions [7]: [avg(agg1#36), avg(agg2#37), avg(agg3#38), avg(agg4#39), avg(agg5#40), avg(agg6#41), avg(agg7#42)] -Aggregate Attributes [7]: [avg(agg1#36)#263, avg(agg2#37)#264, avg(agg3#38)#265, avg(agg4#39)#266, avg(agg5#40)#267, avg(agg6#41)#268, avg(agg7#42)#269] -Results [11]: [null AS i_item_id#270, null AS ca_country#271, null AS ca_state#272, null AS county#273, avg(agg1#36)#263 AS agg1#274, avg(agg2#37)#264 AS agg2#275, avg(agg3#38)#265 AS agg3#276, avg(agg4#39)#266 AS agg4#277, avg(agg5#40)#267 AS agg5#278, avg(agg6#41)#268 AS agg6#279, avg(agg7#42)#269 AS agg7#280] +Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] +Aggregate Attributes [7]: [avg(agg1#37)#332, avg(agg2#38)#333, avg(agg3#39)#334, avg(agg4#40)#335, avg(agg5#41)#336, avg(agg6#42)#337, avg(agg7#43)#338] +Results [11]: [null AS i_item_id#339, null AS ca_country#340, null AS ca_state#341, null AS county#342, avg(agg1#37)#332 AS agg1#343, avg(agg2#38)#333 AS agg2#344, avg(agg3#39)#334 AS agg3#345, avg(agg4#40)#335 AS agg4#346, avg(agg5#41)#336 AS agg5#347, avg(agg6#42)#337 AS agg6#348, avg(agg7#43)#338 AS agg7#349] (159) Union (160) TakeOrderedAndProject -Input [11]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, agg1#79, agg2#80, agg3#81, agg4#82, agg5#83, agg6#84, agg7#85] -Arguments: 100, [ca_country#33 ASC NULLS FIRST, ca_state#32 ASC NULLS FIRST, ca_county#31 ASC NULLS FIRST, i_item_id#19 ASC NULLS FIRST], [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, agg1#79, agg2#80, agg3#81, agg4#82, agg5#83, agg6#84, agg7#85] +Input [11]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29, agg1#80, agg2#81, agg3#82, agg4#83, agg5#84, agg6#85, agg7#86] +Arguments: 100, [ca_country#31 ASC NULLS FIRST, ca_state#30 ASC NULLS FIRST, ca_county#29 ASC NULLS FIRST, i_item_id#20 ASC NULLS FIRST], [i_item_id#20, ca_country#31, ca_state#30, ca_county#29, agg1#80, agg2#81, agg3#82, agg4#83, agg5#84, agg6#85, agg7#86] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +ReusedExchange (161) + + +(161) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#16] + +Subquery:2 Hosting operator id = 103 Hosting Expression = cs_sold_date_sk#224 IN dynamicpruning#10 + +Subquery:3 Hosting operator id = 138 Hosting Expression = cs_sold_date_sk#295 IN dynamicpruning#10 + diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/simplified.txt index 4566929712713..5514e335f1b51 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/simplified.txt @@ -54,37 +54,37 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag Exchange [c_customer_sk] #6 WholeStageCodegen (11) Project [c_customer_sk,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_customer_sk,c_current_addr_sk,c_birth_year] - SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] - InputAdapter - WholeStageCodegen (7) - Sort [c_current_cdemo_sk] - InputAdapter - Exchange [c_current_cdemo_sk] #7 - WholeStageCodegen (6) + SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] + InputAdapter + WholeStageCodegen (8) + Sort [c_current_cdemo_sk] + InputAdapter + Exchange [c_current_cdemo_sk] #7 + WholeStageCodegen (7) + Project [c_customer_sk,c_current_cdemo_sk,c_birth_year,ca_county,ca_state,ca_country] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Project [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] Filter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - InputAdapter - WholeStageCodegen (9) - Sort [cd_demo_sk] - InputAdapter - Exchange [cd_demo_sk] #8 - WholeStageCodegen (8) - Filter [cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (6) + Filter [ca_state,ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (10) - Filter [ca_state,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + WholeStageCodegen (10) + Sort [cd_demo_sk] + InputAdapter + Exchange [cd_demo_sk] #9 + WholeStageCodegen (9) + Filter [cd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_demographics [cd_demo_sk] WholeStageCodegen (28) 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] InputAdapter @@ -130,7 +130,7 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (24) Sort [cd_demo_sk] InputAdapter - ReusedExchange [cd_demo_sk] #8 + ReusedExchange [cd_demo_sk] #9 WholeStageCodegen (42) HashAggregate [i_item_id,ca_country,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),ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter @@ -177,7 +177,7 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (38) Sort [cd_demo_sk] InputAdapter - ReusedExchange [cd_demo_sk] #8 + ReusedExchange [cd_demo_sk] #9 WholeStageCodegen (50) HashAggregate [i_item_id,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),ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt index 15cde1a45a99b..3c2fc95d274b6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt @@ -8,24 +8,24 @@ TakeOrderedAndProject (27) +- Exchange (21) +- * HashAggregate (20) +- * Project (19) - +- * SortMergeJoin Inner (18) - :- * Sort (12) - : +- Exchange (11) - : +- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.catalog_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.date_dim (4) - +- * Sort (17) - +- Exchange (16) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (12) + : +- * SortMergeJoin Inner (11) + : :- * Sort (5) + : : +- Exchange (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.catalog_sales (1) + : +- * Sort (10) + : +- Exchange (9) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.item (6) + +- BroadcastExchange (17) + +- * Project (16) +- * Filter (15) +- * ColumnarToRow (14) - +- Scan parquet default.item (13) + +- Scan parquet default.date_dim (13) (1) Scan parquet default.catalog_sales @@ -35,118 +35,118 @@ Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_sold_date_sk#1)) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct +(4) Exchange +Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] +Arguments: hashpartitioning(cs_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#4] -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] +(5) Sort [codegen id : 2] +Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] +Arguments: [cs_item_sk#2 ASC NULLS FIRST], false, 0 -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] -Condition : (((isnotnull(d_date#5) AND (d_date#5 >= 10644)) AND (d_date#5 <= 10674)) AND isnotnull(d_date_sk#4)) +(6) Scan parquet default.item +Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_date#5] +(7) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] +(8) Filter [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None +(9) Exchange +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: hashpartitioning(i_item_sk#5, 5), ENSURE_REQUIREMENTS, [id=#11] -(10) Project [codegen id : 2] -Output [2]: [cs_item_sk#2, cs_ext_sales_price#3] -Input [4]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3, d_date_sk#4] +(10) Sort [codegen id : 4] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5 ASC NULLS FIRST], false, 0 -(11) Exchange -Input [2]: [cs_item_sk#2, cs_ext_sales_price#3] -Arguments: hashpartitioning(cs_item_sk#2, 5), true, [id=#7] +(11) SortMergeJoin [codegen id : 6] +Left keys [1]: [cs_item_sk#2] +Right keys [1]: [i_item_sk#5] +Join condition: None -(12) Sort [codegen id : 3] -Input [2]: [cs_item_sk#2, cs_ext_sales_price#3] -Arguments: [cs_item_sk#2 ASC NULLS FIRST], false, 0 +(12) Project [codegen id : 6] +Output [7]: [cs_sold_date_sk#1, cs_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [9]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) Scan parquet default.item -Output [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(13) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_date#13] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(14) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(14) ColumnarToRow [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] -(15) Filter [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Condition : (i_category#13 IN (Sports,Books,Home) AND isnotnull(i_item_sk#8)) +(15) Filter [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) -(16) Exchange -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: hashpartitioning(i_item_sk#8, 5), true, [id=#14] +(16) Project [codegen id : 5] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_date#13] -(17) Sort [codegen id : 5] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: [i_item_sk#8 ASC NULLS FIRST], false, 0 +(17) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [i_item_sk#8] +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_sold_date_sk#1] +Right keys [1]: [d_date_sk#12] Join condition: None (19) Project [codegen id : 6] -Output [6]: [cs_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Input [8]: [cs_item_sk#2, cs_ext_sales_price#3, i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +Output [6]: [cs_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [8]: [cs_sold_date_sk#1, cs_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#12] (20) HashAggregate [codegen id : 6] -Input [6]: [cs_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [cs_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#3))] Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] (21) Exchange -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Arguments: hashpartitioning(i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, 5), true, [id=#17] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [id=#17] (22) HashAggregate [codegen id : 7] -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#3))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#3))#18] -Results [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS _w1#21] +Results [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS _w1#21] (23) Exchange -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: hashpartitioning(i_class#12, 5), true, [id=#22] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [id=#22] (24) Sort [codegen id : 8] -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: [i_class#12 ASC NULLS FIRST], false, 0 +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 (25) Window -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: [sum(_w1#21) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#12] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: [sum(_w1#21) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#9] (26) Project [codegen id : 9] -Output [7]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24] -Input [9]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, _we0#23] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24] +Input [9]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, _we0#23] (27) TakeOrderedAndProject -Input [7]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST, i_item_desc#10 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/simplified.txt index e66efff75180a..3f5e15881c33c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/simplified.txt @@ -14,34 +14,34 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (6) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - SortMergeJoin [cs_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (3) - Sort [cs_item_sk] - InputAdapter - Exchange [cs_item_sk] #3 - WholeStageCodegen (2) - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + SortMergeJoin [cs_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [cs_item_sk] + InputAdapter + Exchange [cs_item_sk] #3 + WholeStageCodegen (1) Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_ext_sales_price] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #4 + WholeStageCodegen (3) + Filter [i_category,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter - WholeStageCodegen (5) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #5 - WholeStageCodegen (4) - Filter [i_category,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastExchange #5 + WholeStageCodegen (5) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.sf100/explain.txt index 3efe02a377d09..0c805bef9c1f1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.sf100/explain.txt @@ -1,32 +1,33 @@ == Physical Plan == -TakeOrderedAndProject (28) -+- * HashAggregate (27) - +- Exchange (26) - +- * HashAggregate (25) - +- * Expand (24) - +- BroadcastNestedLoopJoin Inner BuildRight (23) - :- * Project (19) - : +- * SortMergeJoin Inner (18) - : :- * Sort (12) - : : +- Exchange (11) - : : +- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.inventory (1) - : : +- BroadcastExchange (8) - : : +- * Project (7) - : : +- * Filter (6) - : : +- * ColumnarToRow (5) - : : +- Scan parquet default.date_dim (4) - : +- * Sort (17) - : +- Exchange (16) - : +- * Filter (15) - : +- * ColumnarToRow (14) - : +- Scan parquet default.item (13) - +- BroadcastExchange (22) - +- * ColumnarToRow (21) - +- Scan parquet default.warehouse (20) +TakeOrderedAndProject (29) ++- * HashAggregate (28) + +- Exchange (27) + +- * HashAggregate (26) + +- * Expand (25) + +- * Project (24) + +- BroadcastNestedLoopJoin Inner BuildRight (23) + :- * Project (19) + : +- * SortMergeJoin Inner (18) + : :- * Sort (12) + : : +- Exchange (11) + : : +- * Project (10) + : : +- * BroadcastHashJoin Inner BuildRight (9) + : : :- * Filter (3) + : : : +- * ColumnarToRow (2) + : : : +- Scan parquet default.inventory (1) + : : +- BroadcastExchange (8) + : : +- * Project (7) + : : +- * Filter (6) + : : +- * ColumnarToRow (5) + : : +- Scan parquet default.date_dim (4) + : +- * Sort (17) + : +- Exchange (16) + : +- * Filter (15) + : +- * ColumnarToRow (14) + : +- Scan parquet default.item (13) + +- BroadcastExchange (22) + +- * ColumnarToRow (21) + +- Scan parquet default.warehouse (20) (1) Scan parquet default.inventory @@ -76,7 +77,7 @@ Input [4]: [inv_date_sk#1, inv_item_sk#2, inv_quantity_on_hand#3, d_date_sk#4] (11) Exchange Input [2]: [inv_item_sk#2, inv_quantity_on_hand#3] -Arguments: hashpartitioning(inv_item_sk#2, 5), true, [id=#7] +Arguments: hashpartitioning(inv_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#7] (12) Sort [codegen id : 3] Input [2]: [inv_item_sk#2, inv_quantity_on_hand#3] @@ -98,7 +99,7 @@ Condition : isnotnull(i_item_sk#8) (16) Exchange Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Arguments: hashpartitioning(i_item_sk#8, 5), true, [id=#13] +Arguments: hashpartitioning(i_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#13] (17) Sort [codegen id : 5] Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] @@ -129,29 +130,33 @@ Arguments: IdentityBroadcastMode, [id=#14] (23) BroadcastNestedLoopJoin Join condition: None -(24) Expand [codegen id : 8] +(24) Project [codegen id : 8] +Output [5]: [inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, i_category#11] Input [5]: [inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12] + +(25) Expand [codegen id : 8] +Input [5]: [inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, i_category#11] Arguments: [List(inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, i_category#11, 0), List(inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, null, 1), List(inv_quantity_on_hand#3, i_product_name#12, i_brand#9, null, null, 3), List(inv_quantity_on_hand#3, i_product_name#12, null, null, null, 7), List(inv_quantity_on_hand#3, null, null, null, null, 15)], [inv_quantity_on_hand#3, i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] -(25) HashAggregate [codegen id : 8] +(26) HashAggregate [codegen id : 8] Input [6]: [inv_quantity_on_hand#3, i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] Keys [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] Functions [1]: [partial_avg(cast(inv_quantity_on_hand#3 as bigint))] Aggregate Attributes [2]: [sum#20, count#21] Results [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] -(26) Exchange +(27) Exchange Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] -Arguments: hashpartitioning(i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, 5), true, [id=#24] +Arguments: hashpartitioning(i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, 5), ENSURE_REQUIREMENTS, [id=#24] -(27) HashAggregate [codegen id : 9] +(28) HashAggregate [codegen id : 9] Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] Keys [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] Functions [1]: [avg(cast(inv_quantity_on_hand#3 as bigint))] Aggregate Attributes [1]: [avg(cast(inv_quantity_on_hand#3 as bigint))#25] Results [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, avg(cast(inv_quantity_on_hand#3 as bigint))#25 AS qoh#26] -(28) TakeOrderedAndProject +(29) TakeOrderedAndProject Input [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#26] Arguments: 100, [qoh#26 ASC NULLS FIRST, i_product_name#15 ASC NULLS FIRST, i_brand#16 ASC NULLS FIRST, i_class#17 ASC NULLS FIRST, i_category#18 ASC NULLS FIRST], [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.sf100/simplified.txt index d5f40d419175e..2d2e46ddd7013 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.sf100/simplified.txt @@ -6,43 +6,44 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] WholeStageCodegen (8) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - InputAdapter - BroadcastNestedLoopJoin - WholeStageCodegen (6) - Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - SortMergeJoin [inv_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (3) - Sort [inv_item_sk] - InputAdapter - Exchange [inv_item_sk] #2 - WholeStageCodegen (2) - Project [inv_item_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Filter [inv_date_sk,inv_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq] - InputAdapter - WholeStageCodegen (5) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #4 - WholeStageCodegen (4) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - BroadcastExchange #5 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - Scan parquet default.warehouse + Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + InputAdapter + BroadcastNestedLoopJoin + WholeStageCodegen (6) + Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + SortMergeJoin [inv_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (3) + Sort [inv_item_sk] + InputAdapter + Exchange [inv_item_sk] #2 + WholeStageCodegen (2) + Project [inv_item_sk,inv_quantity_on_hand] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Filter [inv_date_sk,inv_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_month_seq] + InputAdapter + WholeStageCodegen (5) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #4 + WholeStageCodegen (4) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + BroadcastExchange #5 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + Scan parquet default.warehouse diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt index ad83edec0ad33..f2a95f704db57 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt @@ -1,29 +1,30 @@ == Physical Plan == -TakeOrderedAndProject (25) -+- * HashAggregate (24) - +- Exchange (23) - +- * HashAggregate (22) - +- * Expand (21) - +- BroadcastNestedLoopJoin Inner BuildRight (20) - :- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.inventory (1) - : : +- BroadcastExchange (8) - : : +- * Project (7) - : : +- * Filter (6) - : : +- * ColumnarToRow (5) - : : +- Scan parquet default.date_dim (4) - : +- BroadcastExchange (14) - : +- * Filter (13) - : +- * ColumnarToRow (12) - : +- Scan parquet default.item (11) - +- BroadcastExchange (19) - +- * ColumnarToRow (18) - +- Scan parquet default.warehouse (17) +TakeOrderedAndProject (26) ++- * HashAggregate (25) + +- Exchange (24) + +- * HashAggregate (23) + +- * Expand (22) + +- * Project (21) + +- BroadcastNestedLoopJoin Inner BuildRight (20) + :- * Project (16) + : +- * BroadcastHashJoin Inner BuildRight (15) + : :- * Project (10) + : : +- * BroadcastHashJoin Inner BuildRight (9) + : : :- * Filter (3) + : : : +- * ColumnarToRow (2) + : : : +- Scan parquet default.inventory (1) + : : +- BroadcastExchange (8) + : : +- * Project (7) + : : +- * Filter (6) + : : +- * ColumnarToRow (5) + : : +- Scan parquet default.date_dim (4) + : +- BroadcastExchange (14) + : +- * Filter (13) + : +- * ColumnarToRow (12) + : +- Scan parquet default.item (11) + +- BroadcastExchange (19) + +- * ColumnarToRow (18) + +- Scan parquet default.warehouse (17) (1) Scan parquet default.inventory @@ -114,29 +115,33 @@ Arguments: IdentityBroadcastMode, [id=#13] (20) BroadcastNestedLoopJoin Join condition: None -(21) Expand [codegen id : 5] +(21) Project [codegen id : 5] +Output [5]: [inv_quantity_on_hand#3, i_product_name#11, i_brand#8, i_class#9, i_category#10] Input [5]: [inv_quantity_on_hand#3, i_brand#8, i_class#9, i_category#10, i_product_name#11] + +(22) Expand [codegen id : 5] +Input [5]: [inv_quantity_on_hand#3, i_product_name#11, i_brand#8, i_class#9, i_category#10] Arguments: [List(inv_quantity_on_hand#3, i_product_name#11, i_brand#8, i_class#9, i_category#10, 0), List(inv_quantity_on_hand#3, i_product_name#11, i_brand#8, i_class#9, null, 1), List(inv_quantity_on_hand#3, i_product_name#11, i_brand#8, null, null, 3), List(inv_quantity_on_hand#3, i_product_name#11, null, null, null, 7), List(inv_quantity_on_hand#3, null, null, null, null, 15)], [inv_quantity_on_hand#3, i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18] -(22) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 5] Input [6]: [inv_quantity_on_hand#3, i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18] Keys [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18] Functions [1]: [partial_avg(cast(inv_quantity_on_hand#3 as bigint))] Aggregate Attributes [2]: [sum#19, count#20] Results [7]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18, sum#21, count#22] -(23) Exchange +(24) Exchange Input [7]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18, sum#21, count#22] -Arguments: hashpartitioning(i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18, 5), true, [id=#23] +Arguments: hashpartitioning(i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18, 5), ENSURE_REQUIREMENTS, [id=#23] -(24) HashAggregate [codegen id : 6] +(25) HashAggregate [codegen id : 6] Input [7]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18, sum#21, count#22] Keys [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18] Functions [1]: [avg(cast(inv_quantity_on_hand#3 as bigint))] Aggregate Attributes [1]: [avg(cast(inv_quantity_on_hand#3 as bigint))#24] Results [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, avg(cast(inv_quantity_on_hand#3 as bigint))#24 AS qoh#25] -(25) TakeOrderedAndProject +(26) TakeOrderedAndProject Input [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#25] Arguments: 100, [qoh#25 ASC NULLS FIRST, i_product_name#14 ASC NULLS FIRST, i_brand#15 ASC NULLS FIRST, i_class#16 ASC NULLS FIRST, i_category#17 ASC NULLS FIRST], [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#25] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt index cdf9335b1c48b..559e96388689b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt @@ -6,34 +6,35 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] WholeStageCodegen (5) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - InputAdapter - BroadcastNestedLoopJoin - WholeStageCodegen (3) - Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Filter [inv_date_sk,inv_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [i_item_sk] + Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + InputAdapter + BroadcastNestedLoopJoin + WholeStageCodegen (3) + Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Project [inv_item_sk,inv_quantity_on_hand] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Filter [inv_date_sk,inv_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - BroadcastExchange #4 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - Scan parquet default.warehouse + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_month_seq] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + BroadcastExchange #4 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + Scan parquet default.warehouse diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt index 04ff822b1ce52..a993f22e559ab 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt @@ -11,60 +11,60 @@ TakeOrderedAndProject (79) : +- * BroadcastHashJoin LeftOuter BuildRight (65) : :- * Project (60) : : +- * SortMergeJoin Inner (59) - : : :- * Sort (34) - : : : +- Exchange (33) - : : : +- * Project (32) - : : : +- * SortMergeJoin Inner (31) - : : : :- * Sort (25) - : : : : +- Exchange (24) - : : : : +- * Project (23) - : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : :- * Project (17) - : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet default.catalog_sales (1) - : : : : : : +- BroadcastExchange (8) - : : : : : : +- * Project (7) - : : : : : : +- * Filter (6) - : : : : : : +- * ColumnarToRow (5) - : : : : : : +- Scan parquet default.household_demographics (4) - : : : : : +- BroadcastExchange (15) - : : : : : +- * Project (14) - : : : : : +- * Filter (13) - : : : : : +- * ColumnarToRow (12) - : : : : : +- Scan parquet default.customer_demographics (11) - : : : : +- BroadcastExchange (21) - : : : : +- * Filter (20) - : : : : +- * ColumnarToRow (19) - : : : : +- Scan parquet default.date_dim (18) - : : : +- * Sort (30) - : : : +- Exchange (29) - : : : +- * Filter (28) - : : : +- * ColumnarToRow (27) - : : : +- Scan parquet default.item (26) + : : :- * Sort (47) + : : : +- Exchange (46) + : : : +- * Project (45) + : : : +- * BroadcastHashJoin Inner BuildRight (44) + : : : :- * Project (32) + : : : : +- * SortMergeJoin Inner (31) + : : : : :- * Sort (25) + : : : : : +- Exchange (24) + : : : : : +- * Project (23) + : : : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : : : :- * Project (17) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : : : : :- * Project (10) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : : : :- * Filter (3) + : : : : : : : : +- * ColumnarToRow (2) + : : : : : : : : +- Scan parquet default.catalog_sales (1) + : : : : : : : +- BroadcastExchange (8) + : : : : : : : +- * Project (7) + : : : : : : : +- * Filter (6) + : : : : : : : +- * ColumnarToRow (5) + : : : : : : : +- Scan parquet default.household_demographics (4) + : : : : : : +- BroadcastExchange (15) + : : : : : : +- * Project (14) + : : : : : : +- * Filter (13) + : : : : : : +- * ColumnarToRow (12) + : : : : : : +- Scan parquet default.customer_demographics (11) + : : : : : +- BroadcastExchange (21) + : : : : : +- * Filter (20) + : : : : : +- * ColumnarToRow (19) + : : : : : +- Scan parquet default.date_dim (18) + : : : : +- * Sort (30) + : : : : +- Exchange (29) + : : : : +- * Filter (28) + : : : : +- * ColumnarToRow (27) + : : : : +- Scan parquet default.item (26) + : : : +- BroadcastExchange (43) + : : : +- * Project (42) + : : : +- * BroadcastHashJoin Inner BuildLeft (41) + : : : :- BroadcastExchange (37) + : : : : +- * Project (36) + : : : : +- * Filter (35) + : : : : +- * ColumnarToRow (34) + : : : : +- Scan parquet default.date_dim (33) + : : : +- * Filter (40) + : : : +- * ColumnarToRow (39) + : : : +- Scan parquet default.date_dim (38) : : +- * Sort (58) : : +- Exchange (57) : : +- * Project (56) : : +- * BroadcastHashJoin Inner BuildRight (55) - : : :- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildLeft (49) - : : : :- BroadcastExchange (45) - : : : : +- * Project (44) - : : : : +- * BroadcastHashJoin Inner BuildLeft (43) - : : : : :- BroadcastExchange (39) - : : : : : +- * Project (38) - : : : : : +- * Filter (37) - : : : : : +- * ColumnarToRow (36) - : : : : : +- Scan parquet default.date_dim (35) - : : : : +- * Filter (42) - : : : : +- * ColumnarToRow (41) - : : : : +- Scan parquet default.date_dim (40) - : : : +- * Filter (48) - : : : +- * ColumnarToRow (47) - : : : +- Scan parquet default.inventory (46) + : : :- * Filter (50) + : : : +- * ColumnarToRow (49) + : : : +- Scan parquet default.inventory (48) : : +- BroadcastExchange (54) : : +- * Filter (53) : : +- * ColumnarToRow (52) @@ -98,7 +98,7 @@ Condition : (((((isnotnull(cs_quantity#8) AND isnotnull(cs_item_sk#5)) AND isnot Output [2]: [hd_demo_sk#9, hd_buy_potential#10] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000 ), IsNotNull(hd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -106,7 +106,7 @@ Input [2]: [hd_demo_sk#9, hd_buy_potential#10] (6) Filter [codegen id : 1] Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = 1001-5000)) AND isnotnull(hd_demo_sk#9)) +Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = 1001-5000 )) AND isnotnull(hd_demo_sk#9)) (7) Project [codegen id : 1] Output [1]: [hd_demo_sk#9] @@ -213,95 +213,95 @@ Arguments: hashpartitioning(i_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#21] Input [2]: [i_item_sk#19, i_item_desc#20] Arguments: [i_item_sk#19 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin [codegen id : 8] +(31) SortMergeJoin [codegen id : 10] Left keys [1]: [cs_item_sk#5] Right keys [1]: [i_item_sk#19] Join condition: None -(32) Project [codegen id : 8] +(32) Project [codegen id : 10] Output [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] Input [8]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_sk#19, i_item_desc#20] -(33) Exchange -Input [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] -Arguments: hashpartitioning(cs_item_sk#5, cs_sold_date_sk#1, 5), ENSURE_REQUIREMENTS, [id=#22] - -(34) Sort [codegen id : 9] -Input [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] -Arguments: [cs_item_sk#5 ASC NULLS FIRST, cs_sold_date_sk#1 ASC NULLS FIRST], false, 0 - -(35) Scan parquet default.date_dim -Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] +(33) Scan parquet default.date_dim +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct -(36) ColumnarToRow [codegen id : 10] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] +(34) ColumnarToRow [codegen id : 8] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -(37) Filter [codegen id : 10] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] -Condition : ((((isnotnull(d_year#26) AND (d_year#26 = 2001)) AND isnotnull(d_date_sk#23)) AND isnotnull(d_week_seq#25)) AND isnotnull(d_date#24)) +(35) Filter [codegen id : 8] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) -(38) Project [codegen id : 10] -Output [3]: [d_date_sk#23, d_date#24, d_week_seq#25] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] +(36) Project [codegen id : 8] +Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -(39) BroadcastExchange -Input [3]: [d_date_sk#23, d_date#24, d_week_seq#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#27] +(37) BroadcastExchange +Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#26] -(40) Scan parquet default.date_dim -Output [2]: [d_date_sk#28, d_week_seq#29] +(38) Scan parquet default.date_dim +Output [2]: [d_date_sk#27, d_week_seq#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(41) ColumnarToRow -Input [2]: [d_date_sk#28, d_week_seq#29] +(39) ColumnarToRow +Input [2]: [d_date_sk#27, d_week_seq#28] -(42) Filter -Input [2]: [d_date_sk#28, d_week_seq#29] -Condition : (isnotnull(d_week_seq#29) AND isnotnull(d_date_sk#28)) +(40) Filter +Input [2]: [d_date_sk#27, d_week_seq#28] +Condition : (isnotnull(d_week_seq#28) AND isnotnull(d_date_sk#27)) -(43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_week_seq#25] -Right keys [1]: [d_week_seq#29] +(41) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [d_week_seq#24] +Right keys [1]: [d_week_seq#28] Join condition: None -(44) Project [codegen id : 11] -Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28] -Input [5]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28, d_week_seq#29] +(42) Project [codegen id : 9] +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] +Input [5]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27, d_week_seq#28] + +(43) BroadcastExchange +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] + +(44) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_sold_date_sk#1] +Right keys [1]: [d_date_sk#22] +Join condition: (d_date#16 > d_date#23 + 5 days) + +(45) Project [codegen id : 10] +Output [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Input [11]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20, d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] -(45) BroadcastExchange -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[3, int, true] as bigint)),false), [id=#30] +(46) Exchange +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Arguments: hashpartitioning(cs_item_sk#5, d_date_sk#27, 5), ENSURE_REQUIREMENTS, [id=#30] -(46) Scan parquet default.inventory +(47) Sort [codegen id : 11] +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Arguments: [cs_item_sk#5 ASC NULLS FIRST, d_date_sk#27 ASC NULLS FIRST], false, 0 + +(48) Scan parquet default.inventory Output [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] Batched: true Location [not included in comparison]/{warehouse_dir}/inventory] PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk), IsNotNull(inv_date_sk)] ReadSchema: struct -(47) ColumnarToRow +(49) ColumnarToRow [codegen id : 13] Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] -(48) Filter +(50) Filter [codegen id : 13] Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] Condition : (((isnotnull(inv_quantity_on_hand#34) AND isnotnull(inv_item_sk#32)) AND isnotnull(inv_warehouse_sk#33)) AND isnotnull(inv_date_sk#31)) -(49) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [d_date_sk#28] -Right keys [1]: [inv_date_sk#31] -Join condition: None - -(50) Project [codegen id : 13] -Output [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] -Input [8]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28, inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] - (51) Scan parquet default.warehouse Output [2]: [w_warehouse_sk#35, w_warehouse_name#36] Batched: true @@ -326,25 +326,25 @@ Right keys [1]: [w_warehouse_sk#35] Join condition: None (56) Project [codegen id : 13] -Output [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Input [8]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, w_warehouse_sk#35, w_warehouse_name#36] +Output [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Input [6]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, w_warehouse_sk#35, w_warehouse_name#36] (57) Exchange -Input [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Arguments: hashpartitioning(inv_item_sk#32, d_date_sk#23, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Arguments: hashpartitioning(inv_item_sk#32, inv_date_sk#31, 5), ENSURE_REQUIREMENTS, [id=#38] (58) Sort [codegen id : 14] -Input [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Arguments: [inv_item_sk#32 ASC NULLS FIRST, d_date_sk#23 ASC NULLS FIRST], false, 0 +Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Arguments: [inv_item_sk#32 ASC NULLS FIRST, inv_date_sk#31 ASC NULLS FIRST], false, 0 (59) SortMergeJoin [codegen id : 16] -Left keys [2]: [cs_item_sk#5, cs_sold_date_sk#1] -Right keys [2]: [inv_item_sk#32, d_date_sk#23] -Join condition: ((inv_quantity_on_hand#34 < cs_quantity#8) AND (d_date#16 > d_date#24 + 5 days)) +Left keys [2]: [cs_item_sk#5, d_date_sk#27] +Right keys [2]: [inv_item_sk#32, inv_date_sk#31] +Join condition: (inv_quantity_on_hand#34 < cs_quantity#8) (60) Project [codegen id : 16] -Output [6]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] -Input [13]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20, d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Output [6]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Input [11]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27, inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] (61) Scan parquet default.promotion Output [1]: [p_promo_sk#39] @@ -370,15 +370,15 @@ Right keys [1]: [p_promo_sk#39] Join condition: None (66) Project [codegen id : 16] -Output [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] -Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25, p_promo_sk#39] +Output [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24, p_promo_sk#39] (67) Exchange -Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] Arguments: hashpartitioning(cs_item_sk#5, cs_order_number#7, 5), ENSURE_REQUIREMENTS, [id=#41] (68) Sort [codegen id : 17] -Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] Arguments: [cs_item_sk#5 ASC NULLS FIRST, cs_order_number#7 ASC NULLS FIRST], false, 0 (69) Scan parquet default.catalog_returns @@ -409,28 +409,28 @@ Right keys [2]: [cr_item_sk#42, cr_order_number#43] Join condition: None (75) Project [codegen id : 20] -Output [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#25] -Input [7]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25, cr_item_sk#42, cr_order_number#43] +Output [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Input [7]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24, cr_item_sk#42, cr_order_number#43] (76) HashAggregate [codegen id : 20] -Input [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#25] -Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25] +Input [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#45] -Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] +Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] (77) Exchange -Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] -Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#36, d_week_seq#25, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] +Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#36, d_week_seq#24, 5), ENSURE_REQUIREMENTS, [id=#47] (78) HashAggregate [codegen id : 21] -Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] -Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25] +Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] +Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#48] -Results [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count(1)#48 AS no_promo#49, count(1)#48 AS promo#50, count(1)#48 AS total_cnt#51] +Results [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count(1)#48 AS no_promo#49, count(1)#48 AS promo#50, count(1)#48 AS total_cnt#51] (79) TakeOrderedAndProject -Input [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, no_promo#49, promo#50, total_cnt#51] -Arguments: 100, [total_cnt#51 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#36 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, no_promo#49, promo#50, total_cnt#51] +Input [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, no_promo#49, promo#50, total_cnt#51] +Arguments: 100, [total_cnt#51 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#36 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, no_promo#49, promo#50, total_cnt#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt index b88505ad7b9bc..918508787c4b0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt @@ -16,95 +16,95 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] BroadcastHashJoin [cs_promo_sk,p_promo_sk] Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - SortMergeJoin [cs_item_sk,cs_sold_date_sk,inv_item_sk,d_date_sk,inv_quantity_on_hand,cs_quantity,d_date,d_date] + SortMergeJoin [cs_item_sk,d_date_sk,inv_item_sk,inv_date_sk,inv_quantity_on_hand,cs_quantity] InputAdapter - WholeStageCodegen (9) - Sort [cs_item_sk,cs_sold_date_sk] + WholeStageCodegen (11) + Sort [cs_item_sk,d_date_sk] InputAdapter - Exchange [cs_item_sk,cs_sold_date_sk] #3 - WholeStageCodegen (8) - Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date,i_item_desc] - SortMergeJoin [cs_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (5) - Sort [cs_item_sk] - InputAdapter - Exchange [cs_item_sk] #4 - WholeStageCodegen (4) - Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] - ColumnarToRow + Exchange [cs_item_sk,d_date_sk] #3 + WholeStageCodegen (10) + Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_week_seq,d_date_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date] + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date,i_item_desc] + SortMergeJoin [cs_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (5) + Sort [cs_item_sk] + InputAdapter + Exchange [cs_item_sk] #4 + WholeStageCodegen (4) + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastExchange #5 + WholeStageCodegen (1) + Project [hd_demo_sk] + Filter [hd_buy_potential,hd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk] + BroadcastExchange #6 + WholeStageCodegen (2) + Project [cd_demo_sk] + Filter [cd_marital_status,cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [cd_demo_sk] - Filter [cd_marital_status,cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + BroadcastExchange #7 + WholeStageCodegen (3) + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (7) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #8 + WholeStageCodegen (6) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_desc] InputAdapter - WholeStageCodegen (7) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #8 - WholeStageCodegen (6) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_desc] + BroadcastExchange #9 + WholeStageCodegen (9) + Project [d_date_sk,d_date,d_week_seq,d_date_sk] + BroadcastHashJoin [d_week_seq,d_week_seq] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + Project [d_date_sk,d_date,d_week_seq] + Filter [d_year,d_date_sk,d_week_seq,d_date] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + Filter [d_week_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter WholeStageCodegen (14) - Sort [inv_item_sk,d_date_sk] + Sort [inv_item_sk,inv_date_sk] InputAdapter - Exchange [inv_item_sk,d_date_sk] #9 + Exchange [inv_item_sk,inv_date_sk] #11 WholeStageCodegen (13) - Project [d_date_sk,d_date,d_week_seq,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] + Project [inv_date_sk,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [d_date_sk,d_date,d_week_seq,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - BroadcastHashJoin [d_date_sk,inv_date_sk] + Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #10 - WholeStageCodegen (11) - Project [d_date_sk,d_date,d_week_seq,d_date_sk] - BroadcastHashJoin [d_week_seq,d_week_seq] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (10) - Project [d_date_sk,d_date,d_week_seq] - Filter [d_year,d_date_sk,d_week_seq,d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - Filter [d_week_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq] - Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #12 WholeStageCodegen (12) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt index 6fa7b04a3b463..694b1ead8d0f0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt @@ -9,24 +9,24 @@ +- Exchange (21) +- * HashAggregate (20) +- * Project (19) - +- * SortMergeJoin Inner (18) - :- * Sort (12) - : +- Exchange (11) - : +- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.store_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.date_dim (4) - +- * Sort (17) - +- Exchange (16) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (12) + : +- * SortMergeJoin Inner (11) + : :- * Sort (5) + : : +- Exchange (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.store_sales (1) + : +- * Sort (10) + : +- Exchange (9) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.item (6) + +- BroadcastExchange (17) + +- * Project (16) +- * Filter (15) +- * ColumnarToRow (14) - +- Scan parquet default.item (13) + +- Scan parquet default.date_dim (13) (1) Scan parquet default.store_sales @@ -36,122 +36,122 @@ Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct +(4) Exchange +Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +Arguments: hashpartitioning(ss_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#4] -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] +(5) Sort [codegen id : 2] +Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] -Condition : (((isnotnull(d_date#5) AND (d_date#5 >= 10644)) AND (d_date#5 <= 10674)) AND isnotnull(d_date_sk#4)) +(6) Scan parquet default.item +Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_date#5] +(7) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] +(8) Filter [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None +(9) Exchange +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: hashpartitioning(i_item_sk#5, 5), ENSURE_REQUIREMENTS, [id=#11] -(10) Project [codegen id : 2] -Output [2]: [ss_item_sk#2, ss_ext_sales_price#3] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3, d_date_sk#4] +(10) Sort [codegen id : 4] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5 ASC NULLS FIRST], false, 0 -(11) Exchange -Input [2]: [ss_item_sk#2, ss_ext_sales_price#3] -Arguments: hashpartitioning(ss_item_sk#2, 5), true, [id=#7] +(11) SortMergeJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#2] +Right keys [1]: [i_item_sk#5] +Join condition: None -(12) Sort [codegen id : 3] -Input [2]: [ss_item_sk#2, ss_ext_sales_price#3] -Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 +(12) Project [codegen id : 6] +Output [7]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) Scan parquet default.item -Output [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(13) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_date#13] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(14) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(14) ColumnarToRow [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] -(15) Filter [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Condition : (i_category#13 IN (Sports,Books,Home) AND isnotnull(i_item_sk#8)) +(15) Filter [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) -(16) Exchange -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: hashpartitioning(i_item_sk#8, 5), true, [id=#14] +(16) Project [codegen id : 5] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_date#13] -(17) Sort [codegen id : 5] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: [i_item_sk#8 ASC NULLS FIRST], false, 0 +(17) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#8] +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#12] Join condition: None (19) Project [codegen id : 6] -Output [6]: [ss_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Input [8]: [ss_item_sk#2, ss_ext_sales_price#3, i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +Output [6]: [ss_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [8]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#12] (20) HashAggregate [codegen id : 6] -Input [6]: [ss_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [ss_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] (21) Exchange -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Arguments: hashpartitioning(i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, 5), true, [id=#17] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [id=#17] (22) HashAggregate [codegen id : 7] -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#18] -Results [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS _w1#21] +Results [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS _w1#21] (23) Exchange -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: hashpartitioning(i_class#12, 5), true, [id=#22] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [id=#22] (24) Sort [codegen id : 8] -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: [i_class#12 ASC NULLS FIRST], false, 0 +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 (25) Window -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: [sum(_w1#21) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#12] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: [sum(_w1#21) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#9] (26) Project [codegen id : 9] -Output [7]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24] -Input [9]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, _we0#23] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24] +Input [9]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, _we0#23] (27) Exchange -Input [7]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST, i_item_desc#10 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST, 5), true, [id=#25] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] +Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#25] (28) Sort [codegen id : 10] -Input [7]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] -Arguments: [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST, i_item_desc#10 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], true, 0 +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] +Arguments: [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/simplified.txt index ded65becfe826..7066245b3973e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/simplified.txt @@ -17,34 +17,34 @@ WholeStageCodegen (10) WholeStageCodegen (6) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - SortMergeJoin [ss_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (3) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #4 - WholeStageCodegen (2) - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + SortMergeJoin [ss_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #4 + WholeStageCodegen (1) Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #5 + WholeStageCodegen (3) + Filter [i_category,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter - WholeStageCodegen (5) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #6 - WholeStageCodegen (4) - Filter [i_category,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastExchange #6 + WholeStageCodegen (5) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala index 177517236ef88..70e69843f743c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala @@ -584,21 +584,6 @@ trait CharVarcharTestSuite extends QueryTest with SQLTestUtils { } } - test("SPARK-33992: char/varchar resolution in correlated sub query") { - withTable("t1", "t2") { - sql(s"CREATE TABLE t1(v VARCHAR(3), c CHAR(5)) USING $format") - sql(s"CREATE TABLE t2(v VARCHAR(3), c CHAR(5)) USING $format") - sql("INSERT INTO t1 VALUES ('c', 'b')") - sql("INSERT INTO t2 VALUES ('a', 'b')") - - checkAnswer(sql( - """ - |SELECT v FROM t1 - |WHERE 'a' IN (SELECT v FROM t2 WHERE t1.c = t2.c )""".stripMargin), - Row("c")) - } - } - test("SPARK-34003: fix char/varchar fails w/ both group by and order by ") { withTable("t") { sql(s"CREATE TABLE t(v VARCHAR(3), i INT) USING $format") @@ -633,6 +618,48 @@ trait CharVarcharTestSuite extends QueryTest with SQLTestUtils { checkAnswer(spark.table("t"), Row("c ")) } } + + test("SPARK-34833: right-padding applied correctly for correlated subqueries - join keys") { + withTable("t1", "t2") { + sql(s"CREATE TABLE t1(v VARCHAR(3), c CHAR(5)) USING $format") + sql(s"CREATE TABLE t2(v VARCHAR(5), c CHAR(8)) USING $format") + sql("INSERT INTO t1 VALUES ('c', 'b')") + sql("INSERT INTO t2 VALUES ('a', 'b')") + Seq("t1.c = t2.c", "t2.c = t1.c", + "t1.c = 'b'", "'b' = t1.c", "t1.c = 'b '", "'b ' = t1.c", + "t1.c = 'b '", "'b ' = t1.c").foreach { predicate => + checkAnswer(sql( + s""" + |SELECT v FROM t1 + |WHERE 'a' IN (SELECT v FROM t2 WHERE $predicate) + """.stripMargin), + Row("c")) + } + } + } + + test("SPARK-34833: right-padding applied correctly for correlated subqueries - other preds") { + withTable("t") { + sql(s"CREATE TABLE t(c0 INT, c1 CHAR(5), c2 CHAR(7)) USING $format") + sql("INSERT INTO t VALUES (1, 'abc', 'abc')") + Seq("c1 = 'abc'", "'abc' = c1", "c1 = 'abc '", "'abc ' = c1", + "c1 = 'abc '", "'abc ' = c1", "c1 = c2", "c2 = c1", + "c1 IN ('xxx', 'abc', 'xxxxx')", "c1 IN ('xxx', 'abc ', 'xxxxx')", + "c1 IN ('xxx', 'abc ', 'xxxxx')", + "c1 IN (c2)", "c2 IN (c1)").foreach { predicate => + checkAnswer(sql( + s""" + |SELECT c0 FROM t t1 + |WHERE ( + | SELECT count(*) AS c + | FROM t + | WHERE c0 = t1.c0 AND $predicate + |) > 0 + """.stripMargin), + Row(1)) + } + } + } } // Some basic char/varchar tests which doesn't rely on table implementation. @@ -779,6 +806,18 @@ class FileSourceCharVarcharTestSuite extends CharVarcharTestSuite with SharedSpa } } + test("SPARK-35359: create table and insert data over length values") { + Seq("char", "varchar").foreach { typ => + withSQLConf((SQLConf.LEGACY_CHAR_VARCHAR_AS_STRING.key, "true")) { + withTable("t") { + sql(s"CREATE TABLE t (col $typ(2)) using $format") + sql("INSERT INTO t SELECT 'aaa'") + checkAnswer(sql("select * from t"), Row("aaa")) + } + } + } + } + test("alter table set location w/ fit length values") { Seq("char", "varchar").foreach { typ => withTempPath { dir => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index 01b1508d034c3..186091db19280 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -1660,6 +1660,61 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { StructType(Seq(StructField("a", IntegerType, nullable = true)))) } + test("SPARK-35213: chained withField operations should have correct schema for new columns") { + val df = spark.createDataFrame( + sparkContext.parallelize(Row(null) :: Nil), + StructType(Seq(StructField("data", NullType)))) + + checkAnswer( + df.withColumn("data", struct() + .withField("a", struct()) + .withField("b", struct()) + .withField("a.aa", lit("aa1")) + .withField("b.ba", lit("ba1")) + .withField("a.ab", lit("ab1"))), + Row(Row(Row("aa1", "ab1"), Row("ba1"))) :: Nil, + StructType(Seq( + StructField("data", StructType(Seq( + StructField("a", StructType(Seq( + StructField("aa", StringType, nullable = false), + StructField("ab", StringType, nullable = false) + )), nullable = false), + StructField("b", StructType(Seq( + StructField("ba", StringType, nullable = false) + )), nullable = false) + )), nullable = false) + )) + ) + } + + test("SPARK-35213: optimized withField operations should maintain correct nested struct " + + "ordering") { + val df = spark.createDataFrame( + sparkContext.parallelize(Row(null) :: Nil), + StructType(Seq(StructField("data", NullType)))) + + checkAnswer( + df.withColumn("data", struct() + .withField("a", struct().withField("aa", lit("aa1"))) + .withField("b", struct().withField("ba", lit("ba1"))) + ) + .withColumn("data", col("data").withField("b.bb", lit("bb1"))) + .withColumn("data", col("data").withField("a.ab", lit("ab1"))), + Row(Row(Row("aa1", "ab1"), Row("ba1", "bb1"))) :: Nil, + StructType(Seq( + StructField("data", StructType(Seq( + StructField("a", StructType(Seq( + StructField("aa", StringType, nullable = false), + StructField("ab", StringType, nullable = false) + )), nullable = false), + StructField("b", StructType(Seq( + StructField("ba", StringType, nullable = false), + StructField("bb", StringType, nullable = false) + )), nullable = false) + )), nullable = false) + )) + ) + } test("dropFields should throw an exception if called on a non-StructType column") { intercept[AnalysisException] { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index c6d134b16d06d..b5d0ebd307fca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -1091,6 +1091,25 @@ class DataFrameAggregateSuite extends QueryTest val df = spark.sql(query) checkAnswer(df, Row(0, "0", 0, 0) :: Row(-1, "1", 1, 1) :: Row(-2, "2", 2, 2) :: Nil) } + + test("SPARK-34713: group by CreateStruct with ExtractValue") { + val structDF = Seq(Tuple1(1 -> 1)).toDF("col") + checkAnswer(structDF.groupBy(struct($"col._1")).count().select("count"), Row(1)) + + val arrayOfStructDF = Seq(Tuple1(Seq(1 -> 1))).toDF("col") + checkAnswer(arrayOfStructDF.groupBy(struct($"col._1")).count().select("count"), Row(1)) + + val mapDF = Seq(Tuple1(Map("a" -> "a"))).toDF("col") + checkAnswer(mapDF.groupBy(struct($"col.a")).count().select("count"), Row(1)) + + val nonStringMapDF = Seq(Tuple1(Map(1 -> 1))).toDF("col") + // Spark implicit casts string literal "a" to int to match the key type. + checkAnswer(nonStringMapDF.groupBy(struct($"col.a")).count().select("count"), Row(1)) + + val arrayDF = Seq(Tuple1(Seq(1))).toDF("col") + val e = intercept[AnalysisException](arrayDF.groupBy(struct($"col.a")).count()) + assert(e.message.contains("requires integral type")) + } } case class B(c: Option[Double]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index aa1678e4f82fa..1c40215b8b243 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql +import java.io.File import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} @@ -24,7 +25,8 @@ import scala.util.Random import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions.{Alias, ArraysZip, AttributeReference, Expression, NamedExpression} import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.plans.logical.OneRowRelation import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{withDefaultTimeZone, UTC} @@ -552,6 +554,74 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Row(Seq(Row(0, 1, 2, 3, 4, 5)))) } + test("SPARK-35876: arrays_zip should retain field names") { + withTempDir { dir => + val df = spark.sparkContext.parallelize( + Seq((Seq(9001, 9002, 9003), Seq(4, 5, 6)))).toDF("val1", "val2") + val qualifiedDF = df.as("foo") + + // Fields are UnresolvedAttribute + val zippedDF1 = qualifiedDF.select(arrays_zip($"foo.val1", $"foo.val2") as "zipped") + val maybeAlias1 = zippedDF1.queryExecution.logical.expressions.head + assert(maybeAlias1.isInstanceOf[Alias]) + val maybeArraysZip1 = maybeAlias1.children.head + assert(maybeArraysZip1.isInstanceOf[ArraysZip]) + assert(maybeArraysZip1.children.forall(_.isInstanceOf[UnresolvedAttribute])) + val file1 = new File(dir, "arrays_zip1") + zippedDF1.write.parquet(file1.getAbsolutePath) + val restoredDF1 = spark.read.parquet(file1.getAbsolutePath) + val fieldNames1 = restoredDF1.schema.head.dataType.asInstanceOf[ArrayType] + .elementType.asInstanceOf[StructType].fieldNames + assert(fieldNames1.toSeq === Seq("val1", "val2")) + + // Fields are resolved NamedExpression + val zippedDF2 = df.select(arrays_zip(df("val1"), df("val2")) as "zipped") + val maybeAlias2 = zippedDF2.queryExecution.logical.expressions.head + assert(maybeAlias2.isInstanceOf[Alias]) + val maybeArraysZip2 = maybeAlias2.children.head + assert(maybeArraysZip2.isInstanceOf[ArraysZip]) + assert(maybeArraysZip2.children.forall( + e => e.isInstanceOf[AttributeReference] && e.resolved)) + val file2 = new File(dir, "arrays_zip2") + zippedDF2.write.parquet(file2.getAbsolutePath) + val restoredDF2 = spark.read.parquet(file2.getAbsolutePath) + val fieldNames2 = restoredDF2.schema.head.dataType.asInstanceOf[ArrayType] + .elementType.asInstanceOf[StructType].fieldNames + assert(fieldNames2.toSeq === Seq("val1", "val2")) + + // Fields are unresolved NamedExpression + val zippedDF3 = df.select(arrays_zip($"val1" as "val3", $"val2" as "val4") as "zipped") + val maybeAlias3 = zippedDF3.queryExecution.logical.expressions.head + assert(maybeAlias3.isInstanceOf[Alias]) + val maybeArraysZip3 = maybeAlias3.children.head + assert(maybeArraysZip3.isInstanceOf[ArraysZip]) + assert(maybeArraysZip3.children.forall(e => e.isInstanceOf[Alias] && !e.resolved)) + val file3 = new File(dir, "arrays_zip3") + zippedDF3.write.parquet(file3.getAbsolutePath) + val restoredDF3 = spark.read.parquet(file3.getAbsolutePath) + val fieldNames3 = restoredDF3.schema.head.dataType.asInstanceOf[ArrayType] + .elementType.asInstanceOf[StructType].fieldNames + assert(fieldNames3.toSeq === Seq("val3", "val4")) + + // Fields are neither UnresolvedAttribute nor NamedExpression + val zippedDF4 = df.select(arrays_zip(array_sort($"val1"), array_sort($"val2")) as "zipped") + val maybeAlias4 = zippedDF4.queryExecution.logical.expressions.head + assert(maybeAlias4.isInstanceOf[Alias]) + val maybeArraysZip4 = maybeAlias4.children.head + assert(maybeArraysZip4.isInstanceOf[ArraysZip]) + assert(maybeArraysZip4.children.forall { + case _: UnresolvedAttribute | _: NamedExpression => false + case _ => true + }) + val file4 = new File(dir, "arrays_zip4") + zippedDF4.write.parquet(file4.getAbsolutePath) + val restoredDF4 = spark.read.parquet(file4.getAbsolutePath) + val fieldNames4 = restoredDF4.schema.head.dataType.asInstanceOf[ArrayType] + .elementType.asInstanceOf[StructType].fieldNames + assert(fieldNames4.toSeq === Seq("0", "1")) + } + } + def testSizeOfMap(sizeOfNull: Any): Unit = { val df = Seq( (Map[Int, Int](1 -> 1, 2 -> 2), "x"), @@ -3629,6 +3699,29 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { df.select(map(map_entries($"m"), lit(1))), Row(Map(Seq(Row(1, "a")) -> 1))) } + + test("SPARK-34794: lambda variable name issues in nested functions") { + val df1 = Seq((Seq(1, 2), Seq("a", "b"))).toDF("numbers", "letters") + + checkAnswer(df1.select(flatten(transform($"numbers", (number: Column) => + transform($"letters", (letter: Column) => + struct(number, letter))))), + Seq(Row(Seq(Row(1, "a"), Row(1, "b"), Row(2, "a"), Row(2, "b")))) + ) + checkAnswer(df1.select(flatten(transform($"numbers", (number: Column, i: Column) => + transform($"letters", (letter: Column, j: Column) => + struct(number + j, concat(letter, i)))))), + Seq(Row(Seq(Row(1, "a0"), Row(2, "b0"), Row(2, "a1"), Row(3, "b1")))) + ) + + val df2 = Seq((Map("a" -> 1, "b" -> 2), Map("a" -> 2, "b" -> 3))).toDF("m1", "m2") + + checkAnswer(df2.select(map_zip_with($"m1", $"m2", (k1: Column, ov1: Column, ov2: Column) => + map_zip_with($"m1", $"m2", (k2: Column, iv1: Column, iv2: Column) => + ov1 + iv1 + ov2 + iv2))), + Seq(Row(Map("a" -> Map("a" -> 6, "b" -> 8), "b" -> Map("a" -> 8, "b" -> 10)))) + ) + } } object DataFrameFunctionsSuite { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala index 091877f7cac37..23c2349f89574 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala @@ -460,4 +460,29 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession { Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) :: Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) :: Nil) } + + test("SPARK-34417 - test fillMap() for column with a dot in the name") { + val na = "n/a" + checkAnswer( + Seq(("abc", 23L), ("def", 44L), (null, 0L)).toDF("ColWith.Dot", "Col") + .na.fill(Map("`ColWith.Dot`" -> na)), + Row("abc", 23) :: Row("def", 44L) :: Row(na, 0L) :: Nil) + } + + test("SPARK-34417 - test fillMap() for qualified-column with a dot in the name") { + val na = "n/a" + checkAnswer( + Seq(("abc", 23L), ("def", 44L), (null, 0L)).toDF("ColWith.Dot", "Col").as("testDF") + .na.fill(Map("testDF.`ColWith.Dot`" -> na)), + Row("abc", 23) :: Row("def", 44L) :: Row(na, 0L) :: Nil) + } + + test("SPARK-34417 - test fillMap() for column without a dot in the name" + + " and dataframe with another column having a dot in the name") { + val na = "n/a" + checkAnswer( + Seq(("abc", 23L), ("def", 44L), (null, 0L)).toDF("Col", "ColWith.Dot") + .na.fill(Map("Col" -> na)), + Row("abc", 23) :: Row("def", 44L) :: Row(na, 0L) :: Nil) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala index 76f07b5b0132d..9994981cb0e75 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala @@ -17,11 +17,15 @@ package org.apache.spark.sql +import org.apache.spark.api.python.PythonEvalType +import org.apache.spark.sql.catalyst.expressions.{Alias, Ascending, AttributeReference, PythonUDF, SortOrder} +import org.apache.spark.sql.catalyst.plans.logical.{Expand, Generate, ScriptInputOutputSchema, ScriptTransformation, Window => WindowPlan} import org.apache.spark.sql.expressions.Window -import org.apache.spark.sql.functions.{count, sum} +import org.apache.spark.sql.functions.{count, explode, sum} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData.TestData +import org.apache.spark.sql.types.{IntegerType, LongType, StructField, StructType} class DataFrameSelfJoinSuite extends QueryTest with SharedSparkSession { import testImplicits._ @@ -257,4 +261,205 @@ class DataFrameSelfJoinSuite extends QueryTest with SharedSparkSession { checkAnswer(df1.join(df2, df1("b") === 2), Row(1, 2, 1)) } } + + test("SPARK-35454: __dataset_id and __col_position should be correctly set") { + val ds = Seq[TestData]( + TestData(1, "sales"), + TestData(2, "personnel"), + TestData(3, "develop"), + TestData(4, "IT")).toDS() + var dsIdSetOpt = ds.logicalPlan.getTagValue(Dataset.DATASET_ID_TAG) + assert(dsIdSetOpt.get.size === 1) + var col1DsId = -1L + val col1 = ds.col("key") + col1.expr.foreach { + case a: AttributeReference => + col1DsId = a.metadata.getLong(Dataset.DATASET_ID_KEY) + assert(dsIdSetOpt.get.contains(col1DsId)) + assert(a.metadata.getLong(Dataset.COL_POS_KEY) === 0) + } + + val df = ds.toDF() + dsIdSetOpt = df.logicalPlan.getTagValue(Dataset.DATASET_ID_TAG) + assert(dsIdSetOpt.get.size === 2) + var col2DsId = -1L + val col2 = df.col("key") + col2.expr.foreach { + case a: AttributeReference => + col2DsId = a.metadata.getLong(Dataset.DATASET_ID_KEY) + assert(dsIdSetOpt.get.contains(a.metadata.getLong(Dataset.DATASET_ID_KEY))) + assert(a.metadata.getLong(Dataset.COL_POS_KEY) === 0) + } + assert(col1DsId !== col2DsId) + } + + test("SPARK-35454: fail ambiguous self join - toDF") { + val df1 = spark.range(3).toDF() + val df2 = df1.filter($"id" > 0).toDF() + + withSQLConf( + SQLConf.FAIL_AMBIGUOUS_SELF_JOIN_ENABLED.key -> "true", + SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + assertAmbiguousSelfJoin(df1.join(df2, df1.col("id") > df2.col("id"))) + } + } + + test("SPARK-35454: fail ambiguous self join - join four tables") { + val df1 = spark.range(3).select($"id".as("a"), $"id".as("b")) + val df2 = df1.filter($"a" > 0).select("b") + val df3 = df1.filter($"a" <= 2).select("b") + val df4 = df1.filter($"b" <= 2) + val df5 = spark.range(1) + + withSQLConf( + SQLConf.FAIL_AMBIGUOUS_SELF_JOIN_ENABLED.key -> "false", + SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + // `df2("b") < df4("b")` is always false + checkAnswer(df1.join(df2).join(df3).join(df4, df2("b") < df4("b")), Nil) + // `df2("b")` actually points to the column of `df1`. + checkAnswer( + df1.join(df2).join(df5).join(df4).select(df2("b")), + Seq(0, 0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 1, 2, 2, 2, 2, 2, 2).map(Row(_))) + // `df5("id")` is not ambiguous. + checkAnswer( + df1.join(df5).join(df3).select(df5("id")), + Seq(0, 0, 0, 0, 0, 0, 0, 0, 0).map(Row(_))) + + // Alias the dataframe and use qualified column names can fix ambiguous self-join. + val aliasedDf1 = df1.alias("w") + val aliasedDf2 = df2.as("x") + val aliasedDf3 = df3.as("y") + val aliasedDf4 = df3.as("z") + checkAnswer( + aliasedDf1.join(aliasedDf2).join(aliasedDf3).join(aliasedDf4, $"x.b" < $"y.b"), + Seq(Row(0, 0, 1, 2, 0), Row(0, 0, 1, 2, 1), Row(0, 0, 1, 2, 2), + Row(1, 1, 1, 2, 0), Row(1, 1, 1, 2, 1), Row(1, 1, 1, 2, 2), + Row(2, 2, 1, 2, 0), Row(2, 2, 1, 2, 1), Row(2, 2, 1, 2, 2))) + checkAnswer( + aliasedDf1.join(df5).join(aliasedDf3).select($"y.b"), + Seq(0, 0, 0, 1, 1, 1, 2, 2, 2).map(Row(_))) + } + + withSQLConf( + SQLConf.FAIL_AMBIGUOUS_SELF_JOIN_ENABLED.key -> "true", + SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + assertAmbiguousSelfJoin(df1.join(df2).join(df3).join(df4, df2("b") < df4("b"))) + assertAmbiguousSelfJoin(df1.join(df2).join(df5).join(df4).select(df2("b"))) + } + } + + test("SPARK-36874: DeduplicateRelations should copy dataset_id tag " + + "to avoid ambiguous self join") { + // Test for Project + val df1 = Seq((1, 2, "A1"), (2, 1, "A2")).toDF("key1", "key2", "value") + val df2 = df1.filter($"value" === "A2") + assertAmbiguousSelfJoin(df1.join(df2, df1("key1") === df2("key2"))) + assertAmbiguousSelfJoin(df2.join(df1, df1("key1") === df2("key2"))) + + // Test for SerializeFromObject + val df3 = spark.sparkContext.parallelize(1 to 10).map(x => (x, x)).toDF + val df4 = df3.filter($"_1" <=> 0) + assertAmbiguousSelfJoin(df3.join(df4, df3("_1") === df4("_2"))) + assertAmbiguousSelfJoin(df4.join(df3, df3("_1") === df4("_2"))) + + // Test For Aggregate + val df5 = df1.groupBy($"key1").agg(count($"value") as "count") + val df6 = df5.filter($"key1" > 0) + assertAmbiguousSelfJoin(df5.join(df6, df5("key1") === df6("count"))) + assertAmbiguousSelfJoin(df6.join(df5, df5("key1") === df6("count"))) + + // Test for MapInPandas + val mapInPandasUDF = PythonUDF("mapInPandasUDF", null, + StructType(Seq(StructField("x", LongType), StructField("y", LongType))), + Seq.empty, + PythonEvalType.SQL_MAP_PANDAS_ITER_UDF, + true) + val df7 = df1.mapInPandas(mapInPandasUDF) + val df8 = df7.filter($"x" > 0) + assertAmbiguousSelfJoin(df7.join(df8, df7("x") === df8("y"))) + assertAmbiguousSelfJoin(df8.join(df7, df7("x") === df8("y"))) + + // Test for FlatMapGroupsInPandas + val flatMapGroupsInPandasUDF = PythonUDF("flagMapGroupsInPandasUDF", null, + StructType(Seq(StructField("x", LongType), StructField("y", LongType))), + Seq.empty, + PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF, + true) + val df9 = df1.groupBy($"key1").flatMapGroupsInPandas(flatMapGroupsInPandasUDF) + val df10 = df9.filter($"x" > 0) + assertAmbiguousSelfJoin(df9.join(df10, df9("x") === df10("y"))) + assertAmbiguousSelfJoin(df10.join(df9, df9("x") === df10("y"))) + + // Test for FlatMapCoGroupsInPandas + val flatMapCoGroupsInPandasUDF = PythonUDF("flagMapCoGroupsInPandasUDF", null, + StructType(Seq(StructField("x", LongType), StructField("y", LongType))), + Seq.empty, + PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, + true) + val df11 = df1.groupBy($"key1").flatMapCoGroupsInPandas( + df1.groupBy($"key2"), flatMapCoGroupsInPandasUDF) + val df12 = df11.filter($"x" > 0) + assertAmbiguousSelfJoin(df11.join(df12, df11("x") === df12("y"))) + assertAmbiguousSelfJoin(df12.join(df11, df11("x") === df12("y"))) + + // Test for Generate + // Ensure that the root of the plan is Generate + val df13 = Seq((1, Seq(1, 2, 3))).toDF("a", "intList").select($"a", explode($"intList")) + .queryExecution.optimizedPlan.find(_.isInstanceOf[Generate]).get.toDF + val df14 = df13.filter($"a" > 0) + assertAmbiguousSelfJoin(df13.join(df14, df13("a") === df14("col"))) + assertAmbiguousSelfJoin(df14.join(df13, df13("a") === df14("col"))) + + // Test for Expand + // Ensure that the root of the plan is Expand + val df15 = + Expand( + Seq(Seq($"key1".expr, $"key2".expr)), + Seq( + AttributeReference("x", IntegerType)(), + AttributeReference("y", IntegerType)()), + df1.queryExecution.logical).toDF + val df16 = df15.filter($"x" > 0) + assertAmbiguousSelfJoin(df15.join(df16, df15("x") === df16("y"))) + assertAmbiguousSelfJoin(df16.join(df15, df15("x") === df16("y"))) + + // Test for Window + val dfWithTS = spark.sql("SELECT timestamp'2021-10-15 01:52:00' time, 1 a, 2 b") + // Ensure that the root of the plan is Window + val df17 = WindowPlan( + Seq(Alias(dfWithTS("time").expr, "ts")()), + Seq(dfWithTS("a").expr), + Seq(SortOrder(dfWithTS("a").expr, Ascending)), + dfWithTS.queryExecution.logical).toDF + val df18 = df17.filter($"a" > 0) + assertAmbiguousSelfJoin(df17.join(df18, df17("a") === df18("b"))) + assertAmbiguousSelfJoin(df18.join(df17, df17("a") === df18("b"))) + + // Test for ScriptTransformation + val ioSchema = + ScriptInputOutputSchema( + Seq(("TOK_TABLEROWFORMATFIELD", ","), + ("TOK_TABLEROWFORMATCOLLITEMS", "#"), + ("TOK_TABLEROWFORMATMAPKEYS", "@"), + ("TOK_TABLEROWFORMATNULL", "null"), + ("TOK_TABLEROWFORMATLINES", "\n")), + Seq(("TOK_TABLEROWFORMATFIELD", ","), + ("TOK_TABLEROWFORMATCOLLITEMS", "#"), + ("TOK_TABLEROWFORMATMAPKEYS", "@"), + ("TOK_TABLEROWFORMATNULL", "null"), + ("TOK_TABLEROWFORMATLINES", "\n")), None, None, + List.empty, List.empty, None, None, false) + // Ensure that the root of the plan is ScriptTransformation + val df19 = ScriptTransformation( + Seq($"key1".expr, $"key2".expr), + "cat", + Seq( + AttributeReference("x", IntegerType)(), + AttributeReference("y", IntegerType)()), + df1.queryExecution.logical, + ioSchema).toDF + val df20 = df19.filter($"x" > 0) + assertAmbiguousSelfJoin(df19.join(df20, df19("x") === df20("y"))) + assertAmbiguousSelfJoin(df20.join(df19, df19("x") === df20("y"))) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 8d95f83e49f3a..de252900f69ce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -41,6 +41,7 @@ import org.apache.spark.sql.execution.{FilterExec, QueryExecution, WholeStageCod import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec} +import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSparkSession} @@ -2584,6 +2585,95 @@ class DataFrameSuite extends QueryTest val col2 = df.colRegex("test\n_table.`tes.*\n.*mn`") checkAnswer(df.select(col2), Row(1) :: Row(2) :: Row(3) :: Nil) } + + test("SPARK-34776: Nested column pruning should not prune Window produced attributes") { + val df = Seq( + ("t1", "123", "bob"), + ("t1", "456", "bob"), + ("t2", "123", "sam") + ).toDF("type", "value", "name") + + val test = df.select( + $"*", + struct(count($"*").over(Window.partitionBy($"type", $"value", $"name")) + .as("count"), $"name").as("name_count") + ).select( + $"*", + max($"name_count").over(Window.partitionBy($"type", $"value")).as("best_name") + ) + checkAnswer(test.select($"best_name.name"), Row("bob") :: Row("bob") :: Row("sam") :: Nil) + } + + test("SPARK-34829: Multiple applications of typed ScalaUDFs in higher order functions work") { + val reverse = udf((s: String) => s.reverse) + val reverse2 = udf((b: Bar2) => Bar2(b.s.reverse)) + + val df = Seq(Array("abc", "def")).toDF("array") + val test = df.select(transform(col("array"), s => reverse(s))) + checkAnswer(test, Row(Array("cba", "fed")) :: Nil) + + val df2 = Seq(Array(Bar2("abc"), Bar2("def"))).toDF("array") + val test2 = df2.select(transform(col("array"), b => reverse2(b))) + checkAnswer(test2, Row(Array(Row("cba"), Row("fed"))) :: Nil) + + val df3 = Seq(Map("abc" -> 1, "def" -> 2)).toDF("map") + val test3 = df3.select(transform_keys(col("map"), (s, _) => reverse(s))) + checkAnswer(test3, Row(Map("cba" -> 1, "fed" -> 2)) :: Nil) + + val df4 = Seq(Map(Bar2("abc") -> 1, Bar2("def") -> 2)).toDF("map") + val test4 = df4.select(transform_keys(col("map"), (b, _) => reverse2(b))) + checkAnswer(test4, Row(Map(Row("cba") -> 1, Row("fed") -> 2)) :: Nil) + + val df5 = Seq(Map(1 -> "abc", 2 -> "def")).toDF("map") + val test5 = df5.select(transform_values(col("map"), (_, s) => reverse(s))) + checkAnswer(test5, Row(Map(1 -> "cba", 2 -> "fed")) :: Nil) + + val df6 = Seq(Map(1 -> Bar2("abc"), 2 -> Bar2("def"))).toDF("map") + val test6 = df6.select(transform_values(col("map"), (_, b) => reverse2(b))) + checkAnswer(test6, Row(Map(1 -> Row("cba"), 2 -> Row("fed"))) :: Nil) + + val reverseThenConcat = udf((s1: String, s2: String) => s1.reverse ++ s2.reverse) + val reverseThenConcat2 = udf((b1: Bar2, b2: Bar2) => Bar2(b1.s.reverse ++ b2.s.reverse)) + + val df7 = Seq((Map(1 -> "abc", 2 -> "def"), Map(1 -> "ghi", 2 -> "jkl"))).toDF("map1", "map2") + val test7 = + df7.select(map_zip_with(col("map1"), col("map2"), (_, s1, s2) => reverseThenConcat(s1, s2))) + checkAnswer(test7, Row(Map(1 -> "cbaihg", 2 -> "fedlkj")) :: Nil) + + val df8 = Seq((Map(1 -> Bar2("abc"), 2 -> Bar2("def")), + Map(1 -> Bar2("ghi"), 2 -> Bar2("jkl")))).toDF("map1", "map2") + val test8 = + df8.select(map_zip_with(col("map1"), col("map2"), (_, b1, b2) => reverseThenConcat2(b1, b2))) + checkAnswer(test8, Row(Map(1 -> Row("cbaihg"), 2 -> Row("fedlkj"))) :: Nil) + + val df9 = Seq((Array("abc", "def"), Array("ghi", "jkl"))).toDF("array1", "array2") + val test9 = + df9.select(zip_with(col("array1"), col("array2"), (s1, s2) => reverseThenConcat(s1, s2))) + checkAnswer(test9, Row(Array("cbaihg", "fedlkj")) :: Nil) + + val df10 = Seq((Array(Bar2("abc"), Bar2("def")), Array(Bar2("ghi"), Bar2("jkl")))) + .toDF("array1", "array2") + val test10 = + df10.select(zip_with(col("array1"), col("array2"), (b1, b2) => reverseThenConcat2(b1, b2))) + checkAnswer(test10, Row(Array(Row("cbaihg"), Row("fedlkj"))) :: Nil) + } + + test("SPARK-35886: PromotePrecision should be subexpr replaced") { + withTable("tbl") { + sql( + """ + |CREATE TABLE tbl ( + | c1 DECIMAL(18,6), + | c2 DECIMAL(18,6), + | c3 DECIMAL(18,6)) + |USING parquet; + |""".stripMargin) + sql("INSERT INTO tbl SELECT 1, 1, 1") + checkAnswer(sql("SELECT sum(c1 * c3) + sum(c2 * c3) FROM tbl"), Row(2.00000000000) :: Nil) + } + } } case class GroupByKey(a: Int, b: Int) + +case class Bar2(s: String) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 69fbb9b137575..074a517151b42 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -1980,6 +1980,41 @@ class DatasetSuite extends QueryTest checkAnswer(withUDF, Row(Row(1), null, null) :: Row(Row(1), null, null) :: Nil) } + + test("SPARK-35652: joinWith on two table generated from same one performing a cartesian join," + + " which should be inner join") { + val df = Seq(1, 2, 3).toDS() + + val joined = df.joinWith(df, df("value") === df("value"), "inner") + + val expectedSchema = StructType(Seq( + StructField("_1", IntegerType, nullable = false), + StructField("_2", IntegerType, nullable = false) + )) + + assert(joined.schema === expectedSchema) + + checkDataset( + joined, + (1, 1), (2, 2), (3, 3)) + } + + test("SPARK-36210: withColumns preserve insertion ordering") { + val df = Seq(1, 2, 3).toDS() + + val colNames = (1 to 10).map(i => s"value${i}") + val cols = (1 to 10).map(i => col("value") + i) + + val inserted = df.withColumns(colNames, cols) + + assert(inserted.columns === "value" +: colNames) + + checkDataset( + inserted.as[(Int, Int, Int, Int, Int, Int, Int, Int, Int, Int, Int)], + (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11), + (2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12), + (3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13)) + } } case class Bar(a: Int) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala index 55437aaa47298..db7b0dd4b67e8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -1361,6 +1361,26 @@ abstract class DynamicPartitionPruningSuiteBase checkAnswer(df, Nil) } } + + test("SPARK-34436: DPP support LIKE ANY/ALL expression") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { + val df = sql( + """ + |SELECT date_id, product_id FROM fact_sk f + |JOIN dim_store s + |ON f.store_id = s.store_id WHERE s.country LIKE ANY ('%D%E%', '%A%B%') + """.stripMargin) + + checkPartitionPruningPredicate(df, false, true) + + checkAnswer(df, + Row(1030, 2) :: + Row(1040, 2) :: + Row(1050, 2) :: + Row(1060, 2) :: Nil + ) + } + } } class DynamicPartitionPruningSuiteAEOff extends DynamicPartitionPruningSuiteBase { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileScanSuite.scala new file mode 100644 index 0000000000000..4e7fe8455ff93 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileScanSuite.scala @@ -0,0 +1,374 @@ +/* + * 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.collection.mutable + +import com.google.common.collect.ImmutableMap +import org.apache.hadoop.fs.{FileStatus, Path} + +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.{And, Expression, IsNull, LessThan} +import org.apache.spark.sql.execution.datasources.{PartitioningAwareFileIndex, PartitionSpec} +import org.apache.spark.sql.execution.datasources.v2.FileScan +import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan +import org.apache.spark.sql.execution.datasources.v2.json.JsonScan +import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan +import org.apache.spark.sql.execution.datasources.v2.text.TextScan +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +trait FileScanSuiteBase extends SharedSparkSession { + private def newPartitioningAwareFileIndex() = { + new PartitioningAwareFileIndex(spark, Map.empty, None) { + override def partitionSpec(): PartitionSpec = { + PartitionSpec.emptySpec + } + + override protected def leafFiles: mutable.LinkedHashMap[Path, FileStatus] = { + mutable.LinkedHashMap.empty + } + + override protected def leafDirToChildrenFiles: Map[Path, Array[FileStatus]] = { + Map.empty + } + + override def rootPaths: Seq[Path] = { + Seq.empty + } + + override def refresh(): Unit = {} + } + } + + type ScanBuilder = ( + SparkSession, + PartitioningAwareFileIndex, + StructType, + StructType, + StructType, + Array[Filter], + CaseInsensitiveStringMap, + Seq[Expression], + Seq[Expression]) => FileScan + + def run(scanBuilders: Seq[(String, ScanBuilder, Seq[String])]): Unit = { + val dataSchema = StructType.fromDDL("data INT, partition INT, other INT") + val dataSchemaNotEqual = StructType.fromDDL("data INT, partition INT, other INT, new INT") + val readDataSchema = StructType.fromDDL("data INT") + val readDataSchemaNotEqual = StructType.fromDDL("data INT, other INT") + val readPartitionSchema = StructType.fromDDL("partition INT") + val readPartitionSchemaNotEqual = StructType.fromDDL("partition INT, other INT") + val pushedFilters = + Array[Filter](sources.And(sources.IsNull("data"), sources.LessThan("data", 0))) + val pushedFiltersNotEqual = + Array[Filter](sources.And(sources.IsNull("data"), sources.LessThan("data", 1))) + val optionsMap = ImmutableMap.of("key", "value") + val options = new CaseInsensitiveStringMap(ImmutableMap.copyOf(optionsMap)) + val optionsNotEqual = + new CaseInsensitiveStringMap(ImmutableMap.copyOf(ImmutableMap.of("key2", "value2"))) + val partitionFilters = Seq(And(IsNull('data.int), LessThan('data.int, 0))) + val partitionFiltersNotEqual = Seq(And(IsNull('data.int), LessThan('data.int, 1))) + val dataFilters = Seq(And(IsNull('data.int), LessThan('data.int, 0))) + val dataFiltersNotEqual = Seq(And(IsNull('data.int), LessThan('data.int, 1))) + + scanBuilders.foreach { case (name, scanBuilder, exclusions) => + test(s"SPARK-33482: Test $name equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val scanEquals = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema.copy(), + readDataSchema.copy(), + readPartitionSchema.copy(), + pushedFilters.clone(), + new CaseInsensitiveStringMap(ImmutableMap.copyOf(optionsMap)), + Seq(partitionFilters: _*), + Seq(dataFilters: _*)) + + assert(scan === scanEquals) + } + + test(s"SPARK-33482: Test $name fileIndex not equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val partitioningAwareFileIndexNotEqual = newPartitioningAwareFileIndex() + + val scanNotEqual = scanBuilder( + spark, + partitioningAwareFileIndexNotEqual, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + assert(scan !== scanNotEqual) + } + + if (!exclusions.contains("dataSchema")) { + test(s"SPARK-33482: Test $name dataSchema not equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val scanNotEqual = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchemaNotEqual, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + assert(scan !== scanNotEqual) + } + } + + test(s"SPARK-33482: Test $name readDataSchema not equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val scanNotEqual = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchemaNotEqual, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + assert(scan !== scanNotEqual) + } + + test(s"SPARK-33482: Test $name readPartitionSchema not equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val scanNotEqual = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchemaNotEqual, + pushedFilters, + options, + partitionFilters, + dataFilters) + + assert(scan !== scanNotEqual) + } + + if (!exclusions.contains("pushedFilters")) { + test(s"SPARK-33482: Test $name pushedFilters not equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val scanNotEqual = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFiltersNotEqual, + options, + partitionFilters, + dataFilters) + + assert(scan !== scanNotEqual) + } + } + + test(s"SPARK-33482: Test $name options not equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val scanNotEqual = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + optionsNotEqual, + partitionFilters, + dataFilters) + + assert(scan !== scanNotEqual) + } + + test(s"SPARK-33482: Test $name partitionFilters not equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val scanNotEqual = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFiltersNotEqual, + dataFilters) + assert(scan !== scanNotEqual) + } + + test(s"SPARK-33482: Test $name dataFilters not equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val scanNotEqual = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFiltersNotEqual) + assert(scan !== scanNotEqual) + } + } + } +} + +class FileScanSuite extends FileScanSuiteBase { + val scanBuilders = Seq[(String, ScanBuilder, Seq[String])]( + ("ParquetScan", + (s, fi, ds, rds, rps, f, o, pf, df) => + ParquetScan(s, s.sessionState.newHadoopConf(), fi, ds, rds, rps, f, o, pf, df), + Seq.empty), + ("OrcScan", + (s, fi, ds, rds, rps, f, o, pf, df) => + OrcScan(s, s.sessionState.newHadoopConf(), fi, ds, rds, rps, o, f, pf, df), + Seq.empty), + ("CSVScan", + (s, fi, ds, rds, rps, f, o, pf, df) => CSVScan(s, fi, ds, rds, rps, o, f, pf, df), + Seq.empty), + ("JsonScan", + (s, fi, ds, rds, rps, f, o, pf, df) => JsonScan(s, fi, ds, rds, rps, o, f, pf, df), + Seq.empty), + ("TextScan", + (s, fi, _, rds, rps, _, o, pf, df) => TextScan(s, fi, rds, rps, o, pf, df), + Seq("dataSchema", "pushedFilters"))) + + run(scanBuilders) +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala index fd9655fdbef42..9e8f3fbeaef44 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala @@ -105,4 +105,10 @@ class RowSuite extends SparkFunSuite with SharedSparkSession { val empty = Row() assert(empty.toString == "[]") } + + test("SPARK-37654: row contains a null at the requested index should return null") { + assert(Row(Seq("value")).getSeq(0) === List("value")) + assert(Row(Seq()).getSeq(0) === List()) + assert(Row(null).getSeq(0) === null) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala index c7446c7a9f443..a0943437bc8b9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} /** - * The base trait for DML - insert syntax + * The base trait for SQL INSERT. */ trait SQLInsertTestSuite extends QueryTest with SQLTestUtils { @@ -208,6 +208,55 @@ trait SQLInsertTestSuite extends QueryTest with SQLTestUtils { checkAnswer(spark.table("t"), Row("1", null)) } } + + test("SPARK-34556: " + + "checking duplicate static partition columns should respect case sensitive conf") { + withTable("t") { + sql(s"CREATE TABLE t(i STRING, c string) USING PARQUET PARTITIONED BY (c)") + val e = intercept[AnalysisException] { + sql("INSERT OVERWRITE t PARTITION (c='2', C='3') VALUES (1)") + } + assert(e.getMessage.contains("Found duplicate keys 'c'")) + } + // The following code is skipped for Hive because columns stored in Hive Metastore is always + // case insensitive and we cannot create such table in Hive Metastore. + if (!format.startsWith("hive")) { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + withTable("t") { + sql(s"CREATE TABLE t(i int, c string, C string) USING PARQUET PARTITIONED BY (c, C)") + sql("INSERT OVERWRITE t PARTITION (c='2', C='3') VALUES (1)") + checkAnswer(spark.table("t"), Row(1, "2", "3")) + } + } + } + } + + test("SPARK-30844: static partition should also follow StoreAssignmentPolicy") { + val testingPolicies = if (format == "foo") { + // DS v2 doesn't support the legacy policy + Seq(SQLConf.StoreAssignmentPolicy.ANSI, SQLConf.StoreAssignmentPolicy.STRICT) + } else { + SQLConf.StoreAssignmentPolicy.values + } + testingPolicies.foreach { policy => + withSQLConf( + SQLConf.STORE_ASSIGNMENT_POLICY.key -> policy.toString) { + withTable("t") { + sql("create table t(a int, b string) using parquet partitioned by (a)") + policy match { + case SQLConf.StoreAssignmentPolicy.ANSI | SQLConf.StoreAssignmentPolicy.STRICT => + val errorMsg = intercept[NumberFormatException] { + sql("insert into t partition(a='ansi') values('ansi')") + }.getMessage + assert(errorMsg.contains("invalid input syntax for type numeric: ansi")) + case SQLConf.StoreAssignmentPolicy.LEGACY => + sql("insert into t partition(a='ansi') values('ansi')") + checkAnswer(sql("select * from t"), Row("ansi", null) :: Nil) + } + } + } + } + } } class FileSourceSQLInsertTestSuite extends SQLInsertTestSuite with SharedSparkSession { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index e42fb963d5f8c..58cf9f4ffd5b0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -32,11 +32,13 @@ import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec -import org.apache.spark.sql.execution.command.FunctionsCommand +import org.apache.spark.sql.execution.command.{DataWritingCommandExec, FunctionsCommand} +import org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand import org.apache.spark.sql.execution.datasources.SchemaColumnConvertNotSupportedException import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan +import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, CartesianProductExec, SortMergeJoinExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -3458,6 +3460,19 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark } } + test("SPARK-36339: References to grouping attributes should be replaced") { + withTempView("t") { + Seq("a", "a", "b").toDF("x").createOrReplaceTempView("t") + checkAnswer( + sql( + """ + |select count(x) c, x from t + |group by x grouping sets(x) + """.stripMargin), + Seq(Row(2, "a"), Row(1, "b"))) + } + } + test("SPARK-31166: UNION map and other maps should not fail") { checkAnswer( sql("(SELECT map()) UNION ALL (SELECT map(1, 2))"), @@ -3596,6 +3611,9 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark Row("A", null, 5) :: Row("A", "{\"i\": 1}", 3) :: Row("A", "{\"i\": 2}", 2) :: Row("B", null, 1) :: Row("B", "{\"i\": 1}", 1) :: Row("C", null, 3) :: Row("C", "{\"i\": 1}", 3) :: Nil) + + assert(spark.table("t").groupBy($"c.json_string").count().schema.fieldNames === + Seq("json_string", "count")) } } @@ -3926,6 +3944,80 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark } } } + + test("SPARK-34796: Avoid code-gen compilation error for LIMIT query") { + withTable("left_table", "empty_right_table", "output_table") { + spark.range(5).toDF("k").write.saveAsTable("left_table") + spark.range(0).toDF("k").write.saveAsTable("empty_right_table") + + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + spark.sql("CREATE TABLE output_table (k INT) USING parquet") + spark.sql( + """ + |INSERT INTO TABLE output_table + |SELECT t1.k FROM left_table t1 + |JOIN empty_right_table t2 + |ON t1.k = t2.k + |LIMIT 3 + """.stripMargin) + } + } + } + + test("SPARK-33482: Fix FileScan canonicalization") { + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "") { + withTempPath { path => + spark.range(5).toDF().write.mode("overwrite").parquet(path.toString) + withTempView("t") { + spark.read.parquet(path.toString).createOrReplaceTempView("t") + val df = sql( + """ + |SELECT * + |FROM t AS t1 + |JOIN t AS t2 ON t2.id = t1.id + |JOIN t AS t3 ON t3.id = t2.id + |""".stripMargin) + df.collect() + val reusedExchanges = collect(df.queryExecution.executedPlan) { + case r: ReusedExchangeExec => r + } + assert(reusedExchanges.size == 1) + } + } + } + } + + test("SPARK-36093: RemoveRedundantAliases should not change expression's name") { + withTable("t1", "t2") { + withView("t1_v") { + sql("CREATE TABLE t1(cal_dt DATE) USING PARQUET") + sql( + """ + |INSERT INTO t1 VALUES + |(date'2021-06-27'), + |(date'2021-06-28'), + |(date'2021-06-29'), + |(date'2021-06-30')""".stripMargin) + sql("CREATE VIEW t1_v AS SELECT * FROM t1") + sql( + """ + |CREATE TABLE t2(FLAG INT, CAL_DT DATE) + |USING PARQUET + |PARTITIONED BY (CAL_DT)""".stripMargin) + val insert = sql( + """ + |INSERT INTO t2 SELECT 2 AS FLAG,CAL_DT FROM t1_v + |WHERE CAL_DT BETWEEN '2021-06-29' AND '2021-06-30'""".stripMargin) + insert.queryExecution.executedPlan.collectFirst { + case DataWritingCommandExec(i: InsertIntoHadoopFsRelationCommand, _) => i + }.get.partitionColumns.map(_.name).foreach(name => assert(name == "CAL_DT")) + checkAnswer(sql("SELECT FLAG, CAST(CAL_DT as STRING) FROM t2 "), + Row(2, "2021-06-29") :: Row(2, "2021-06-30") :: Nil) + checkAnswer(sql("SHOW PARTITIONS t2"), + Row("CAL_DT=2021-06-29") :: Row("CAL_DT=2021-06-30") :: Nil) + } + } + } } case class Foo(bar: Option[String]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala index 35d2513835611..03514d850fc67 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala @@ -355,6 +355,32 @@ class SparkSessionExtensionSuite extends SparkFunSuite { stop(session) } } + + test("SPARK-35673: user-defined hint and unrecognized hint in subquery") { + withSession(Seq(_.injectPostHocResolutionRule(MyHintRule))) { session => + // unrecognized hint + QueryTest.checkAnswer( + session.sql( + """ + |SELECT * + |FROM ( + | SELECT /*+ some_random_hint_that_does_not_exist */ 42 + |) + |""".stripMargin), + Row(42) :: Nil) + + // user-defined hint + QueryTest.checkAnswer( + session.sql( + """ + |SELECT * + |FROM ( + | SELECT /*+ CONVERT_TO_EMPTY */ 42 + |) + |""".stripMargin), + Nil) + } + } } case class MyRule(spark: SparkSession) extends Rule[LogicalPlan] { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 73b23496de515..fafe1bb39336f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -542,7 +542,7 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark sql("select a, (select sum(b) from l l2 where l2.a < l1.a) sum_b from l l1") } assert(msg1.getMessage.contains( - "Correlated column is not allowed in a non-equality predicate:")) + "Correlated column is not allowed in predicate (l2.`a` < outer(l1.`a`))")) } test("disjunctive correlated scalar subquery") { @@ -1753,4 +1753,13 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark } } } + + test("SPARK-35080: correlated equality predicates contain only outer references") { + withTempView("t") { + Seq((0, 1), (1, 1)).toDF("c1", "c2").createOrReplaceTempView("t") + checkAnswer( + sql("select c1, c2, (select count(*) from l where c1 = c2) from t"), + Row(0, 1, 0) :: Row(1, 1, 8) :: Nil) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 0e12eba84eb03..795776b572be5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -2649,6 +2649,37 @@ class DataSourceV2SQLSuite } } + test("SPARK-34547: metadata columns are resolved last") { + val t1 = s"${catalogAndNamespace}tableOne" + val t2 = "t2" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format " + + "PARTITIONED BY (bucket(4, id), id)") + sql(s"INSERT INTO $t1 VALUES (1, 'a'), (2, 'b'), (3, 'c')") + withTempView(t2) { + sql(s"CREATE TEMPORARY VIEW $t2 AS SELECT * FROM " + + s"VALUES (1, -1), (2, -2), (3, -3) AS $t2(id, index)") + + val sqlQuery = spark.sql(s"SELECT $t1.id, $t2.id, data, index, $t1.index, $t2.index FROM " + + s"$t1 JOIN $t2 WHERE $t1.id = $t2.id") + val t1Table = spark.table(t1) + val t2Table = spark.table(t2) + val dfQuery = t1Table.join(t2Table, t1Table.col("id") === t2Table.col("id")) + .select(s"$t1.id", s"$t2.id", "data", "index", s"$t1.index", s"$t2.index") + + Seq(sqlQuery, dfQuery).foreach { query => + checkAnswer(query, + Seq( + Row(1, 1, "a", -1, 0, -1), + Row(2, 2, "b", -2, 0, -2), + Row(3, 3, "c", -3, 0, -3) + ) + ) + } + } + } + } + test("SPARK-33505: insert into partitioned table") { val t = "testpart.ns1.ns2.tbl" withTable(t) { @@ -2668,6 +2699,27 @@ class DataSourceV2SQLSuite } } + test("SPARK-34555: Resolve DataFrame metadata column") { + val tbl = s"${catalogAndNamespace}table" + withTable(tbl) { + sql(s"CREATE TABLE $tbl (id bigint, data string) USING $v2Format " + + "PARTITIONED BY (bucket(4, id), id)") + sql(s"INSERT INTO $tbl VALUES (1, 'a'), (2, 'b'), (3, 'c')") + val table = spark.table(tbl) + val dfQuery = table.select( + table.col("id"), + table.col("data"), + table.col("index"), + table.col("_partition") + ) + + checkAnswer( + dfQuery, + Seq(Row(1, "a", 0, "3/1"), Row(2, "b", 0, "0/2"), Row(3, "c", 0, "1/3")) + ) + } + } + private def testNotSupportedV2Command( sqlCommand: String, sqlParams: String, @@ -2699,6 +2751,109 @@ class DataSourceV2SQLSuite }.getMessage assert(errMsg.contains(expectedError)) } + + test("SPARK-34923: do not propagate metadata columns through Project") { + val t1 = s"${catalogAndNamespace}table" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format " + + "PARTITIONED BY (bucket(4, id), id)") + sql(s"INSERT INTO $t1 VALUES (1, 'a'), (2, 'b'), (3, 'c')") + + assertThrows[AnalysisException] { + sql(s"SELECT index, _partition from (SELECT id, data FROM $t1)") + } + assertThrows[AnalysisException] { + spark.table(t1).select("id", "data").select("index", "_partition") + } + } + } + + test("SPARK-34923: do not propagate metadata columns through View") { + val t1 = s"${catalogAndNamespace}table" + val view = "view" + + withTable(t1) { + withTempView(view) { + sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format " + + "PARTITIONED BY (bucket(4, id), id)") + sql(s"INSERT INTO $t1 VALUES (1, 'a'), (2, 'b'), (3, 'c')") + sql(s"CACHE TABLE $view AS SELECT * FROM $t1") + assertThrows[AnalysisException] { + sql(s"SELECT index, _partition FROM $view") + } + } + } + } + + test("SPARK-34923: propagate metadata columns through Filter") { + val t1 = s"${catalogAndNamespace}table" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format " + + "PARTITIONED BY (bucket(4, id), id)") + sql(s"INSERT INTO $t1 VALUES (1, 'a'), (2, 'b'), (3, 'c')") + + val sqlQuery = spark.sql(s"SELECT id, data, index, _partition FROM $t1 WHERE id > 1") + val dfQuery = spark.table(t1).where("id > 1").select("id", "data", "index", "_partition") + + Seq(sqlQuery, dfQuery).foreach { query => + checkAnswer(query, Seq(Row(2, "b", 0, "0/2"), Row(3, "c", 0, "1/3"))) + } + } + } + + test("SPARK-34923: propagate metadata columns through Sort") { + val t1 = s"${catalogAndNamespace}table" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format " + + "PARTITIONED BY (bucket(4, id), id)") + sql(s"INSERT INTO $t1 VALUES (1, 'a'), (2, 'b'), (3, 'c')") + + val sqlQuery = spark.sql(s"SELECT id, data, index, _partition FROM $t1 ORDER BY id") + val dfQuery = spark.table(t1).orderBy("id").select("id", "data", "index", "_partition") + + Seq(sqlQuery, dfQuery).foreach { query => + checkAnswer(query, Seq(Row(1, "a", 0, "3/1"), Row(2, "b", 0, "0/2"), Row(3, "c", 0, "1/3"))) + } + } + } + + test("SPARK-34923: propagate metadata columns through RepartitionBy") { + val t1 = s"${catalogAndNamespace}table" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format " + + "PARTITIONED BY (bucket(4, id), id)") + sql(s"INSERT INTO $t1 VALUES (1, 'a'), (2, 'b'), (3, 'c')") + + val sqlQuery = spark.sql( + s"SELECT /*+ REPARTITION_BY_RANGE(3, id) */ id, data, index, _partition FROM $t1") + val tbl = spark.table(t1) + val dfQuery = tbl.repartitionByRange(3, tbl.col("id")) + .select("id", "data", "index", "_partition") + + Seq(sqlQuery, dfQuery).foreach { query => + checkAnswer(query, Seq(Row(1, "a", 0, "3/1"), Row(2, "b", 0, "0/2"), Row(3, "c", 0, "1/3"))) + } + } + } + + test("SPARK-34923: propagate metadata columns through SubqueryAlias") { + val t1 = s"${catalogAndNamespace}table" + val sbq = "sbq" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format " + + "PARTITIONED BY (bucket(4, id), id)") + sql(s"INSERT INTO $t1 VALUES (1, 'a'), (2, 'b'), (3, 'c')") + + val sqlQuery = spark.sql( + s"SELECT $sbq.id, $sbq.data, $sbq.index, $sbq._partition FROM $t1 as $sbq") + val dfQuery = spark.table(t1).as(sbq).select( + s"$sbq.id", s"$sbq.data", s"$sbq.index", s"$sbq._partition") + + Seq(sqlQuery, dfQuery).foreach { query => + checkAnswer(query, Seq(Row(1, "a", 0, "3/1"), Row(2, "b", 0, "0/2"), Row(3, "c", 0, "1/3"))) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala index 2cc7a1f994645..ad730376b2e3a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala @@ -477,5 +477,15 @@ trait InsertIntoSQLOnlyTests verifyTable(t1, spark.table(view)) } } + + test("SPARK-34599: InsertInto: overwrite - dot in the partition column name - static mode") { + import testImplicits._ + val t1 = "tbl" + withTable(t1) { + sql(s"CREATE TABLE $t1 (`a.b` string, `c.d` string) USING $v2Format PARTITIONED BY (`a.b`)") + sql(s"INSERT OVERWRITE $t1 PARTITION (`a.b` = 'a') (`c.d`) VALUES('b')") + verifyTable(t1, Seq("a" -> "b").toDF("id", "data")) + } + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/CoalesceShufflePartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/CoalesceShufflePartitionsSuite.scala index 22c5b651f7e12..4cb8cf1cab9f5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/CoalesceShufflePartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/CoalesceShufflePartitionsSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution import org.scalatest.BeforeAndAfterAll import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.IO_ENCRYPTION_ENABLED import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql._ import org.apache.spark.sql.execution.adaptive._ @@ -57,15 +58,18 @@ class CoalesceShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterAl def withSparkSession( f: SparkSession => Unit, targetPostShuffleInputSize: Int, - minNumPostShufflePartitions: Option[Int]): Unit = { + minNumPostShufflePartitions: Option[Int], + enableIOEncryption: Boolean = false): Unit = { val sparkConf = new SparkConf(false) .setMaster("local[*]") .setAppName("test") .set(UI_ENABLED, false) + .set(IO_ENCRYPTION_ENABLED, enableIOEncryption) .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") .set(SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key, "5") .set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true") + .set(SQLConf.FETCH_SHUFFLE_BLOCKS_IN_BATCH.key, "true") .set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1") .set( SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key, @@ -408,6 +412,25 @@ class CoalesceShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterAl } withSparkSession(test, 100, None) } + + test("SPARK-34790: enable IO encryption in AQE partition coalescing") { + val test: SparkSession => Unit = { spark: SparkSession => + val ds = spark.range(0, 100, 1, numInputPartitions) + val resultDf = ds.repartition(ds.col("id")) + resultDf.collect() + + val finalPlan = resultDf.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlanExec].executedPlan + assert( + finalPlan.collect { + case r @ CoalescedShuffleReader() => r + }.isDefinedAt(0)) + } + Seq(true, false).foreach { enableIOEncryption => + // Before SPARK-34790, it will throw an exception when io encryption enabled. + withSparkSession(test, Int.MaxValue, None, enableIOEncryption) + } + } } object CoalescedShuffleReader { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala index 040c5189abcb6..7da66e4ed3cb7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.{DataFrame, QueryTest, Row} +import org.apache.spark.sql.connector.SimpleWritableDataSource import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -28,6 +29,7 @@ abstract class RemoveRedundantProjectsSuiteBase extends QueryTest with SharedSparkSession with AdaptiveSparkPlanHelper { + import testImplicits._ private def assertProjectExecCount(df: DataFrame, expected: Int): Unit = { withClue(df.queryExecution) { @@ -211,7 +213,40 @@ abstract class RemoveRedundantProjectsSuiteBase |ORDER BY t1.key, t2.key, s1, s2 |LIMIT 10 |""".stripMargin - assertProjectExec(query, 0, 3) + // The Project above the Expand is not removed due to SPARK-36020. + assertProjectExec(query, 1, 3) + } + + test("SPARK-36020: Project should not be removed when child's logical link is different") { + val query = + """ + |WITH t AS ( + | SELECT key, a, b, c, explode(d) AS d FROM testView + |) + |SELECT t1.key, t1.d, t2.key + |FROM (SELECT d, key FROM t) t1 + |JOIN testView t2 ON t1.key = t2.key + |""".stripMargin + // The ProjectExec above the GenerateExec should not be removed because + // they have different logical links. + assertProjectExec(query, enabled = 2, disabled = 3) + } + + Seq("true", "false").foreach { codegenEnabled => + test("SPARK-35287: project generating unsafe row for DataSourceV2ScanRelation " + + s"should not be removed (codegen=$codegenEnabled)") { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> codegenEnabled) { + withTempPath { path => + val format = classOf[SimpleWritableDataSource].getName + spark.range(3).select($"id" as "i", $"id" as "j") + .write.format(format).mode("overwrite").save(path.getCanonicalPath) + + val df = + spark.read.format(format).load(path.getCanonicalPath).filter($"i" > 0).orderBy($"i") + assert(df.collect === Array(Row(1, 1), Row(2, 2))) + } + } + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RowToColumnConverterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RowToColumnConverterSuite.scala new file mode 100644 index 0000000000000..1afe742b988ee --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/RowToColumnConverterSuite.scala @@ -0,0 +1,145 @@ +/* + * 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.execution + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData} +import org.apache.spark.sql.execution.vectorized.{OnHeapColumnVector, WritableColumnVector} +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +class RowToColumnConverterSuite extends SparkFunSuite { + def convertRows(rows: Seq[InternalRow], schema: StructType): Seq[WritableColumnVector] = { + val converter = new RowToColumnConverter(schema) + val vectors = + schema.map(f => new OnHeapColumnVector(5, f.dataType)).toArray[WritableColumnVector] + for (row <- rows) { + converter.convert(row, vectors) + } + vectors + } + + test("integer column") { + val schema = StructType(Seq(StructField("i", IntegerType))) + val rows = (0 until 100).map(i => InternalRow(i)) + val vectors = convertRows(rows, schema) + rows.zipWithIndex.map { case (row, i) => + assert(vectors.head.getInt(i) === row.getInt(0)) + } + } + + test("array column") { + val schema = StructType(Seq(StructField("a", ArrayType(IntegerType)))) + val rows = (0 until 100).map { i => + InternalRow(new GenericArrayData(0 until i)) + } + val vectors = convertRows(rows, schema) + rows.zipWithIndex.map { case (row, i) => + assert(vectors.head.getArray(i).array().array === row.getArray(0).array) + } + } + + test("non-nullable array column with null elements") { + val arrayType = ArrayType(IntegerType, containsNull = true) + val schema = StructType(Seq(StructField("a", arrayType, nullable = false))) + val rows = (0 until 100).map { i => + InternalRow(new GenericArrayData((0 until i).map { j => + if (j % 3 == 0) { + null + } else { + j + } + })) + } + val vectors = convertRows(rows, schema) + rows.zipWithIndex.map { case (row, i) => + assert(vectors.head.getArray(i).array().array === row.getArray(0).array) + } + } + + test("nested array column") { + val arrayType = ArrayType(ArrayType(IntegerType)) + val schema = StructType(Seq(StructField("a", arrayType))) + val rows = (0 until 100).map { i => + InternalRow(new GenericArrayData((0 until i).map(j => new GenericArrayData(0 until j)))) + } + val vectors = convertRows(rows, schema) + rows.zipWithIndex.map { case (row, i) => + val result = vectors.head.getArray(i).array().array + .map(_.asInstanceOf[ArrayData].array) + val expected = row.getArray(0).array + .map(_.asInstanceOf[ArrayData].array) + assert(result === expected) + } + } + + test("map column") { + val mapType = MapType(IntegerType, StringType) + val schema = StructType(Seq(StructField("m", mapType))) + val rows = (0 until 100).map { i => + InternalRow(new ArrayBasedMapData( + new GenericArrayData(0 until i), + new GenericArrayData((0 until i).map(j => UTF8String.fromString(s"str$j"))))) + } + val vectors = convertRows(rows, schema) + rows.zipWithIndex.map { case (row, i) => + val result = vectors.head.getMap(i) + val expected = row.getMap(0) + assert(result.keyArray().array().array === expected.keyArray().array) + assert(result.valueArray().array().array === expected.valueArray().array) + } + } + + test("non-nullable map column with null values") { + val mapType = MapType(IntegerType, StringType, valueContainsNull = true) + val schema = StructType(Seq(StructField("m", mapType, nullable = false))) + val rows = (0 until 100).map { i => + InternalRow(new ArrayBasedMapData( + new GenericArrayData(0 until i), + new GenericArrayData((0 until i).map { j => + if (j % 3 == 0) { + null + } else { + UTF8String.fromString(s"str$j") + } + }))) + } + val vectors = convertRows(rows, schema) + rows.zipWithIndex.map { case (row, i) => + val result = vectors.head.getMap(i) + val expected = row.getMap(0) + assert(result.keyArray().array().array === expected.keyArray().array) + assert(result.valueArray().array().array === expected.valueArray().array) + } + } + + test("multiple columns") { + val schema = StructType( + Seq(StructField("s", ShortType), StructField("i", IntegerType), StructField("l", LongType))) + val rows = (0 until 100).map { i => + InternalRow((3 * i).toShort, 3 * i + 1, (3 * i + 2).toLong) + } + val vectors = convertRows(rows, schema) + rows.zipWithIndex.map { case (row, i) => + assert(vectors(0).getShort(i) === row.getShort(0)) + assert(vectors(1).getInt(i) === row.getInt(1)) + assert(vectors(2).getLong(i) === row.getLong(2)) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala index 68e1a682562ac..cf5ef63a46c18 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.catalyst.plans.logical.Repartition import org.apache.spark.sql.internal.SQLConf._ import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} @@ -258,6 +259,59 @@ abstract class SQLViewTestSuite extends QueryTest with SQLTestUtils { checkViewOutput(viewName, Seq(Row(2))) } } + + test("SPARK-34490 - query should fail if the view refers a dropped table") { + withTable("t") { + Seq(2, 3, 1).toDF("c1").write.format("parquet").saveAsTable("t") + val viewName = createView("testView", "SELECT * FROM t") + withView(viewName) { + // Always create a temp view in this case, not use `createView` on purpose + sql("CREATE TEMP VIEW t AS SELECT 1 AS c1") + withTempView("t") { + checkViewOutput(viewName, Seq(Row(2), Row(3), Row(1))) + // Manually drop table `t` to see if the query will fail + sql("DROP TABLE IF EXISTS default.t") + val e = intercept[AnalysisException] { + sql(s"SELECT * FROM $viewName").collect() + }.getMessage + assert(e.contains("Table or view not found: t")) + } + } + } + } + + test("SPARK-34613: Fix view does not capture disable hint config") { + withSQLConf(DISABLE_HINTS.key -> "true") { + val viewName = createView("v1", "SELECT /*+ repartition(1) */ 1") + withView(viewName) { + assert( + sql(s"SELECT * FROM $viewName").queryExecution.analyzed.collect { + case e: Repartition => e + }.isEmpty + ) + checkViewOutput(viewName, Seq(Row(1))) + } + } + } + + test("SPARK-34504: drop an invalid view") { + withTable("t") { + sql("CREATE TABLE t(s STRUCT) USING json") + val viewName = createView("v", "SELECT s.i FROM t") + withView(viewName) { + assert(spark.table(viewName).collect().isEmpty) + + // re-create the table without nested field `i` which is referred by the view. + sql("DROP TABLE t") + sql("CREATE TABLE t(s STRUCT) USING json") + val e = intercept[AnalysisException](spark.table(viewName)) + assert(e.message.contains("No such struct field i in j")) + + // drop invalid view should be fine + sql(s"DROP VIEW $viewName") + } + } + } } class LocalTempViewTestSuite extends SQLViewTestSuite with SharedSparkSession { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 76d309cef8610..43aae7d59f9e1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -21,6 +21,7 @@ import java.io.File import java.net.URI import org.apache.log4j.Level +import org.scalatest.PrivateMethodTester import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent, SparkListenerJobStart} import org.apache.spark.sql.{Dataset, QueryTest, Row, SaveMode, SparkSession, Strategy} @@ -45,7 +46,8 @@ import org.apache.spark.util.Utils class AdaptiveQueryExecSuite extends QueryTest with SharedSparkSession - with AdaptiveSparkPlanHelper { + with AdaptiveSparkPlanHelper + with PrivateMethodTester { import testImplicits._ @@ -869,6 +871,25 @@ class AdaptiveQueryExecSuite } } + test("SPARK-34682: CustomShuffleReaderExec operating on canonicalized plan") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + val (_, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT key FROM testData GROUP BY key") + val readers = collect(adaptivePlan) { + case r: CustomShuffleReaderExec => r + } + assert(readers.length == 1) + val reader = readers.head + val c = reader.canonicalized.asInstanceOf[CustomShuffleReaderExec] + // we can't just call execute() because that has separate checks for canonicalized plans + val ex = intercept[IllegalStateException] { + val doExecute = PrivateMethod[Unit](Symbol("doExecute")) + c.invokePrivate(doExecute()) + } + assert(ex.getMessage === "operating on canonicalized plan") + } + } + test("metrics of the shuffle reader") { withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { val (_, adaptivePlan) = runAdaptiveAndVerifyResult( @@ -1461,6 +1482,27 @@ class AdaptiveQueryExecSuite } } + test("SPARK-36020: Check logical link in remove redundant projects") { + withTempView("t") { + spark.range(10).selectExpr("id % 10 as key", "cast(id * 2 as int) as a", + "cast(id * 3 as int) as b", "array(id, id + 1, id + 3) as c").createOrReplaceTempView("t") + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + val query = + """ + |WITH tt AS ( + | SELECT key, a, b, explode(c) AS c FROM t + |) + |SELECT t1.key, t1.c, t2.key, t2.c + |FROM (SELECT a, b, c, key FROM tt WHERE a > 1) t1 + |JOIN (SELECT a, b, c, key FROM tt) t2 + | ON t1.key = t2.key + |""".stripMargin + // here we only need to make sure this query can run + runAdaptiveAndVerifyResult(query) + } + } + } + test("Skew Repartition Fetch in AQE") { def hasRepartitionShuffle(plan: SparkPlan): Boolean = { find(plan) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala index c90732183cb7a..765d2fc584a7d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -774,4 +774,46 @@ abstract class SchemaPruningSuite assert(scanSchema === expectedScanSchema) } } + + testSchemaPruning("SPARK-34963: extract case-insensitive struct field from array") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + val query1 = spark.table("contacts") + .select("friends.First", "friends.MiDDle") + checkScan(query1, "struct>>") + checkAnswer(query1, + Row(Array.empty[String], Array.empty[String]) :: + Row(Array("Susan"), Array("Z.")) :: + Row(null, null) :: + Row(null, null) :: Nil) + + val query2 = spark.table("contacts") + .where("friends.First is not null") + .select("friends.First", "friends.MiDDle") + checkScan(query2, "struct>>") + checkAnswer(query2, + Row(Array.empty[String], Array.empty[String]) :: + Row(Array("Susan"), Array("Z.")) :: Nil) + } + } + + testSchemaPruning("SPARK-34963: extract case-insensitive struct field from struct") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + val query1 = spark.table("contacts") + .select("Name.First", "NAME.MiDDle") + checkScan(query1, "struct>") + checkAnswer(query1, + Row("Jane", "X.") :: + Row("Janet", null) :: + Row("Jim", null) :: + Row("John", "Y.") :: Nil) + + val query2 = spark.table("contacts") + .where("Name.MIDDLE is not null") + .select("Name.First", "NAME.MiDDle") + checkScan(query2, "struct>") + checkAnswer(query2, + Row("Jane", "X.") :: + Row("John", "Y.") :: Nil) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala index 4c489bdcc649e..40fdcaaf3e6c0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala @@ -594,4 +594,20 @@ class OrcSourceSuite extends OrcSuite with SharedSparkSession { val df = readResourceOrcFile("test-data/TestStringDictionary.testRowIndex.orc") assert(df.where("str < 'row 001000'").count() === 1000) } + + test("SPARK-34897: Support reconcile schemas based on index after nested column pruning") { + withTable("t1") { + spark.sql( + """ + |CREATE TABLE t1 ( + | _col0 INT, + | _col1 STRING, + | _col2 STRUCT) + |USING ORC + |""".stripMargin) + + spark.sql("INSERT INTO t1 values(1, '2', struct('a', 'b', 'c', 10L))") + checkAnswer(spark.sql("SELECT _col0, _col2.c1 FROM t1"), Seq(Row(1, "a"))) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index 24a1ba124e56b..2f8cbc924ba6b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -42,7 +42,9 @@ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType +import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy +import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy.{CORRECTED, LEGACY} +import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType.{INT96, TIMESTAMP_MICROS, TIMESTAMP_MILLIS} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.tags.ExtendedSQLTest @@ -70,11 +72,14 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared protected def createParquetFilters( schema: MessageType, - caseSensitive: Option[Boolean] = None): ParquetFilters = + caseSensitive: Option[Boolean] = None, + datetimeRebaseMode: LegacyBehaviorPolicy.Value = LegacyBehaviorPolicy.CORRECTED + ): ParquetFilters = new ParquetFilters(schema, conf.parquetFilterPushDownDate, conf.parquetFilterPushDownTimestamp, conf.parquetFilterPushDownDecimal, conf.parquetFilterPushDownStringStartWith, conf.parquetFilterPushDownInFilterThreshold, - caseSensitive.getOrElse(conf.caseSensitiveAnalysis)) + caseSensitive.getOrElse(conf.caseSensitiveAnalysis), + datetimeRebaseMode) override def beforeEach(): Unit = { super.beforeEach() @@ -521,62 +526,66 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared def date: Date = Date.valueOf(s) } - val data = Seq("2018-03-18", "2018-03-19", "2018-03-20", "2018-03-21") + val data = Seq("1000-01-01", "2018-03-19", "2018-03-20", "2018-03-21") import testImplicits._ Seq(false, true).foreach { java8Api => - withSQLConf(SQLConf.DATETIME_JAVA8API_ENABLED.key -> java8Api.toString) { - val dates = data.map(i => Tuple1(Date.valueOf(i))).toDF() - withNestedParquetDataFrame(dates) { case (inputDF, colName, fun) => - implicit val df: DataFrame = inputDF + Seq(CORRECTED, LEGACY).foreach { rebaseMode => + withSQLConf( + SQLConf.DATETIME_JAVA8API_ENABLED.key -> java8Api.toString, + SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_WRITE.key -> rebaseMode.toString) { + val dates = data.map(i => Tuple1(Date.valueOf(i))).toDF() + withNestedParquetDataFrame(dates) { case (inputDF, colName, fun) => + implicit val df: DataFrame = inputDF + + def resultFun(dateStr: String): Any = { + val parsed = if (java8Api) LocalDate.parse(dateStr) else Date.valueOf(dateStr) + fun(parsed) + } - def resultFun(dateStr: String): Any = { - val parsed = if (java8Api) LocalDate.parse(dateStr) else Date.valueOf(dateStr) - fun(parsed) + val dateAttr: Expression = df(colName).expr + assert(df(colName).expr.dataType === DateType) + + checkFilterPredicate(dateAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate(dateAttr.isNotNull, classOf[NotEq[_]], + data.map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(dateAttr === "1000-01-01".date, classOf[Eq[_]], + resultFun("1000-01-01")) + checkFilterPredicate(dateAttr <=> "1000-01-01".date, classOf[Eq[_]], + resultFun("1000-01-01")) + checkFilterPredicate(dateAttr =!= "1000-01-01".date, classOf[NotEq[_]], + Seq("2018-03-19", "2018-03-20", "2018-03-21").map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(dateAttr < "2018-03-19".date, classOf[Lt[_]], + resultFun("1000-01-01")) + checkFilterPredicate(dateAttr > "2018-03-20".date, classOf[Gt[_]], + resultFun("2018-03-21")) + checkFilterPredicate(dateAttr <= "1000-01-01".date, classOf[LtEq[_]], + resultFun("1000-01-01")) + checkFilterPredicate(dateAttr >= "2018-03-21".date, classOf[GtEq[_]], + resultFun("2018-03-21")) + + checkFilterPredicate(Literal("1000-01-01".date) === dateAttr, classOf[Eq[_]], + resultFun("1000-01-01")) + checkFilterPredicate(Literal("1000-01-01".date) <=> dateAttr, classOf[Eq[_]], + resultFun("1000-01-01")) + checkFilterPredicate(Literal("2018-03-19".date) > dateAttr, classOf[Lt[_]], + resultFun("1000-01-01")) + checkFilterPredicate(Literal("2018-03-20".date) < dateAttr, classOf[Gt[_]], + resultFun("2018-03-21")) + checkFilterPredicate(Literal("1000-01-01".date) >= dateAttr, classOf[LtEq[_]], + resultFun("1000-01-01")) + checkFilterPredicate(Literal("2018-03-21".date) <= dateAttr, classOf[GtEq[_]], + resultFun("2018-03-21")) + + checkFilterPredicate(!(dateAttr < "2018-03-21".date), classOf[GtEq[_]], + resultFun("2018-03-21")) + checkFilterPredicate( + dateAttr < "2018-03-19".date || dateAttr > "2018-03-20".date, + classOf[Operators.Or], + Seq(Row(resultFun("1000-01-01")), Row(resultFun("2018-03-21")))) } - - val dateAttr: Expression = df(colName).expr - assert(df(colName).expr.dataType === DateType) - - checkFilterPredicate(dateAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate(dateAttr.isNotNull, classOf[NotEq[_]], - data.map(i => Row.apply(resultFun(i)))) - - checkFilterPredicate(dateAttr === "2018-03-18".date, classOf[Eq[_]], - resultFun("2018-03-18")) - checkFilterPredicate(dateAttr <=> "2018-03-18".date, classOf[Eq[_]], - resultFun("2018-03-18")) - checkFilterPredicate(dateAttr =!= "2018-03-18".date, classOf[NotEq[_]], - Seq("2018-03-19", "2018-03-20", "2018-03-21").map(i => Row.apply(resultFun(i)))) - - checkFilterPredicate(dateAttr < "2018-03-19".date, classOf[Lt[_]], - resultFun("2018-03-18")) - checkFilterPredicate(dateAttr > "2018-03-20".date, classOf[Gt[_]], - resultFun("2018-03-21")) - checkFilterPredicate(dateAttr <= "2018-03-18".date, classOf[LtEq[_]], - resultFun("2018-03-18")) - checkFilterPredicate(dateAttr >= "2018-03-21".date, classOf[GtEq[_]], - resultFun("2018-03-21")) - - checkFilterPredicate(Literal("2018-03-18".date) === dateAttr, classOf[Eq[_]], - resultFun("2018-03-18")) - checkFilterPredicate(Literal("2018-03-18".date) <=> dateAttr, classOf[Eq[_]], - resultFun("2018-03-18")) - checkFilterPredicate(Literal("2018-03-19".date) > dateAttr, classOf[Lt[_]], - resultFun("2018-03-18")) - checkFilterPredicate(Literal("2018-03-20".date) < dateAttr, classOf[Gt[_]], - resultFun("2018-03-21")) - checkFilterPredicate(Literal("2018-03-18".date) >= dateAttr, classOf[LtEq[_]], - resultFun("2018-03-18")) - checkFilterPredicate(Literal("2018-03-21".date) <= dateAttr, classOf[GtEq[_]], - resultFun("2018-03-21")) - - checkFilterPredicate(!(dateAttr < "2018-03-21".date), classOf[GtEq[_]], - resultFun("2018-03-21")) - checkFilterPredicate( - dateAttr < "2018-03-19".date || dateAttr > "2018-03-20".date, - classOf[Operators.Or], - Seq(Row(resultFun("2018-03-18")), Row(resultFun("2018-03-21")))) } } } @@ -584,35 +593,36 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared test("filter pushdown - timestamp") { Seq(true, false).foreach { java8Api => - withSQLConf( - SQLConf.DATETIME_JAVA8API_ENABLED.key -> java8Api.toString, - SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_WRITE.key -> "CORRECTED", - SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_WRITE.key -> "CORRECTED") { - // spark.sql.parquet.outputTimestampType = TIMESTAMP_MILLIS + Seq(CORRECTED, LEGACY).foreach { rebaseMode => val millisData = Seq( "1000-06-14 08:28:53.123", "1582-06-15 08:28:53.001", "1900-06-16 08:28:53.0", "2018-06-17 08:28:53.999") - withSQLConf(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> - ParquetOutputTimestampType.TIMESTAMP_MILLIS.toString) { + withSQLConf( + SQLConf.DATETIME_JAVA8API_ENABLED.key -> java8Api.toString, + SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_WRITE.key -> rebaseMode.toString, + SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> TIMESTAMP_MILLIS.toString) { testTimestampPushdown(millisData, java8Api) } - // spark.sql.parquet.outputTimestampType = TIMESTAMP_MICROS val microsData = Seq( "1000-06-14 08:28:53.123456", "1582-06-15 08:28:53.123456", "1900-06-16 08:28:53.123456", "2018-06-17 08:28:53.123456") - withSQLConf(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> - ParquetOutputTimestampType.TIMESTAMP_MICROS.toString) { + withSQLConf( + SQLConf.DATETIME_JAVA8API_ENABLED.key -> java8Api.toString, + SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_WRITE.key -> rebaseMode.toString, + SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> TIMESTAMP_MICROS.toString) { testTimestampPushdown(microsData, java8Api) } - // spark.sql.parquet.outputTimestampType = INT96 doesn't support pushdown - withSQLConf(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> - ParquetOutputTimestampType.INT96.toString) { + // INT96 doesn't support pushdown + withSQLConf( + SQLConf.DATETIME_JAVA8API_ENABLED.key -> java8Api.toString, + SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_WRITE.key -> rebaseMode.toString, + SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> INT96.toString) { import testImplicits._ withTempPath { file => millisData.map(i => Tuple1(Timestamp.valueOf(i))).toDF diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala index 2fe5953cbe12e..f7a2535d7486e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{ArrayType, IntegerType, StructField, StructType} class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedSparkSession { test("parquet files with different physical schemas but share the same logical schema") { @@ -97,6 +98,58 @@ class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedS } } + test("SPARK-36803: parquet files with legacy mode and schema evolution") { + // This test case writes arrays in Parquet legacy mode and schema evolution and verifies that + // the data can be correctly read back. + + Seq(false, true).foreach { legacyMode => + withSQLConf(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key -> legacyMode.toString) { + withTempPath { tableDir => + val schema1 = StructType( + StructField("col-0", ArrayType( + StructType( + StructField("col-0", IntegerType, true) :: + Nil + ), + containsNull = false // allows to create 2-level Parquet LIST type in legacy mode + )) :: + Nil + ) + val row1 = Row(Seq(Row(1))) + val df1 = spark.createDataFrame(spark.sparkContext.parallelize(row1 :: Nil, 1), schema1) + df1.write.parquet(tableDir.getAbsolutePath) + + val schema2 = StructType( + StructField("col-0", ArrayType( + StructType( + StructField("col-0", IntegerType, true) :: + StructField("col-1", IntegerType, true) :: // additional field + Nil + ), + containsNull = false + )) :: + Nil + ) + val row2 = Row(Seq(Row(1, 2))) + val df2 = spark.createDataFrame(spark.sparkContext.parallelize(row2 :: Nil, 1), schema2) + df2.write.mode("append").parquet(tableDir.getAbsolutePath) + + // Reading of data should succeed and should not fail with + // java.lang.ClassCastException: optional int32 col-0 is not a group + withAllParquetReaders { + checkAnswer( + spark.read.schema(schema2).parquet(tableDir.getAbsolutePath), + Seq( + Row(Seq(Row(1, null))), + Row(Seq(Row(1, 2))) + ) + ) + } + } + } + } + } + test("parquet timestamp conversion") { // Make a table with one parquet file written by impala, and one parquet file written by spark. // We should only adjust the timestamps in the impala file, and only if the conf is set diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala index 9f7e0a14f6a5c..238d37afe1075 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala @@ -104,18 +104,16 @@ class OuterJoinSuite extends SparkPlanTest with SharedSparkSession { ExtractEquiJoinKeys.unapply(join) } - if (joinType != FullOuter) { - test(s"$testName using ShuffledHashJoin") { - extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => - withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { - val buildSide = if (joinType == LeftOuter) BuildRight else BuildLeft - checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => - EnsureRequirements.apply( - ShuffledHashJoinExec( - leftKeys, rightKeys, joinType, buildSide, boundCondition, left, right)), - expectedAnswer.map(Row.fromTuple), - sortAnswers = true) - } + test(s"$testName using ShuffledHashJoin") { + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + val buildSide = if (joinType == LeftOuter) BuildRight else BuildLeft + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + EnsureRequirements.apply( + ShuffledHashJoinExec( + leftKeys, rightKeys, joinType, buildSide, boundCondition, left, right)), + expectedAnswer.map(Row.fromTuple), + sortAnswers = true) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 21d17f40abb34..d5f9875a5926b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.execution.{FilterExec, RangeExec, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite import org.apache.spark.sql.execution.aggregate.HashAggregateExec +import org.apache.spark.sql.execution.command.DataWritingCommandExec import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec import org.apache.spark.sql.functions._ @@ -755,4 +756,20 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils } } } + + test("SPARK-34567: Add metrics for CTAS operator") { + withTable("t") { + val df = sql("CREATE TABLE t USING PARQUET AS SELECT 1 as a") + val dataWritingCommandExec = + df.queryExecution.executedPlan.asInstanceOf[DataWritingCommandExec] + dataWritingCommandExec.executeCollect() + val createTableAsSelect = dataWritingCommandExec.cmd + assert(createTableAsSelect.metrics.contains("numFiles")) + assert(createTableAsSelect.metrics("numFiles").value == 1) + assert(createTableAsSelect.metrics.contains("numOutputBytes")) + assert(createTableAsSelect.metrics("numOutputBytes").value > 0) + assert(createTableAsSelect.metrics.contains("numOutputRows")) + assert(createTableAsSelect.metrics("numOutputRows").value == 1) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala index 247efd5554a8f..43f48abb9734f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala @@ -243,6 +243,93 @@ class ColumnVectorSuite extends SparkFunSuite with BeforeAndAfterEach { assert(testVector.getArray(3).toIntArray() === Array(3, 4, 5)) } + testVectors("SPARK-35898: array append", 1, arrayType) { testVector => + // Populate it with arrays [0], [1, 2], [], [3, 4, 5] + val data = testVector.arrayData() + testVector.appendArray(1) + data.appendInt(0) + testVector.appendArray(2) + data.appendInt(1) + data.appendInt(2) + testVector.appendArray(0) + testVector.appendArray(3) + data.appendInt(3) + data.appendInt(4) + data.appendInt(5) + + assert(testVector.getArray(0).toIntArray === Array(0)) + assert(testVector.getArray(1).toIntArray === Array(1, 2)) + assert(testVector.getArray(2).toIntArray === Array.empty[Int]) + assert(testVector.getArray(3).toIntArray === Array(3, 4, 5)) + } + + val mapType: MapType = MapType(IntegerType, StringType) + testVectors("SPARK-35898: map", 5, mapType) { testVector => + val keys = testVector.getChild(0) + val values = testVector.getChild(1) + var i = 0 + while (i < 6) { + keys.appendInt(i) + val utf8 = s"str$i".getBytes("utf8") + values.appendByteArray(utf8, 0, utf8.length) + i += 1 + } + + testVector.putArray(0, 0, 1) + testVector.putArray(1, 1, 2) + testVector.putArray(2, 3, 0) + testVector.putArray(3, 3, 3) + + assert(testVector.getMap(0).keyArray().toIntArray === Array(0)) + assert(testVector.getMap(0).valueArray().toArray[UTF8String](StringType) === + Array(UTF8String.fromString(s"str0"))) + assert(testVector.getMap(1).keyArray().toIntArray === Array(1, 2)) + assert(testVector.getMap(1).valueArray().toArray[UTF8String](StringType) === + (1 to 2).map(i => UTF8String.fromString(s"str$i")).toArray) + assert(testVector.getMap(2).keyArray().toIntArray === Array.empty[Int]) + assert(testVector.getMap(2).valueArray().toArray[UTF8String](StringType) === + Array.empty[UTF8String]) + assert(testVector.getMap(3).keyArray().toIntArray === Array(3, 4, 5)) + assert(testVector.getMap(3).valueArray().toArray[UTF8String](StringType) === + (3 to 5).map(i => UTF8String.fromString(s"str$i")).toArray) + } + + testVectors("SPARK-35898: map append", 1, mapType) { testVector => + val keys = testVector.getChild(0) + val values = testVector.getChild(1) + def appendPair(i: Int): Unit = { + keys.appendInt(i) + val utf8 = s"str$i".getBytes("utf8") + values.appendByteArray(utf8, 0, utf8.length) + } + + // Populate it with the maps [0 -> str0], [1 -> str1, 2 -> str2], [], + // [3 -> str3, 4 -> str4, 5 -> str5] + testVector.appendArray(1) + appendPair(0) + testVector.appendArray(2) + appendPair(1) + appendPair(2) + testVector.appendArray(0) + testVector.appendArray(3) + appendPair(3) + appendPair(4) + appendPair(5) + + assert(testVector.getMap(0).keyArray().toIntArray === Array(0)) + assert(testVector.getMap(0).valueArray().toArray[UTF8String](StringType) === + Array(UTF8String.fromString(s"str0"))) + assert(testVector.getMap(1).keyArray().toIntArray === Array(1, 2)) + assert(testVector.getMap(1).valueArray().toArray[UTF8String](StringType) === + (1 to 2).map(i => UTF8String.fromString(s"str$i")).toArray) + assert(testVector.getMap(2).keyArray().toIntArray === Array.empty[Int]) + assert(testVector.getMap(2).valueArray().toArray[UTF8String](StringType) === + Array.empty[UTF8String]) + assert(testVector.getMap(3).keyArray().toIntArray === Array(3, 4, 5)) + assert(testVector.getMap(3).valueArray().toArray[UTF8String](StringType) === + (3 to 5).map(i => UTF8String.fromString(s"str$i")).toArray) + } + val structType: StructType = new StructType().add("int", IntegerType).add("double", DoubleType) testVectors("struct", 10, structType) { testVector => val c1 = testVector.getChild(0) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 1ea2d4fd0b32c..bf0edcac99fb7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -117,6 +117,21 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { } } + test(s"SPARK-35168: ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} should respect" + + s" ${SQLConf.SHUFFLE_PARTITIONS.key}") { + spark.sessionState.conf.clear() + try { + sql(s"SET ${SQLConf.ADAPTIVE_EXECUTION_ENABLED.key}=true") + sql(s"SET ${SQLConf.COALESCE_PARTITIONS_ENABLED.key}=true") + sql(s"SET ${SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key}=1") + sql(s"SET ${SQLConf.SHUFFLE_PARTITIONS.key}=2") + checkAnswer(sql(s"SET ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS}"), + Row(SQLConf.SHUFFLE_PARTITIONS.key, "2")) + } finally { + spark.sessionState.conf.clear() + } + } + test("SPARK-31234: reset will not change static sql configs and spark core configs") { val conf = spark.sparkContext.getConf.getAll.toMap val appName = conf.get("spark.app.name") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala index 179cdeb976391..9b60f5d94bcce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala @@ -258,4 +258,30 @@ abstract class DisableUnnecessaryBucketedScanSuite } } } + + test("Aggregates with no groupby over tables having 1 BUCKET, return multiple rows") { + withTable("t1") { + withSQLConf(SQLConf.AUTO_BUCKETED_SCAN_ENABLED.key -> "true") { + sql( + """ + |CREATE TABLE t1 (`id` BIGINT, `event_date` DATE) + |USING PARQUET + |CLUSTERED BY (id) + |INTO 1 BUCKETS + |""".stripMargin) + sql( + """ + |INSERT INTO TABLE t1 VALUES(1.23, cast("2021-07-07" as date)) + |""".stripMargin) + sql( + """ + |INSERT INTO TABLE t1 VALUES(2.28, cast("2021-08-08" as date)) + |""".stripMargin) + val df = spark.sql("select sum(id) from t1 where id is not null") + assert(df.count == 1) + checkDisableBucketedScan(query = "SELECT SUM(id) FROM t1 WHERE id is not null", + expectedNumScanWithAutoScanEnabled = 1, expectedNumScanWithAutoScanDisabled = 1) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index aaf8765c04425..1d0a57a216c32 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources -import java.io.File +import java.io.{File, IOException} import java.sql.Date import org.apache.hadoop.fs.{FileAlreadyExistsException, FSDataOutputStream, Path, RawLocalFileSystem} @@ -797,27 +797,6 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { } } - test("SPARK-30844: static partition should also follow StoreAssignmentPolicy") { - SQLConf.StoreAssignmentPolicy.values.foreach { policy => - withSQLConf( - SQLConf.STORE_ASSIGNMENT_POLICY.key -> policy.toString) { - withTable("t") { - sql("create table t(a int, b string) using parquet partitioned by (a)") - policy match { - case SQLConf.StoreAssignmentPolicy.ANSI | SQLConf.StoreAssignmentPolicy.STRICT => - val errorMsg = intercept[NumberFormatException] { - sql("insert into t partition(a='ansi') values('ansi')") - }.getMessage - assert(errorMsg.contains("invalid input syntax for type numeric: ansi")) - case SQLConf.StoreAssignmentPolicy.LEGACY => - sql("insert into t partition(a='ansi') values('ansi')") - checkAnswer(sql("select * from t"), Row("ansi", null) :: Nil) - } - } - } - } - } - test("SPARK-24860: dynamic partition overwrite specified per source without catalog table") { withTempPath { path => Seq((1, 1), (2, 2)).toDF("i", "part") @@ -954,6 +933,127 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { assert(msg.contains("cannot resolve '`c3`' given input columns")) } } + + test("SPARK-34926: PartitioningUtils.getPathFragment() should respect partition value is null") { + withTable("t1", "t2") { + sql("CREATE TABLE t1(id INT) USING PARQUET") + sql( + """ + |CREATE TABLE t2 (c1 INT, part STRING) + | USING parquet + |PARTITIONED BY (part) + |""".stripMargin) + sql( + """ + |INSERT INTO TABLE t2 PARTITION (part = null) + |SELECT * FROM t1 where 1=0""".stripMargin) + checkAnswer(spark.table("t2"), Nil) + } + } + + test("SPARK-35106: insert overwrite with custom partition path") { + withTempPath { path => + withTable("t") { + sql( + """ + |create table t(i int, part1 int, part2 int) using parquet + |partitioned by (part1, part2) + """.stripMargin) + + sql(s"alter table t add partition(part1=1, part2=1) location '${path.getAbsolutePath}'") + sql(s"insert into t partition(part1=1, part2=1) select 1") + checkAnswer(spark.table("t"), Row(1, 1, 1)) + + sql("insert overwrite table t partition(part1=1, part2=1) select 2") + checkAnswer(spark.table("t"), Row(2, 1, 1)) + + sql("insert overwrite table t partition(part1=2, part2) select 2, 2") + checkAnswer(spark.table("t"), Row(2, 1, 1) :: Row(2, 2, 2) :: Nil) + + sql("insert overwrite table t partition(part1=1, part2=2) select 3") + checkAnswer(spark.table("t"), Row(2, 1, 1) :: Row(2, 2, 2) :: Row(3, 1, 2) :: Nil) + + sql("insert overwrite table t partition(part1=1, part2) select 4, 1") + checkAnswer(spark.table("t"), Row(4, 1, 1) :: Row(2, 2, 2) :: Nil) + } + } + } + + test("SPARK-35106: dynamic partition overwrite with custom partition path") { + withSQLConf(SQLConf.PARTITION_OVERWRITE_MODE.key -> PartitionOverwriteMode.DYNAMIC.toString) { + withTempPath { path => + withTable("t") { + sql( + """ + |create table t(i int, part1 int, part2 int) using parquet + |partitioned by (part1, part2) + """.stripMargin) + + sql(s"insert into t partition(part1=1, part2=1) select 1") + checkAnswer(spark.table("t"), Row(1, 1, 1)) + + sql(s"alter table t add partition(part1=1, part2=2) location '${path.getAbsolutePath}'") + + // dynamic partition overwrite to empty custom partition + sql(s"insert overwrite table t partition(part1=1, part2=2) select 1") + checkAnswer(spark.table("t"), Row(1, 1, 1) :: Row(1, 1, 2) :: Nil) + + // dynamic partition overwrite to non-empty custom partition + sql("insert overwrite table t partition(part1=1, part2=2) select 2") + checkAnswer(spark.table("t"), Row(1, 1, 1) :: Row(2, 1, 2) :: Nil) + } + } + } + } + + test("SPARK-35106: Throw exception when rename custom partition paths returns false") { + withSQLConf( + "fs.file.impl" -> classOf[RenameFromSparkStagingToFinalDirAlwaysTurnsFalseFilesystem].getName, + "fs.file.impl.disable.cache" -> "true") { + withTempPath { path => + withTable("t") { + sql( + """ + |create table t(i int, part1 int, part2 int) using parquet + |partitioned by (part1, part2) + """.stripMargin) + + sql(s"alter table t add partition(part1=1, part2=1) location '${path.getAbsolutePath}'") + + val e = intercept[SparkException] { + sql(s"insert into t partition(part1=1, part2=1) select 1") + }.getCause + assert(e.isInstanceOf[IOException]) + assert(e.getMessage.contains("Failed to rename")) + assert(e.getMessage.contains("when committing files staged for absolute location")) + } + } + } + } + + test("SPARK-35106: Throw exception when rename dynamic partition paths returns false") { + withSQLConf( + "fs.file.impl" -> classOf[RenameFromSparkStagingToFinalDirAlwaysTurnsFalseFilesystem].getName, + "fs.file.impl.disable.cache" -> "true", + SQLConf.PARTITION_OVERWRITE_MODE.key -> PartitionOverwriteMode.DYNAMIC.toString) { + + withTable("t") { + sql( + """ + |create table t(i int, part1 int, part2 int) using parquet + |partitioned by (part1, part2) + """.stripMargin) + + val e = intercept[SparkException] { + sql(s"insert overwrite table t partition(part1, part2) values (1, 1, 1)") + }.getCause + assert(e.isInstanceOf[IOException]) + assert(e.getMessage.contains("Failed to rename")) + assert(e.getMessage.contains( + "when committing files staged for overwriting dynamic partitions")) + } + } + } } class FileExistingTestFileSystem extends RawLocalFileSystem { @@ -966,3 +1066,13 @@ class FileExistingTestFileSystem extends RawLocalFileSystem { throw new FileAlreadyExistsException(s"${f.toString} already exists") } } + +class RenameFromSparkStagingToFinalDirAlwaysTurnsFalseFilesystem extends RawLocalFileSystem { + override def rename(src: Path, dst: Path): Boolean = { + (!isSparkStagingDir(src) || isSparkStagingDir(dst)) && super.rename(src, dst) + } + + private def isSparkStagingDir(path: Path): Boolean = { + path.toString.contains(".spark-staging-") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index ed284df10aced..0d2d00f1f5b2b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -34,7 +34,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.{SparkConf, SparkContext, TaskContext, TestUtils} import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.plans.logical.Range +import org.apache.spark.sql.catalyst.plans.logical.{Range, RepartitionByExpression} import org.apache.spark.sql.catalyst.streaming.{InternalOutputModes, StreamingRelationV2} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.{LocalLimitExec, SimpleMode, SparkPlan} @@ -1264,6 +1264,37 @@ class StreamSuite extends StreamTest { } } } + + test("SPARK-34482: correct active SparkSession for logicalPlan") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "10") { + val df = spark.readStream.format(classOf[FakeDefaultSource].getName).load() + var query: StreamExecution = null + try { + query = + df.repartition($"a") + .writeStream + .format("memory") + .queryName("memory") + .start() + .asInstanceOf[StreamingQueryWrapper] + .streamingQuery + query.awaitInitialization(streamingTimeout.toMillis) + val plan = query.logicalPlan + val numPartition = plan + .find { _.isInstanceOf[RepartitionByExpression] } + .map(_.asInstanceOf[RepartitionByExpression].numPartitions) + // Before the fix of SPARK-34482, the numPartition is the value of + // `COALESCE_PARTITIONS_INITIAL_PARTITION_NUM`. + assert(numPartition.get === spark.sqlContext.conf.getConf(SQLConf.SHUFFLE_PARTITIONS)) + } finally { + if (query != null) { + query.stop() + } + } + } + } } abstract class FakeSource extends StreamSourceProvider { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index 156528776d945..adf9764f0f22e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -417,7 +417,8 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { min($"value").as("min_val"), max($"value").as("max_val"), sum($"value").as("sum_val"), - count(when($"value" % 2 === 0, 1)).as("num_even")) + count(when($"value" % 2 === 0, 1)).as("num_even"), + percentile_approx($"value", lit(0.5), lit(100)).as("percentile_approx_val")) .observe( name = "other_event", avg($"value").cast("int").as("avg_val")) @@ -444,7 +445,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { AddData(inputData, 1, 2), AdvanceManualClock(100), checkMetrics { metrics => - assert(metrics.get("my_event") === Row(1, 2, 3L, 1L)) + assert(metrics.get("my_event") === Row(1, 2, 3L, 1L, 1)) assert(metrics.get("other_event") === Row(1)) }, @@ -452,7 +453,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { AddData(inputData, 10, 30, -10, 5), AdvanceManualClock(100), checkMetrics { metrics => - assert(metrics.get("my_event") === Row(-10, 30, 35L, 3L)) + assert(metrics.get("my_event") === Row(-10, 30, 35L, 3L, 5)) assert(metrics.get("other_event") === Row(8)) }, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala index b17c93503804c..db2b7b7504a13 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.util +import java.lang.{Long => JLong} + import scala.collection.mutable.ArrayBuffer import org.apache.spark._ @@ -29,6 +31,7 @@ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.execution.datasources.{CreateTable, InsertIntoHadoopFsRelationCommand} import org.apache.spark.sql.execution.datasources.json.JsonFileFormat +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StringType @@ -234,18 +237,58 @@ class DataFrameCallbackSuite extends QueryTest } test("get observable metrics by callback") { - val metricMaps = ArrayBuffer.empty[Map[String, Row]] - val listener = new QueryExecutionListener { - override def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { - metricMaps += qe.observedMetrics - } + val df = spark.range(100) + .observe( + name = "my_event", + min($"id").as("min_val"), + max($"id").as("max_val"), + // Test unresolved alias + sum($"id"), + count(when($"id" % 2 === 0, 1)).as("num_even")) + .observe( + name = "other_event", + avg($"id").cast("int").as("avg_val")) + + validateObservedMetrics(df) + } - override def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit = { - // No-op - } - } - spark.listenerManager.register(listener) - try { + test("SPARK-35296: observe should work even if a task contains multiple partitions") { + val df = spark.range(0, 100, 1, 3) + .observe( + name = "my_event", + min($"id").as("min_val"), + max($"id").as("max_val"), + // Test unresolved alias + sum($"id"), + count(when($"id" % 2 === 0, 1)).as("num_even")) + .observe( + name = "other_event", + avg($"id").cast("int").as("avg_val")) + .coalesce(2) + + validateObservedMetrics(df) + } + + test("SPARK-35695: get observable metrics with persist by callback") { + val df = spark.range(100) + .observe( + name = "my_event", + min($"id").as("min_val"), + max($"id").as("max_val"), + // Test unresolved alias + sum($"id"), + count(when($"id" % 2 === 0, 1)).as("num_even")) + .persist() + .observe( + name = "other_event", + avg($"id").cast("int").as("avg_val")) + .persist() + + validateObservedMetrics(df) + } + + test("SPARK-35695: get observable metrics with adaptive execution by callback") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { val df = spark.range(100) .observe( name = "my_event", @@ -253,10 +296,28 @@ class DataFrameCallbackSuite extends QueryTest max($"id").as("max_val"), sum($"id").as("sum_val"), count(when($"id" % 2 === 0, 1)).as("num_even")) + .repartition($"id") .observe( name = "other_event", avg($"id").cast("int").as("avg_val")) + validateObservedMetrics(df) + } + } + + private def validateObservedMetrics(df: Dataset[JLong]): Unit = { + val metricMaps = ArrayBuffer.empty[Map[String, Row]] + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { + metricMaps += qe.observedMetrics + } + + override def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit = { + // No-op + } + } + spark.listenerManager.register(listener) + try { def checkMetrics(metrics: Map[String, Row]): Unit = { assert(metrics.size === 2) assert(metrics("my_event") === Row(0L, 99L, 4950L, 50L)) @@ -281,6 +342,7 @@ class DataFrameCallbackSuite extends QueryTest } } + testQuietly("SPARK-31144: QueryExecutionListener should receive `java.lang.Error`") { var e: Exception = null val listener = new QueryExecutionListener { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 46ebcb72bbec5..1e43f2cc62ae6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -40,7 +40,7 @@ import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, CharVarcharUtils, DateTimeUtils} import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.{PartitioningUtils, SourceOptions} import org.apache.spark.sql.hive.client.HiveClient @@ -429,12 +429,17 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat val properties = new mutable.HashMap[String, String] properties.put(CREATED_SPARK_VERSION, table.createVersion) - + // This is for backward compatibility to Spark 2 to read tables with char/varchar created by + // Spark 3.1. At read side, we will restore a table schema from its properties. So, we need to + // clear the `varchar(n)` and `char(n)` and replace them with `string` as Spark 2 does not have + // a type mapping for them in `DataType.nameToType`. + // See `restoreHiveSerdeTable` for example. + val newSchema = CharVarcharUtils.replaceCharVarcharWithStringInSchema(schema) // Serialized JSON schema string may be too long to be stored into a single metastore table // property. In this case, we split the JSON string and store each part as a separate table // property. val threshold = conf.get(SCHEMA_STRING_LENGTH_THRESHOLD) - val schemaJsonString = schema.json + val schemaJsonString = newSchema.json // Split the JSON string. val parts = schemaJsonString.grouped(threshold).toSeq properties.put(DATASOURCE_SCHEMA_NUMPARTS, parts.size.toString) @@ -745,7 +750,8 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat // If this is a view created by Spark 2.2 or higher versions, we should restore its schema // from table properties. if (table.properties.contains(DATASOURCE_SCHEMA_NUMPARTS)) { - table = table.copy(schema = getSchemaFromTableProperties(table)) + val newSchema = CharVarcharUtils.getRawSchema(getSchemaFromTableProperties(table)) + table = table.copy(schema = newSchema) } // No provider in table properties, which means this is a Hive serde table. @@ -796,7 +802,8 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat // If this is a Hive serde table created by Spark 2.1 or higher versions, we should restore its // schema from table properties. if (table.properties.contains(DATASOURCE_SCHEMA_NUMPARTS)) { - val schemaFromTableProps = getSchemaFromTableProperties(table) + val schemaFromTableProps = + CharVarcharUtils.getRawSchema(getSchemaFromTableProperties(table)) val partColumnNames = getPartitionColumnsFromTableProperties(table) val reorderedSchema = reorderSchema(schema = schemaFromTableProps, partColumnNames) @@ -836,7 +843,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat storageWithLocation.properties.filterKeys(!HIVE_GENERATED_STORAGE_PROPERTIES(_)).toMap) val partitionProvider = table.properties.get(TABLE_PARTITION_PROVIDER) - val schemaFromTableProps = getSchemaFromTableProperties(table) + val schemaFromTableProps = CharVarcharUtils.getRawSchema(getSchemaFromTableProperties(table)) val partColumnNames = getPartitionColumnsFromTableProperties(table) val reorderedSchema = reorderSchema(schema = schemaFromTableProps, partColumnNames) @@ -1081,7 +1088,11 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat // scalastyle:on caselocale val actualPartitionPath = new Path(currentFullPath, actualPartitionString) try { - fs.rename(actualPartitionPath, expectedPartitionPath) + fs.mkdirs(expectedPartitionPath) + if(!fs.rename(actualPartitionPath, expectedPartitionPath)) { + throw new IOException(s"Renaming partition path from $actualPartitionPath to " + + s"$expectedPartitionPath returned false") + } } catch { case e: IOException => throw new SparkException("Unable to rename partition path from " + diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index a89243c331c7b..c67bc7d344781 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -244,7 +244,9 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log paths = rootPath.toString :: Nil, userSpecifiedSchema = Option(updatedTable.dataSchema), bucketSpec = None, - options = options, + // Do not interpret the 'path' option at all when tables are read using the Hive + // source, since the URIs will already have been read from the table's LOCATION. + options = options.filter { case (k, _) => !k.equalsIgnoreCase("path") }, className = fileType).resolveRelation(), table = updatedTable) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala index 3a53a2a8dadd8..351cde58427c6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala @@ -101,10 +101,19 @@ private[hive] object HiveShim { def toCatalystDecimal(hdoi: HiveDecimalObjectInspector, data: Any): Decimal = { if (hdoi.preferWritable()) { - Decimal(hdoi.getPrimitiveWritableObject(data).getHiveDecimal().bigDecimalValue, - hdoi.precision(), hdoi.scale()) + val value = hdoi.getPrimitiveWritableObject(data) + if (value == null) { + null + } else { + Decimal(value.getHiveDecimal().bigDecimalValue, hdoi.precision(), hdoi.scale()) + } } else { - Decimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue(), hdoi.precision(), hdoi.scale()) + val value = hdoi.getPrimitiveJavaObject(data) + if (value == null) { + null + } else { + Decimal(value.bigDecimalValue(), hdoi.precision(), hdoi.scale()) + } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index ed088648bc20a..db67480ceb77a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -700,7 +700,7 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { } def unapply(values: Set[Any]): Option[Seq[String]] = { - val extractables = values.toSeq.map(valueToLiteralString.lift) + val extractables = values.filter(_ != null).toSeq.map(valueToLiteralString.lift) if (extractables.nonEmpty && extractables.forall(_.isDefined)) { Some(extractables.map(_.get)) } else { @@ -715,7 +715,7 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { } def unapply(values: Set[Any]): Option[Seq[String]] = { - val extractables = values.toSeq.map(valueToLiteralString.lift) + val extractables = values.filter(_ != null).toSeq.map(valueToLiteralString.lift) if (extractables.nonEmpty && extractables.forall(_.isDefined)) { Some(extractables.map(_.get)) } else { @@ -769,7 +769,9 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { case InSet(child, values) if useAdvanced && values.size > inSetThreshold => val dataType = child.dataType - val sortedValues = values.toSeq.sorted(TypeUtils.getInterpretedOrdering(dataType)) + // Skip null here is safe, more details could see at ExtractableLiterals. + val sortedValues = values.filter(_ != null).toSeq + .sorted(TypeUtils.getInterpretedOrdering(dataType)) convert(And(GreaterThanOrEqual(child, Literal(sortedValues.head, dataType)), LessThanOrEqual(child, Literal(sortedValues.last, dataType)))) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala index ccaa4502d9d2a..283c254b39602 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -55,6 +55,7 @@ trait CreateHiveTableAsSelectBase extends DataWritingCommand { val command = getWritingCommand(catalog, tableDesc, tableExists = true) command.run(sparkSession, child) + DataWritingCommand.propogateMetrics(sparkSession.sparkContext, command, metrics) } else { // TODO ideally, we should get the output data ready first and then // add the relation into catalog, just in case of failure occurs while data @@ -69,6 +70,7 @@ trait CreateHiveTableAsSelectBase extends DataWritingCommand { val createdTableMeta = catalog.getTableMetadata(tableDesc.identifier) val command = getWritingCommand(catalog, createdTableMeta, tableExists = false) command.run(sparkSession, child) + DataWritingCommand.propogateMetrics(sparkSession.sparkContext, command, metrics) } catch { case NonFatal(e) => // drop the created table. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationExec.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationExec.scala index 2d1dfbf185830..832463e2512b5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationExec.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationExec.scala @@ -64,7 +64,7 @@ private[hive] case class HiveScriptTransformationExec( outputSoi: StructObjectInspector, hadoopConf: Configuration): Iterator[InternalRow] = { new Iterator[InternalRow] with HiveInspectors { - var curLine: String = null + private var completed = false val scriptOutputStream = new DataInputStream(inputStream) val scriptOutputReader = @@ -78,6 +78,9 @@ private[hive] case class HiveScriptTransformationExec( lazy val unwrappers = outputSoi.getAllStructFieldRefs.asScala.map(unwrapperFor) override def hasNext: Boolean = { + if (completed) { + return false + } try { if (scriptOutputWritable == null) { scriptOutputWritable = reusedWritableObject @@ -85,6 +88,7 @@ private[hive] case class HiveScriptTransformationExec( if (scriptOutputReader != null) { if (scriptOutputReader.next(scriptOutputWritable) <= 0) { checkFailureAndPropagate(writerThread, null, proc, stderrBuffer) + completed = true return false } } else { @@ -97,6 +101,7 @@ private[hive] case class HiveScriptTransformationExec( // there can be a lag between EOF being written out and the process // being terminated. So explicitly waiting for the process to be done. checkFailureAndPropagate(writerThread, null, proc, stderrBuffer) + completed = true return false } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala index 1a6f6843d3911..af3d4555bc5cf 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala @@ -363,4 +363,51 @@ class DataSourceWithHiveMetastoreCatalogSuite } }) } + + Seq( + "parquet" -> ( + "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe", + HiveUtils.CONVERT_METASTORE_PARQUET.key), + "orc" -> ( + "org.apache.hadoop.hive.ql.io.orc.OrcSerde", + HiveUtils.CONVERT_METASTORE_ORC.key) + ).foreach { case (format, (serde, formatConvertConf)) => + test("SPARK-28266: convertToLogicalRelation should not interpret `path` property when " + + s"reading Hive tables using $format file format") { + withTempPath(dir => { + val baseDir = dir.getAbsolutePath + withSQLConf(formatConvertConf -> "true") { + + withTable("t1") { + hiveClient.runSqlHive( + s""" + |CREATE TABLE t1 (id bigint) + |ROW FORMAT SERDE '$serde' + |WITH SERDEPROPERTIES ('path'='someNonLocationValue') + |STORED AS $format LOCATION '$baseDir' + |""".stripMargin) + + assertResult(0) { + spark.sql("SELECT * FROM t1").count() + } + } + + spark.range(3).selectExpr("id").write.format(format).save(baseDir) + withTable("t2") { + hiveClient.runSqlHive( + s""" + |CREATE TABLE t2 (id bigint) + |ROW FORMAT SERDE '$serde' + |WITH SERDEPROPERTIES ('path'='$baseDir') + |STORED AS $format LOCATION '$baseDir' + |""".stripMargin) + + assertResult(3) { + spark.sql("SELECT * FROM t2").count() + } + } + } + }) + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala index df96b0675cc2d..10d5d01bf9728 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala @@ -123,4 +123,28 @@ class HiveParquetSuite extends QueryTest with ParquetTest with TestHiveSingleton assert(msg.contains("cannot resolve '`c3`' given input columns")) } } + + test("SPARK-37098: Alter table properties should invalidate cache") { + // specify the compression in case we change it in future + withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> "snappy") { + withTempPath { dir => + withTable("t") { + sql(s"CREATE TABLE t (c int) STORED AS PARQUET LOCATION '${dir.getCanonicalPath}'") + // cache table metadata + sql("SELECT * FROM t") + sql("ALTER TABLE t SET TBLPROPERTIES('parquet.compression'='gzip')") + sql("INSERT INTO TABLE t values(1)") + val files1 = dir.listFiles().filter(_.getName.endsWith("gz.parquet")) + assert(files1.length == 1) + + // cache table metadata again + sql("SELECT * FROM t") + sql("ALTER TABLE t UNSET TBLPROPERTIES('parquet.compression')") + sql("INSERT INTO TABLE t values(1)") + val files2 = dir.listFiles().filter(_.getName.endsWith("snappy.parquet")) + assert(files2.length == 1) + } + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala index 12ed0e5305299..79b34bd141de3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala @@ -179,5 +179,28 @@ class FiltersSuite extends SparkFunSuite with Logging with PlanTest { } } + test("SPARK-34515: Fix NPE if InSet contains null value during getPartitionsByFilter") { + withSQLConf(SQLConf.HIVE_METASTORE_PARTITION_PRUNING_INSET_THRESHOLD.key -> "2") { + val filter = InSet(a("p", IntegerType), Set(null, 1, 2)) + val converted = shim.convertFilters(testTable, Seq(filter), conf.sessionLocalTimeZone) + assert(converted == "(p >= 1 and p <= 2)") + } + } + + test("SPARK-34538: Skip InSet null value during push filter to Hive metastore") { + withSQLConf(SQLConf.HIVE_METASTORE_PARTITION_PRUNING_INSET_THRESHOLD.key -> "3") { + val intFilter = InSet(a("p", IntegerType), Set(null, 1, 2)) + val intConverted = shim.convertFilters(testTable, Seq(intFilter), conf.sessionLocalTimeZone) + assert(intConverted == "(p = 1 or p = 2)") + } + + withSQLConf(SQLConf.HIVE_METASTORE_PARTITION_PRUNING_INSET_THRESHOLD.key -> "3") { + val dateFilter = InSet(a("p", DateType), Set(null, + Literal(Date.valueOf("2020-01-01")).eval(), Literal(Date.valueOf("2021-01-01")).eval())) + val dateConverted = shim.convertFilters(testTable, Seq(dateFilter), conf.sessionLocalTimeZone) + assert(dateConverted == "(p = 2020-01-01 or p = 2021-01-01)") + } + } + private def a(name: String, dataType: DataType) = AttributeReference(name, dataType)() } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSQLViewSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSQLViewSuite.scala index 8aae7a1545b1a..feb2c6765d352 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSQLViewSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSQLViewSuite.scala @@ -19,8 +19,9 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.{AnalysisException, Row} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, HiveTableRelation} import org.apache.spark.sql.execution.SQLViewSuite +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.types.{NullType, StructType} @@ -157,4 +158,25 @@ class HiveSQLViewSuite extends SQLViewSuite with TestHiveSingleton { ) } } + + test("SPARK-35792: ignore optimization configs used in RelationConversions") { + withTable("t_orc") { + withView("v_orc") { + withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> "true") { + spark.sql("create table t_orc stored as orc as select 1 as a, 2 as b") + spark.sql("create view v_orc as select * from t_orc") + } + withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> "false") { + val relationInTable = sql("select * from t_orc").queryExecution.analyzed.collect { + case r: HiveTableRelation => r + }.headOption + val relationInView = sql("select * from v_orc").queryExecution.analyzed.collect { + case r: HiveTableRelation => r + }.headOption + assert(relationInTable.isDefined) + assert(relationInView.isDefined) + } + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala index 1018ae5b68895..9982dec7c1c60 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala @@ -23,6 +23,7 @@ import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.scalatest.exceptions.TestFailedException import org.apache.spark.{SparkException, TestUtils} +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression} import org.apache.spark.sql.execution._ import org.apache.spark.sql.functions._ @@ -438,4 +439,21 @@ class HiveScriptTransformationSuite extends BaseScriptTransformationSuite with T assert(e2.contains("array cannot be converted to Hive TypeInfo")) } } + + test("SPARK-38075: ORDER BY with LIMIT should not add fake rows") { + withTempView("v") { + val df = Seq((1), (2), (3)).toDF("a") + df.createTempView("v") + checkAnswer(sql( + """ + |SELECT TRANSFORM(a) + | USING 'cat' AS (a) + |FROM v + |ORDER BY a + |LIMIT 10 + |""".stripMargin), + identity, + Row("1") :: Row("2") :: Row("3") :: Nil) + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala index ab37645b1c90c..74b6dd3063381 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, HadoopFsRelation, LogicalRelation, PruneFileSourcePartitions} import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec import org.apache.spark.sql.functions.broadcast import org.apache.spark.sql.internal.SQLConf @@ -109,9 +110,26 @@ class PruneFileSourcePartitionsSuite extends PrunePartitionSuiteBase { } } + test("SPARK-35985 push filters for empty read schema") { + // Force datasource v2 for parquet + withSQLConf((SQLConf.USE_V1_SOURCE_LIST.key, "")) { + withTempPath { dir => + spark.range(10).selectExpr("id", "id % 3 as p") + .write.partitionBy("p").parquet(dir.getCanonicalPath) + withTempView("tmp") { + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("tmp"); + assertPrunedPartitions("SELECT COUNT(*) FROM tmp WHERE p = 0", 1, "(tmp.`p` = 0)") + assertPrunedPartitions( + "SELECT input_file_name() FROM tmp WHERE p = 0", 1, "(tmp.`p` = 0)") + } + } + } + } + override def getScanExecPartitionSize(plan: SparkPlan): Long = { plan.collectFirst { - case p: FileSourceScanExec => p - }.get.selectedPartitions.length + case p: FileSourceScanExec => p.selectedPartitions.length + case b: BatchScanExec => b.partitions.size + }.get } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PrunePartitionSuiteBase.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PrunePartitionSuiteBase.scala index bc170fcd59026..47cc3609ba7af 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PrunePartitionSuiteBase.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PrunePartitionSuiteBase.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.expressions.{AttributeReference, BinaryOperator, Expression, IsNotNull, Literal} import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} +import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, FileScan} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf.ADAPTIVE_EXECUTION_ENABLED import org.apache.spark.sql.test.SQLTestUtils @@ -95,9 +96,10 @@ abstract class PrunePartitionSuiteBase extends QueryTest with SQLTestUtils with val plan = qe.sparkPlan assert(getScanExecPartitionSize(plan) == expectedPartitionCount) - val pushedDownPartitionFilters = qe.executedPlan.collectFirst { + val pushedDownPartitionFilters = plan.collectFirst { case scan: FileSourceScanExec => scan.partitionFilters case scan: HiveTableScanExec => scan.partitionPruningPred + case BatchScanExec(_, scan: FileScan) => scan.partitionFilters }.map(exps => exps.filterNot(e => e.isInstanceOf[IsNotNull])) val pushedFilters = pushedDownPartitionFilters.map(filters => { filters.foldLeft("")((currentStr, exp) => { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala index 4d6dafd598a2e..a2de43d737704 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala @@ -18,7 +18,9 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite +import org.apache.spark.sql.execution.command.DataWritingCommandExec import org.apache.spark.sql.execution.metric.SQLMetricsTestUtils +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.test.TestHiveSingleton // Disable AQE because metric info is different with AQE on/off @@ -34,4 +36,29 @@ class SQLMetricsSuite extends SQLMetricsTestUtils with TestHiveSingleton testMetricsDynamicPartition("hive", "hive", "t1") } } + + test("SPARK-34567: Add metrics for CTAS operator") { + Seq(false, true).foreach { canOptimized => + withSQLConf(HiveUtils.CONVERT_METASTORE_CTAS.key -> canOptimized.toString) { + withTable("t") { + val df = sql(s"CREATE TABLE t STORED AS PARQUET AS SELECT 1 as a") + val dataWritingCommandExec = + df.queryExecution.executedPlan.asInstanceOf[DataWritingCommandExec] + dataWritingCommandExec.executeCollect() + val createTableAsSelect = dataWritingCommandExec.cmd + if (canOptimized) { + assert(createTableAsSelect.isInstanceOf[OptimizedCreateHiveTableAsSelectCommand]) + } else { + assert(createTableAsSelect.isInstanceOf[CreateHiveTableAsSelectCommand]) + } + assert(createTableAsSelect.metrics.contains("numFiles")) + assert(createTableAsSelect.metrics("numFiles").value == 1) + assert(createTableAsSelect.metrics.contains("numOutputBytes")) + assert(createTableAsSelect.metrics("numOutputBytes").value > 0) + assert(createTableAsSelect.metrics.contains("numOutputRows")) + assert(createTableAsSelect.metrics("numOutputRows").value == 1) + } + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index aaa77ae2a92bf..3d4357cabb4f7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -2584,6 +2584,23 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi } } + test("SPARK-37196: HiveDecimal Precision Scale match failed should return null") { + withTempDir { dir => + withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> "false") { + withTable("test_precision") { + val df = sql(s"SELECT 'dummy' AS name, ${"1" * 20}.${"2" * 18} AS value") + df.write.mode("Overwrite").parquet(dir.getAbsolutePath) + sql( + s""" + |CREATE EXTERNAL TABLE test_precision(name STRING, value DECIMAL(18,6)) + |STORED AS PARQUET LOCATION '${dir.getAbsolutePath}' + |""".stripMargin) + checkAnswer(sql("SELECT * FROM test_precision"), Row("dummy", null)) + } + } + } + } + test("SPARK-11374 Support skip.header.line.count option in Hive table") { withTable("skip_table_0", "skip_table_1", "skip_table_2", "skip_table_3", "skip_table_4") { withTempDir { dir =>