From 3afebac383a29bd0dfa3e04650810c1e6d79d3bf Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 13 Mar 2023 11:32:38 +0100 Subject: [PATCH 1/9] Get bytes from ByteBuffer safely `ByteBuffer.array()` is a convenient and efficient way to get bytes from a `ByteBuffer`. It has, however, numerous preconditions that should be checked before using the returned array. The commit replaces all `ByteBuffer.array()` usages where these preconditions are assumed to be true. --- .../io/trino/plugin/base/io/ByteBuffers.java | 55 +++++++++++++++++++ .../trino/plugin/base/io/TestByteBuffers.java | 35 ++++++++++++ .../io/trino/plugin/iceberg/IcebergTypes.java | 3 +- .../io/trino/plugin/iceberg/IcebergUtil.java | 3 +- .../IcebergThetaSketchForStats.java | 15 +---- .../plugin/iceberg/delete/DeleteFile.java | 5 +- .../kafka/encoder/json/JsonRowEncoder.java | 3 +- .../kafka/encoder/raw/RawRowEncoder.java | 3 +- .../tests/product/cassandra/TestSelect.java | 5 +- 9 files changed, 104 insertions(+), 23 deletions(-) create mode 100644 lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/io/ByteBuffers.java create mode 100644 lib/trino-plugin-toolkit/src/test/java/io/trino/plugin/base/io/TestByteBuffers.java diff --git a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/io/ByteBuffers.java b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/io/ByteBuffers.java new file mode 100644 index 000000000000..d91fc5db8aea --- /dev/null +++ b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/io/ByteBuffers.java @@ -0,0 +1,55 @@ +/* + * Licensed 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 io.trino.plugin.base.io; + +import java.nio.ByteBuffer; + +import static com.google.common.base.Preconditions.checkArgument; + +public final class ByteBuffers +{ + private ByteBuffers() {} + + /** + * Gets the bytes the provided {@link ByteBuffer} wraps, without advancing buffer position. + * Throws when provided buffer does not directly wrap bytes. + */ + public static byte[] getWrappedBytes(ByteBuffer byteBuffer) + { + checkArgument(byteBuffer.hasArray(), "buffer does not have array"); + checkArgument(byteBuffer.arrayOffset() == 0, "buffer has non-zero array offset: %s", byteBuffer.arrayOffset()); + checkArgument(byteBuffer.position() == 0, "buffer has been repositioned to %s", byteBuffer.position()); + byte[] array = byteBuffer.array(); + checkArgument(byteBuffer.remaining() == array.length, "buffer has %s remaining bytes while array length is %s", byteBuffer.remaining(), array.length); + return array; + } + + /** + * Gets the bytes the provided {@link ByteBuffer} represents, without advancing buffer position. + * The returned byte array may be shared with the buffer. + */ + public static byte[] getBytes(ByteBuffer byteBuffer) + { + if (byteBuffer.hasArray() && byteBuffer.arrayOffset() == 0 && byteBuffer.position() == 0) { + byte[] array = byteBuffer.array(); + if (byteBuffer.remaining() == array.length) { + return array; + } + } + + byte[] bytes = new byte[byteBuffer.remaining()]; + byteBuffer.asReadOnlyBuffer().get(bytes); + return bytes; + } +} diff --git a/lib/trino-plugin-toolkit/src/test/java/io/trino/plugin/base/io/TestByteBuffers.java b/lib/trino-plugin-toolkit/src/test/java/io/trino/plugin/base/io/TestByteBuffers.java new file mode 100644 index 000000000000..50da4844039a --- /dev/null +++ b/lib/trino-plugin-toolkit/src/test/java/io/trino/plugin/base/io/TestByteBuffers.java @@ -0,0 +1,35 @@ +/* + * Licensed 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 io.trino.plugin.base.io; + +import org.testng.annotations.Test; + +import java.nio.ByteBuffer; + +import static io.trino.plugin.base.io.ByteBuffers.getWrappedBytes; +import static org.testng.Assert.assertEquals; + +public class TestByteBuffers +{ + @Test + public void testGetWrappedBytes() + { + ByteBuffer buffer = ByteBuffer.wrap(new byte[] {0, 1, 2, 3}); + assertEquals(getWrappedBytes(buffer), new byte[] {0, 1, 2, 3}, "getWrappedBytes"); + + // Assert the buffer position hasn't changed + assertEquals(buffer.position(), 0, "position"); + assertEquals(getWrappedBytes(buffer), new byte[] {0, 1, 2, 3}, "getWrappedBytes again"); + } +} diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergTypes.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergTypes.java index 7b8ef16b6a86..76d837dc4937 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergTypes.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergTypes.java @@ -32,6 +32,7 @@ import java.util.UUID; import static io.airlift.slice.Slices.utf8Slice; +import static io.trino.plugin.base.io.ByteBuffers.getWrappedBytes; import static io.trino.plugin.iceberg.util.Timestamps.timestampTzFromMicros; import static io.trino.plugin.iceberg.util.Timestamps.timestampTzToMicros; import static io.trino.spi.type.BigintType.BIGINT; @@ -169,7 +170,7 @@ public static Object convertIcebergValueToTrino(Type icebergType, Object value) return utf8Slice(((String) value)); } if (icebergType instanceof Types.BinaryType) { - return Slices.wrappedBuffer(((ByteBuffer) value).array().clone()); + return Slices.wrappedBuffer(getWrappedBytes((ByteBuffer) value).clone()); } if (icebergType instanceof Types.DateType) { return (long) (int) value; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergUtil.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergUtil.java index 22fdfb603f09..8d1595643fc3 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergUtil.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergUtil.java @@ -90,6 +90,7 @@ import static com.google.common.collect.ImmutableMap.toImmutableMap; import static com.google.common.collect.ImmutableSet.toImmutableSet; import static io.airlift.slice.Slices.utf8Slice; +import static io.trino.plugin.base.io.ByteBuffers.getWrappedBytes; import static io.trino.plugin.hive.HiveMetadata.TABLE_COMMENT; import static io.trino.plugin.iceberg.ColumnIdentity.createColumnIdentity; import static io.trino.plugin.iceberg.IcebergErrorCode.ICEBERG_BAD_DATA; @@ -543,7 +544,7 @@ public static Map> getPartitionKeys(StructLike partiti String partitionValue; if (type.typeId() == FIXED || type.typeId() == BINARY) { // this is safe because Iceberg PartitionData directly wraps the byte array - partitionValue = Base64.getEncoder().encodeToString(((ByteBuffer) value).array()); + partitionValue = Base64.getEncoder().encodeToString(getWrappedBytes(((ByteBuffer) value))); } else { partitionValue = value.toString(); diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/aggregation/IcebergThetaSketchForStats.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/aggregation/IcebergThetaSketchForStats.java index 1fd368b3c575..e7d4bf6b39f9 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/aggregation/IcebergThetaSketchForStats.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/aggregation/IcebergThetaSketchForStats.java @@ -39,6 +39,7 @@ import java.util.concurrent.atomic.AtomicInteger; import static com.google.common.base.Verify.verify; +import static io.trino.plugin.base.io.ByteBuffers.getBytes; import static io.trino.plugin.iceberg.IcebergTypes.convertTrinoValueToIceberg; import static io.trino.plugin.iceberg.TypeConverter.toIcebergTypeForNewColumn; import static io.trino.spi.type.TypeUtils.readNativeValue; @@ -108,18 +109,4 @@ private static void addIfPresent(Union union, @Nullable Sketch input) union.union(input); } } - - private static byte[] getBytes(ByteBuffer byteBuffer) - { - int length = byteBuffer.remaining(); - if (byteBuffer.hasArray() && byteBuffer.arrayOffset() == 0) { - byte[] bytes = byteBuffer.array(); - if (bytes.length == length) { - return bytes; - } - } - byte[] bytes = new byte[length]; - byteBuffer.get(bytes); - return bytes; - } } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/delete/DeleteFile.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/delete/DeleteFile.java index 6a49411f93e3..17fdcad61e10 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/delete/DeleteFile.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/delete/DeleteFile.java @@ -32,6 +32,7 @@ import static io.airlift.slice.SizeOf.SIZE_OF_INT; import static io.airlift.slice.SizeOf.estimatedSizeOf; import static io.airlift.slice.SizeOf.instanceSize; +import static io.trino.plugin.base.io.ByteBuffers.getWrappedBytes; import static java.util.Objects.requireNonNull; public final class DeleteFile @@ -50,9 +51,9 @@ public final class DeleteFile public static DeleteFile fromIceberg(org.apache.iceberg.DeleteFile deleteFile) { Map lowerBounds = firstNonNull(deleteFile.lowerBounds(), ImmutableMap.of()) - .entrySet().stream().collect(toImmutableMap(Map.Entry::getKey, entry -> entry.getValue().array().clone())); + .entrySet().stream().collect(toImmutableMap(Map.Entry::getKey, entry -> getWrappedBytes(entry.getValue()).clone())); Map upperBounds = firstNonNull(deleteFile.upperBounds(), ImmutableMap.of()) - .entrySet().stream().collect(toImmutableMap(Map.Entry::getKey, entry -> entry.getValue().array().clone())); + .entrySet().stream().collect(toImmutableMap(Map.Entry::getKey, entry -> getWrappedBytes(entry.getValue()).clone())); return new DeleteFile( deleteFile.content(), diff --git a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/encoder/json/JsonRowEncoder.java b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/encoder/json/JsonRowEncoder.java index 55284bf741dd..2fa3008325b3 100644 --- a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/encoder/json/JsonRowEncoder.java +++ b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/encoder/json/JsonRowEncoder.java @@ -39,6 +39,7 @@ import java.util.Set; import static com.google.common.base.Preconditions.checkArgument; +import static io.trino.plugin.base.io.ByteBuffers.getWrappedBytes; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.BooleanType.BOOLEAN; import static io.trino.spi.type.DateType.DATE; @@ -188,7 +189,7 @@ protected void appendString(String value) @Override protected void appendByteBuffer(ByteBuffer value) { - node.put(currentColumnMapping(), value.array()); + node.put(currentColumnMapping(), getWrappedBytes(value)); } @Override diff --git a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/encoder/raw/RawRowEncoder.java b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/encoder/raw/RawRowEncoder.java index e515c7c1168f..48cdd5c2702c 100644 --- a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/encoder/raw/RawRowEncoder.java +++ b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/encoder/raw/RawRowEncoder.java @@ -33,6 +33,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.plugin.base.io.ByteBuffers.getWrappedBytes; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.BooleanType.BOOLEAN; import static io.trino.spi.type.DoubleType.DOUBLE; @@ -307,7 +308,7 @@ protected void appendString(String value) @Override protected void appendByteBuffer(ByteBuffer value) { - byte[] valueBytes = value.array(); + byte[] valueBytes = getWrappedBytes(value); checkArgument(valueBytes.length == columnMappings.get(currentColumnIndex).getLength(), format( "length '%s' of message for column '%s' does not equal expected length '%s'", valueBytes.length, diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/cassandra/TestSelect.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/cassandra/TestSelect.java index 837833699ca7..f0e22e5a5287 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/cassandra/TestSelect.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/cassandra/TestSelect.java @@ -31,7 +31,6 @@ import java.time.ZoneOffset; import java.util.function.Consumer; -import static com.datastax.oss.driver.api.core.data.ByteUtils.fromHexString; import static io.trino.tempto.Requirements.compose; import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertThat; @@ -191,7 +190,7 @@ public void testAllDataTypes() .containsOnly( row("\0", Long.MIN_VALUE, - fromHexString("0x00").array(), + new byte[] {0}, false, 0f, Double.MIN_VALUE, @@ -305,7 +304,7 @@ public void testSelectAllTypePartitioningMaterializedView() .containsOnly( row("\0", Long.MIN_VALUE, - fromHexString("0x00").array(), + new byte[] {0}, false, 0f, Double.MIN_VALUE, From caa5ef38b95cd78282262bea4e743eda8962930a Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 13 Mar 2023 14:53:52 +0100 Subject: [PATCH 2/9] Improve assertions in testCleaningUpWithTableWithSpecifiedLocation - prefer `assertThat(List).hasSize` to `assertThat(list.size())` as the former includes list's elements upon failure, - drop table after test. --- .../iceberg/BaseIcebergConnectorTest.java | 25 +++++++++++-------- 1 file changed, 15 insertions(+), 10 deletions(-) diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java index a2e8666c2e94..9238effff8f9 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java @@ -5701,26 +5701,31 @@ public void testCleaningUpWithTableWithSpecifiedLocation() private void testCleaningUpWithTableWithSpecifiedLocation(String suffix) throws IOException { - File tempDir = getDistributedQueryRunner().getCoordinator().getBaseDataDir().toFile(); - String tempDirPath = tempDir.toURI().toASCIIString() + randomNameSuffix() + suffix; String tableName = "test_table_cleaning_up_with_location" + randomNameSuffix(); + String tableLocation = getDistributedQueryRunner().getCoordinator().getBaseDataDir().toUri().toASCIIString() + randomNameSuffix() + suffix; - assertUpdate(format("CREATE TABLE %s (key varchar, value integer) WITH(location = '%s')", tableName, tempDirPath)); + assertUpdate(format("CREATE TABLE %s (key varchar, value integer) WITH(location = '%s')", tableName, tableLocation)); assertUpdate("INSERT INTO " + tableName + " VALUES ('one', 1)", 1); assertUpdate("INSERT INTO " + tableName + " VALUES ('two', 2)", 1); - List initialFiles = getAllMetadataFilesFromTableDirectory(tempDirPath); + List initialMetadataFiles = getAllMetadataFilesFromTableDirectory(tableLocation); List initialSnapshots = getSnapshotIds(tableName); + assertThat(initialSnapshots).as("initialSnapshots") + .hasSize(3); // CREATE TABLE creates a snapshot Session sessionWithShortRetentionUnlocked = prepareCleanUpSession(); assertQuerySucceeds(sessionWithShortRetentionUnlocked, "ALTER TABLE " + tableName + " EXECUTE EXPIRE_SNAPSHOTS (retention_threshold => '0s')"); assertQuerySucceeds(sessionWithShortRetentionUnlocked, "ALTER TABLE " + tableName + " EXECUTE REMOVE_ORPHAN_FILES (retention_threshold => '0s')"); - List updatedFiles = getAllMetadataFilesFromTableDirectory(tempDirPath); - List updatedSnapshots = getSnapshotIds(tableName); - assertThat(updatedFiles.size()).isEqualTo(initialFiles.size() - 1); - assertThat(updatedSnapshots.size()).isLessThan(initialSnapshots.size()); - assertThat(updatedSnapshots.size()).isEqualTo(1); - assertThat(initialSnapshots).containsAll(updatedSnapshots); + List prunedMetadataFiles = getAllMetadataFilesFromTableDirectory(tableLocation); + List prunedSnapshots = getSnapshotIds(tableName); + assertThat(prunedMetadataFiles).as("prunedMetadataFiles") + .hasSize(initialMetadataFiles.size() - 1); + assertThat(prunedSnapshots).as("prunedSnapshots") + .hasSizeLessThan(initialSnapshots.size()) + .hasSize(1); + assertThat(initialSnapshots).containsAll(prunedSnapshots); + + assertUpdate("DROP TABLE " + tableName); } @Test From 114853e5187147b8337bfc2d1cf31375c8fc6b8a Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 13 Mar 2023 15:59:46 +0100 Subject: [PATCH 3/9] Fix metadata file listing in testCleaningUpWithTableWithSpecifiedLocation Before the change, the test used `getAllMetadataFilesFromTableDirectory` utility, but it was listing all (metadata and data) files. Similar method, `getAllMetadataFilesFromTableDirectoryForTable`, listing only metadata files existed, but was not used in the test. The change merges the utility methods: the correct logic comes from `getAllMetadataFilesFromTableDirectoryForTable` (so no behavior change for tests other than `testCleaningUpWithTableWithSpecifiedLocation`), and the name comes from `getAllMetadataFilesFromTableDirectory`. --- .../iceberg/BaseIcebergConnectorTest.java | 21 +++++++------------ 1 file changed, 8 insertions(+), 13 deletions(-) diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java index 9238effff8f9..36fc6c5ff87a 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java @@ -5536,12 +5536,12 @@ public void testExpireSnapshots() List initialSnapshots = getSnapshotIds(tableName); String tableLocation = getTableLocation(tableName); - List initialFiles = getAllMetadataFilesFromTableDirectoryForTable(tableLocation); + List initialFiles = getAllMetadataFilesFromTableDirectory(tableLocation); assertQuerySucceeds(sessionWithShortRetentionUnlocked, "ALTER TABLE " + tableName + " EXECUTE EXPIRE_SNAPSHOTS (retention_threshold => '0s')"); assertThat(query("SELECT sum(value), listagg(key, ' ') WITHIN GROUP (ORDER BY key) FROM " + tableName)) .matches("VALUES (BIGINT '3', VARCHAR 'one two')"); - List updatedFiles = getAllMetadataFilesFromTableDirectoryForTable(tableLocation); + List updatedFiles = getAllMetadataFilesFromTableDirectory(tableLocation); List updatedSnapshots = getSnapshotIds(tableName); assertThat(updatedFiles.size()).isEqualTo(initialFiles.size() - 1); assertThat(updatedSnapshots.size()).isLessThan(initialSnapshots.size()); @@ -5679,11 +5679,11 @@ public void testIfRemoveOrphanFilesCleansUnnecessaryMetadataFilesInPartitionedTa assertUpdate("INSERT INTO " + tableName + " VALUES ('two', 2)", 1); String tableLocation = getTableLocation(tableName); Path orphanMetadataFile = Files.createFile(Path.of(getIcebergTableMetadataPath(tableLocation).toString(), "invalidData." + format)); - List initialMetadataFiles = getAllMetadataFilesFromTableDirectoryForTable(tableLocation); + List initialMetadataFiles = getAllMetadataFilesFromTableDirectory(tableLocation); assertQuerySucceeds(sessionWithShortRetentionUnlocked, "ALTER TABLE " + tableName + " EXECUTE REMOVE_ORPHAN_FILES (retention_threshold => '0s')"); - List updatedMetadataFiles = getAllMetadataFilesFromTableDirectoryForTable(tableLocation); + List updatedMetadataFiles = getAllMetadataFilesFromTableDirectory(tableLocation); assertThat(updatedMetadataFiles.size()).isLessThan(initialMetadataFiles.size()); assertThat(updatedMetadataFiles).doesNotContain(orphanMetadataFile.toString()); } @@ -5703,12 +5703,13 @@ private void testCleaningUpWithTableWithSpecifiedLocation(String suffix) { String tableName = "test_table_cleaning_up_with_location" + randomNameSuffix(); String tableLocation = getDistributedQueryRunner().getCoordinator().getBaseDataDir().toUri().toASCIIString() + randomNameSuffix() + suffix; + String tableDirectory = new File(URI.create(tableLocation)).getPath(); // validates this is file:// URI and normalizes assertUpdate(format("CREATE TABLE %s (key varchar, value integer) WITH(location = '%s')", tableName, tableLocation)); assertUpdate("INSERT INTO " + tableName + " VALUES ('one', 1)", 1); assertUpdate("INSERT INTO " + tableName + " VALUES ('two', 2)", 1); - List initialMetadataFiles = getAllMetadataFilesFromTableDirectory(tableLocation); + List initialMetadataFiles = getAllMetadataFilesFromTableDirectory(tableDirectory); List initialSnapshots = getSnapshotIds(tableName); assertThat(initialSnapshots).as("initialSnapshots") .hasSize(3); // CREATE TABLE creates a snapshot @@ -5716,7 +5717,7 @@ private void testCleaningUpWithTableWithSpecifiedLocation(String suffix) Session sessionWithShortRetentionUnlocked = prepareCleanUpSession(); assertQuerySucceeds(sessionWithShortRetentionUnlocked, "ALTER TABLE " + tableName + " EXECUTE EXPIRE_SNAPSHOTS (retention_threshold => '0s')"); assertQuerySucceeds(sessionWithShortRetentionUnlocked, "ALTER TABLE " + tableName + " EXECUTE REMOVE_ORPHAN_FILES (retention_threshold => '0s')"); - List prunedMetadataFiles = getAllMetadataFilesFromTableDirectory(tableLocation); + List prunedMetadataFiles = getAllMetadataFilesFromTableDirectory(tableDirectory); List prunedSnapshots = getSnapshotIds(tableName); assertThat(prunedMetadataFiles).as("prunedMetadataFiles") .hasSize(initialMetadataFiles.size() - 1); @@ -6707,18 +6708,12 @@ private Session prepareCleanUpSession() .build(); } - private List getAllMetadataFilesFromTableDirectoryForTable(String tableLocation) + private List getAllMetadataFilesFromTableDirectory(String tableLocation) throws IOException { return listAllTableFilesInDirectory(getIcebergTableMetadataPath(tableLocation)); } - private List getAllMetadataFilesFromTableDirectory(String tableDataDir) - throws IOException - { - return listAllTableFilesInDirectory(Path.of(URI.create(tableDataDir).getPath())); - } - private List listAllTableFilesInDirectory(Path tableDataPath) throws IOException { From c7987294b7da2b58932f68220c6c94f9d1de08a9 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 16 Dec 2022 17:06:30 +0100 Subject: [PATCH 4/9] Rename test class to contain more scenarios Iceberg connector will collect stats during writes, and it would be good to test this together with ANALYZE. Rename class to contain all related functionalities. --- .../java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java | 2 +- .../{TestIcebergAnalyze.java => TestIcebergStatistics.java} | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) rename plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/{TestIcebergAnalyze.java => TestIcebergStatistics.java} (99%) diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java index 36fc6c5ff87a..f298b57df350 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java @@ -3449,7 +3449,7 @@ public void testBasicTableStatistics() } /** - * @see TestIcebergAnalyze + * @see TestIcebergStatistics */ @Test public void testBasicAnalyze() diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergAnalyze.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergStatistics.java similarity index 99% rename from plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergAnalyze.java rename to plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergStatistics.java index 85c269caf0f9..5bd25797dd6d 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergAnalyze.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergStatistics.java @@ -28,7 +28,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotEquals; -public class TestIcebergAnalyze +public class TestIcebergStatistics extends AbstractTestQueryFramework { @Override From 047c1ed49999f7ab0ab54ac0b1c16aa2ef9eb605 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 15 Mar 2023 09:51:21 +0100 Subject: [PATCH 5/9] Pull schema access out of loop --- .../src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java index dabfbb861f9c..74faa3aca81b 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java @@ -851,9 +851,10 @@ public Optional finishInsert(ConnectorSession session, IcebergWritableTableHandle table = (IcebergWritableTableHandle) insertHandle; Table icebergTable = transaction.table(); + Schema schema = icebergTable.schema(); Type[] partitionColumnTypes = icebergTable.spec().fields().stream() .map(field -> field.transform().getResultType( - icebergTable.schema().findType(field.sourceId()))) + schema.findType(field.sourceId()))) .toArray(Type[]::new); AppendFiles appendFiles = isMergeManifestsOnWrite(session) ? transaction.newAppend() : transaction.newFastAppend(); From adaffe64933bb0333145d145cf2f6b7bc9e10355 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 15 Mar 2023 10:52:54 +0100 Subject: [PATCH 6/9] Adjust class name to follow convention All Iceberg connector test classes match `TestIceberg*ConnectorTest` pattern. --- ...ava => TestIcebergTrinoRestCatalogConnectorSmokeTest.java} | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) rename plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/rest/{TestIcebergRestCatalogConnectorSmokeTest.java => TestIcebergTrinoRestCatalogConnectorSmokeTest.java} (98%) diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/rest/TestIcebergRestCatalogConnectorSmokeTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/rest/TestIcebergTrinoRestCatalogConnectorSmokeTest.java similarity index 98% rename from plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/rest/TestIcebergRestCatalogConnectorSmokeTest.java rename to plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/rest/TestIcebergTrinoRestCatalogConnectorSmokeTest.java index 42fbe4cff641..adc285b34e12 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/rest/TestIcebergRestCatalogConnectorSmokeTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/rest/TestIcebergTrinoRestCatalogConnectorSmokeTest.java @@ -35,12 +35,12 @@ import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThatThrownBy; -public class TestIcebergRestCatalogConnectorSmokeTest +public class TestIcebergTrinoRestCatalogConnectorSmokeTest extends BaseIcebergConnectorSmokeTest { private File warehouseLocation; - public TestIcebergRestCatalogConnectorSmokeTest() + public TestIcebergTrinoRestCatalogConnectorSmokeTest() { super(new IcebergConfig().getFileFormat().toIceberg()); } From b502bd95ce97fbc84dbf63416ba2c5e53d97abd3 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 15 Mar 2023 10:58:58 +0100 Subject: [PATCH 7/9] Pick nicer test values --- .../scalar/TestDateTimeFunctions.java | 4 ++-- .../iceberg/BaseIcebergConnectorTest.java | 20 +++++++++---------- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/core/trino-main/src/test/java/io/trino/operator/scalar/TestDateTimeFunctions.java b/core/trino-main/src/test/java/io/trino/operator/scalar/TestDateTimeFunctions.java index 74ddc3a46910..e8a94ac6a3ea 100644 --- a/core/trino-main/src/test/java/io/trino/operator/scalar/TestDateTimeFunctions.java +++ b/core/trino-main/src/test/java/io/trino/operator/scalar/TestDateTimeFunctions.java @@ -223,8 +223,8 @@ public void testFromUnixTimeWithTimeZone() assertThat(assertions.function("from_unixtime", "7200", "'Asia/Tokyo'")) .matches("TIMESTAMP '1970-01-01 11:00:00.000 Asia/Tokyo'"); - assertThat(assertions.function("from_unixtime", "7200", "'Europe/Moscow'")) - .matches("TIMESTAMP '1970-01-01 05:00:00.000 Europe/Moscow'"); + assertThat(assertions.function("from_unixtime", "7200", "'Europe/Kiev'")) + .matches("TIMESTAMP '1970-01-01 05:00:00.000 Europe/Kiev'"); assertThat(assertions.function("from_unixtime", "7200", "'America/New_York'")) .matches("TIMESTAMP '1969-12-31 21:00:00.000 America/New_York'"); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java index f298b57df350..335102691035 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java @@ -2925,22 +2925,22 @@ public void testTruncateTextTransform() "('abcd', 1)," + "('abxy', 2)," + "('ab598', 3)," + - "('mommy', 4)," + - "('moscow', 5)," + + "('Kielce', 4)," + + "('Kiev', 5)," + "('Greece', 6)," + "('Grozny', 7)", 8); - assertQuery("SELECT partition.d_trunc FROM \"test_truncate_text_transform$partitions\"", "VALUES NULL, 'ab', 'mo', 'Gr'"); + assertQuery("SELECT partition.d_trunc FROM \"test_truncate_text_transform$partitions\"", "VALUES NULL, 'ab', 'Ki', 'Gr'"); assertQuery("SELECT b FROM test_truncate_text_transform WHERE substring(d, 1, 2) = 'ab'", "VALUES 1, 2, 3"); assertQuery( select + " WHERE partition.d_trunc = 'ab'", format == AVRO ? "VALUES ('ab', 3, NULL, NULL, NULL, NULL)" : "VALUES ('ab', 3, 'ab598', 'abxy', 1, 3)"); - assertQuery("SELECT b FROM test_truncate_text_transform WHERE substring(d, 1, 2) = 'mo'", "VALUES 4, 5"); + assertQuery("SELECT b FROM test_truncate_text_transform WHERE substring(d, 1, 2) = 'Ki'", "VALUES 4, 5"); assertQuery( - select + " WHERE partition.d_trunc = 'mo'", - format == AVRO ? "VALUES ('mo', 2, NULL, NULL, NULL, NULL)" : "VALUES ('mo', 2, 'mommy', 'moscow', 4, 5)"); + select + " WHERE partition.d_trunc = 'Ki'", + format == AVRO ? "VALUES ('Ki', 2, NULL, NULL, NULL, NULL)" : "VALUES ('Ki', 2, 'Kielce', 'Kiev', 4, 5)"); assertQuery("SELECT b FROM test_truncate_text_transform WHERE substring(d, 1, 2) = 'Gr'", "VALUES 6, 7"); assertQuery( @@ -2955,7 +2955,7 @@ public void testTruncateTextTransform() assertThat(query("SHOW STATS FOR test_truncate_text_transform")) .skippingTypesCheck() .matches("VALUES " + - " ('d', " + (format == PARQUET ? "553e0" : "NULL") + ", NULL, " + (format == AVRO ? "NULL" : "0.125e0") + ", NULL, NULL, NULL), " + + " ('d', " + (format == PARQUET ? "550e0" : "NULL") + ", NULL, " + (format == AVRO ? "NULL" : "0.125e0") + ", NULL, NULL, NULL), " + (format == AVRO ? " ('b', NULL, NULL, NULL, NULL, NULL, NULL), " : " ('b', NULL, NULL, 0e0, NULL, '1', '101'), ") + " (NULL, NULL, NULL, NULL, 8e0, NULL, NULL)"); @@ -3228,8 +3228,8 @@ public void testApplyFilterWithNonEmptyConstraintPredicate() "('abcd', 1)," + "('abxy', 2)," + "('ab598', 3)," + - "('mommy', 4)," + - "('moscow', 5)," + + "('Kielce', 4)," + + "('Kiev', 5)," + "('Greece', 6)," + "('Grozny', 7)", 7); @@ -3247,7 +3247,7 @@ public void testApplyFilterWithNonEmptyConstraintPredicate() } if (format == PARQUET) { expected = "VALUES " + - " ('d', 367e0, NULL, 0e0, NULL, NULL, NULL), " + + " ('d', 364e0, NULL, 0e0, NULL, NULL, NULL), " + " ('b', NULL, NULL, 0e0, NULL, '1', '7'), " + " (NULL, NULL, NULL, NULL, 7e0, NULL, NULL)"; } From 87d734367f535303744d66a3237c6f188ac00702 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 15 Mar 2023 11:03:36 +0100 Subject: [PATCH 8/9] Avoid initializing expected to null --- .../plugin/iceberg/BaseIcebergConnectorTest.java | 15 +++++---------- 1 file changed, 5 insertions(+), 10 deletions(-) diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java index 335102691035..f1f569cab73b 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java @@ -3238,25 +3238,20 @@ public void testApplyFilterWithNonEmptyConstraintPredicate() "SELECT * FROM test_apply_functional_constraint WHERE length(d) = 4 AND b % 7 = 2", "VALUES ('abxy', 2)"); - String expected = null; - if (format == ORC) { - expected = "VALUES " + + String expected = switch (format) { + case ORC -> "VALUES " + " ('d', NULL, NULL, 0e0, NULL, NULL, NULL), " + " ('b', NULL, NULL, 0e0, NULL, '1', '7'), " + " (NULL, NULL, NULL, NULL, 7e0, NULL, NULL)"; - } - if (format == PARQUET) { - expected = "VALUES " + + case PARQUET -> "VALUES " + " ('d', 364e0, NULL, 0e0, NULL, NULL, NULL), " + " ('b', NULL, NULL, 0e0, NULL, '1', '7'), " + " (NULL, NULL, NULL, NULL, 7e0, NULL, NULL)"; - } - else if (format == AVRO) { - expected = "VALUES " + + case AVRO -> "VALUES " + " ('d', NULL, NULL, NULL, NULL, NULL, NULL), " + " ('b', NULL, NULL, NULL, NULL, NULL, NULL), " + " (NULL, NULL, NULL, NULL, 7e0, NULL, NULL)"; - } + }; assertThat(query("SHOW STATS FOR test_apply_functional_constraint")) .skippingTypesCheck() .matches(expected); From fb405ea87bdffff9e66a8d7717cd86b5a99bc74d Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 10 May 2022 12:07:04 +0200 Subject: [PATCH 9/9] Update Iceberg table statistics on writes --- .../plugin/iceberg/CollectedStatistics.java | 29 ++ .../trino/plugin/iceberg/IcebergConfig.java | 15 + .../trino/plugin/iceberg/IcebergMetadata.java | 172 +++++-- .../iceberg/IcebergSessionProperties.java | 12 + .../plugin/iceberg/TableStatisticsWriter.java | 115 ++++- .../iceberg/BaseIcebergConnectorTest.java | 477 +++++++++--------- .../plugin/iceberg/TestIcebergConfig.java | 3 + ...stIcebergGetTableStatisticsOperations.java | 5 + .../TestIcebergMetadataFileOperations.java | 34 +- .../TestIcebergMetastoreAccessOperations.java | 23 +- .../TestIcebergProjectionPushdownPlans.java | 5 + .../plugin/iceberg/TestIcebergStatistics.java | 391 +++++++++----- .../TestMetadataQueryOptimization.java | 5 + ...MetastoreTableOperationsInsertFailure.java | 6 + ...estIcebergGlueCatalogAccessOperations.java | 29 +- .../TestIcebergGlueCreateTableFailure.java | 6 + ...ebergGlueTableOperationsInsertFailure.java | 6 + ...toreTableOperationsReleaseLockFailure.java | 6 + ...TestConnectorPushdownRulesWithIceberg.java | 6 + .../hive/TestHiveRedirectionToIceberg.java | 8 +- 20 files changed, 943 insertions(+), 410 deletions(-) create mode 100644 plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/CollectedStatistics.java diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/CollectedStatistics.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/CollectedStatistics.java new file mode 100644 index 000000000000..f1e35027ac28 --- /dev/null +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/CollectedStatistics.java @@ -0,0 +1,29 @@ +/* + * Licensed 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 io.trino.plugin.iceberg; + +import com.google.common.collect.ImmutableMap; +import org.apache.datasketches.theta.CompactSketch; + +import java.util.Map; + +import static java.util.Objects.requireNonNull; + +public record CollectedStatistics(Map ndvSketches) +{ + public CollectedStatistics + { + ndvSketches = ImmutableMap.copyOf(requireNonNull(ndvSketches, "ndvSketches is null")); + } +} diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConfig.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConfig.java index 9f0c6d0a3c4e..f2d6dde2ac75 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConfig.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConfig.java @@ -46,6 +46,7 @@ public class IcebergConfig public static final int FORMAT_VERSION_SUPPORT_MAX = 2; public static final String EXTENDED_STATISTICS_CONFIG = "iceberg.extended-statistics.enabled"; public static final String EXTENDED_STATISTICS_DESCRIPTION = "Enable collection (ANALYZE) and use of extended statistics."; + public static final String COLLECT_EXTENDED_STATISTICS_ON_WRITE_DESCRIPTION = "Collect extended statistics during writes"; public static final String EXPIRE_SNAPSHOTS_MIN_RETENTION = "iceberg.expire_snapshots.min-retention"; public static final String REMOVE_ORPHAN_FILES_MIN_RETENTION = "iceberg.remove_orphan_files.min-retention"; @@ -58,6 +59,7 @@ public class IcebergConfig private Duration dynamicFilteringWaitTimeout = new Duration(0, SECONDS); private boolean tableStatisticsEnabled = true; private boolean extendedStatisticsEnabled = true; + private boolean collectExtendedStatisticsOnWrite = true; private boolean projectionPushdownEnabled = true; private boolean registerTableProcedureEnabled; private Optional hiveCatalogName = Optional.empty(); @@ -202,6 +204,19 @@ public IcebergConfig setExtendedStatisticsEnabled(boolean extendedStatisticsEnab return this; } + public boolean isCollectExtendedStatisticsOnWrite() + { + return collectExtendedStatisticsOnWrite; + } + + @Config("iceberg.extended-statistics.collect-on-write") + @ConfigDescription(COLLECT_EXTENDED_STATISTICS_ON_WRITE_DESCRIPTION) + public IcebergConfig setCollectExtendedStatisticsOnWrite(boolean collectExtendedStatisticsOnWrite) + { + this.collectExtendedStatisticsOnWrite = collectExtendedStatisticsOnWrite; + return this; + } + public boolean isProjectionPushdownEnabled() { return projectionPushdownEnabled; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java index 74faa3aca81b..b9a0e745238c 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java @@ -164,6 +164,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; +import java.util.function.Predicate; import java.util.function.Supplier; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -203,6 +204,7 @@ import static io.trino.plugin.iceberg.IcebergMetadataColumn.isMetadataColumnId; import static io.trino.plugin.iceberg.IcebergSessionProperties.getExpireSnapshotMinRetention; import static io.trino.plugin.iceberg.IcebergSessionProperties.getRemoveOrphanFilesMinRetention; +import static io.trino.plugin.iceberg.IcebergSessionProperties.isCollectExtendedStatisticsOnWrite; import static io.trino.plugin.iceberg.IcebergSessionProperties.isExtendedStatisticsEnabled; import static io.trino.plugin.iceberg.IcebergSessionProperties.isMergeManifestsOnWrite; import static io.trino.plugin.iceberg.IcebergSessionProperties.isProjectionPushdownEnabled; @@ -232,6 +234,8 @@ import static io.trino.plugin.iceberg.SortFieldUtils.parseSortFields; import static io.trino.plugin.iceberg.TableStatisticsReader.TRINO_STATS_COLUMN_ID_PATTERN; import static io.trino.plugin.iceberg.TableStatisticsReader.TRINO_STATS_PREFIX; +import static io.trino.plugin.iceberg.TableStatisticsWriter.StatsUpdateMode.INCREMENTAL_UPDATE; +import static io.trino.plugin.iceberg.TableStatisticsWriter.StatsUpdateMode.REPLACE; import static io.trino.plugin.iceberg.TableType.DATA; import static io.trino.plugin.iceberg.TypeConverter.toIcebergTypeForNewColumn; import static io.trino.plugin.iceberg.TypeConverter.toTrinoType; @@ -851,6 +855,7 @@ public Optional finishInsert(ConnectorSession session, IcebergWritableTableHandle table = (IcebergWritableTableHandle) insertHandle; Table icebergTable = transaction.table(); + Optional beforeWriteSnapshotId = Optional.ofNullable(icebergTable.currentSnapshot()).map(Snapshot::snapshotId); Schema schema = icebergTable.schema(); Type[] partitionColumnTypes = icebergTable.spec().fields().stream() .map(field -> field.transform().getResultType( @@ -883,8 +888,39 @@ public Optional finishInsert(ConnectorSession session, commit(appendFiles, session); transaction.commitTransaction(); + // TODO (https://github.com/trinodb/trino/issues/15439) this may not exactly be the snapshot we committed, if there is another writer + long newSnapshotId = transaction.table().currentSnapshot().snapshotId(); transaction = null; + // TODO (https://github.com/trinodb/trino/issues/15439): it would be good to publish data and stats atomically + beforeWriteSnapshotId.ifPresent(previous -> + verify(previous != newSnapshotId, "Failed to get new snapshot ID ")); + + if (!computedStatistics.isEmpty()) { + try { + beginTransaction(catalog.loadTable(session, table.getName())); + Table reloadedTable = transaction.table(); + CollectedStatistics collectedStatistics = processComputedTableStatistics(reloadedTable, computedStatistics); + StatisticsFile statisticsFile = tableStatisticsWriter.writeStatisticsFile( + session, + reloadedTable, + newSnapshotId, + INCREMENTAL_UPDATE, + collectedStatistics); + transaction.updateStatistics() + .setStatistics(newSnapshotId, statisticsFile) + .commit(); + + transaction.commitTransaction(); + } + catch (Exception e) { + // Write was committed, so at this point we cannot fail the query + // TODO (https://github.com/trinodb/trino/issues/15439): it would be good to publish data and stats atomically + log.error(e, "Failed to save table statistics"); + } + transaction = null; + } + return Optional.of(new HiveWrittenPartitions(commitTasks.stream() .map(CommitTaskData::getPath) .collect(toImmutableList()))); @@ -1740,6 +1776,30 @@ private List getColumnMetadatas(Schema schema) return columns.build(); } + @Override + public TableStatisticsMetadata getStatisticsCollectionMetadataForWrite(ConnectorSession session, ConnectorTableMetadata tableMetadata) + { + if (!isExtendedStatisticsEnabled(session) || !isCollectExtendedStatisticsOnWrite(session)) { + return TableStatisticsMetadata.empty(); + } + + IcebergTableHandle tableHandle = getTableHandle(session, tableMetadata.getTable(), Optional.empty(), Optional.empty()); + if (tableHandle == null) { + // Assume new table (CTAS), collect all stats possible + return getStatisticsCollectionMetadata(tableMetadata, Optional.empty(), availableColumnNames -> {}); + } + TableStatistics tableStatistics = getTableStatistics(session, tableHandle); + if (tableStatistics.getRowCount().getValue() == 0.0) { + // Table has no data (empty, or wiped out). Collect all stats possible + return getStatisticsCollectionMetadata(tableMetadata, Optional.empty(), availableColumnNames -> {}); + } + Set columnsWithExtendedStatistics = tableStatistics.getColumnStatistics().entrySet().stream() + .filter(entry -> !entry.getValue().getDistinctValuesCount().isUnknown()) + .map(entry -> ((IcebergColumnHandle) entry.getKey()).getName()) + .collect(toImmutableSet()); + return getStatisticsCollectionMetadata(tableMetadata, Optional.of(columnsWithExtendedStatistics), availableColumnNames -> {}); + } + @Override public ConnectorAnalyzeMetadata getStatisticsCollectionMetadata(ConnectorSession session, ConnectorTableHandle tableHandle, Map analyzeProperties) { @@ -1758,35 +1818,53 @@ public ConnectorAnalyzeMetadata getStatisticsCollectionMetadata(ConnectorSession } ConnectorTableMetadata tableMetadata = getTableMetadata(session, handle); - Set allScalarColumnNames = tableMetadata.getColumns().stream() - .filter(column -> !column.isHidden()) - .filter(column -> column.getType().getTypeParameters().isEmpty()) // is scalar type - .map(ColumnMetadata::getName) - .collect(toImmutableSet()); - - Set analyzeColumnNames = getColumnNames(analyzeProperties) + Optional> analyzeColumnNames = getColumnNames(analyzeProperties) .map(columnNames -> { // validate that proper column names are passed via `columns` analyze property if (columnNames.isEmpty()) { throw new TrinoException(INVALID_ANALYZE_PROPERTY, "Cannot specify empty list of columns for analysis"); } - if (!allScalarColumnNames.containsAll(columnNames)) { - throw new TrinoException( - INVALID_ANALYZE_PROPERTY, - format("Invalid columns specified for analysis: %s", Sets.difference(columnNames, allScalarColumnNames))); - } return columnNames; - }) - .orElse(allScalarColumnNames); + }); + + return new ConnectorAnalyzeMetadata( + tableHandle, + getStatisticsCollectionMetadata( + tableMetadata, + analyzeColumnNames, + availableColumnNames -> { + throw new TrinoException( + INVALID_ANALYZE_PROPERTY, + format("Invalid columns specified for analysis: %s", Sets.difference(analyzeColumnNames.orElseThrow(), availableColumnNames))); + })); + } + + private TableStatisticsMetadata getStatisticsCollectionMetadata( + ConnectorTableMetadata tableMetadata, + Optional> selectedColumnNames, + Consumer> unsatisfiableSelectedColumnsHandler) + { + Set allScalarColumnNames = tableMetadata.getColumns().stream() + .filter(column -> !column.isHidden()) + .filter(column -> column.getType().getTypeParameters().isEmpty()) // is scalar type + .map(ColumnMetadata::getName) + .collect(toImmutableSet()); + + selectedColumnNames.ifPresent(columnNames -> { + if (!allScalarColumnNames.containsAll(columnNames)) { + unsatisfiableSelectedColumnsHandler.accept(allScalarColumnNames); + } + }); Set columnStatistics = tableMetadata.getColumns().stream() - .filter(column -> analyzeColumnNames.contains(column.getName())) + .filter(columnMetadata -> allScalarColumnNames.contains(columnMetadata.getName())) + .filter(selectedColumnNames + .map(columnNames -> (Predicate) columnMetadata -> columnNames.contains(columnMetadata.getName())) + .orElse(columnMetadata -> true)) .map(column -> new ColumnStatisticMetadata(column.getName(), NUMBER_OF_DISTINCT_VALUES_NAME, NUMBER_OF_DISTINCT_VALUES_FUNCTION)) .collect(toImmutableSet()); - return new ConnectorAnalyzeMetadata( - tableHandle, - new TableStatisticsMetadata(columnStatistics, ImmutableSet.of(), ImmutableList.of())); + return new TableStatisticsMetadata(columnStatistics, ImmutableSet.of(), ImmutableList.of()); } @Override @@ -1824,9 +1902,9 @@ public void finishStatisticsCollection(ConnectorSession session, ConnectorTableH } long snapshotId = handle.getSnapshotId().orElseThrow(); - Map columnNameToId = table.schema().columns().stream() - .collect(toImmutableMap(nestedField -> nestedField.name().toLowerCase(ENGLISH), Types.NestedField::fieldId)); - Set columnIds = ImmutableSet.copyOf(columnNameToId.values()); + Set columnIds = table.schema().columns().stream() + .map(Types.NestedField::fieldId) + .collect(toImmutableSet()); // TODO (https://github.com/trinodb/trino/issues/15397): remove support for Trino-specific statistics properties // Drop stats for obsolete columns @@ -1845,31 +1923,13 @@ public void finishStatisticsCollection(ConnectorSession session, ConnectorTableH .forEach(updateProperties::remove); updateProperties.commit(); - ImmutableMap.Builder ndvSketches = ImmutableMap.builder(); - for (ComputedStatistics computedStatistic : computedStatistics) { - verify(computedStatistic.getGroupingColumns().isEmpty() && computedStatistic.getGroupingValues().isEmpty(), "Unexpected grouping"); - verify(computedStatistic.getTableStatistics().isEmpty(), "Unexpected table statistics"); - for (Map.Entry entry : computedStatistic.getColumnStatistics().entrySet()) { - ColumnStatisticMetadata statisticMetadata = entry.getKey(); - if (statisticMetadata.getConnectorAggregationId().equals(NUMBER_OF_DISTINCT_VALUES_NAME)) { - Integer columnId = verifyNotNull( - columnNameToId.get(statisticMetadata.getColumnName()), - "Column not found in table: [%s]", - statisticMetadata.getColumnName()); - CompactSketch sketch = DataSketchStateSerializer.deserialize(entry.getValue(), 0); - ndvSketches.put(columnId, sketch); - } - else { - throw new UnsupportedOperationException("Unsupported statistic: " + statisticMetadata); - } - } - } - + CollectedStatistics collectedStatistics = processComputedTableStatistics(table, computedStatistics); StatisticsFile statisticsFile = tableStatisticsWriter.writeStatisticsFile( session, table, snapshotId, - ndvSketches.buildOrThrow()); + REPLACE, + collectedStatistics); transaction.updateStatistics() .setStatistics(snapshotId, statisticsFile) .commit(); @@ -2721,6 +2781,34 @@ public Optional redirectTable(ConnectorSession session, return catalog.redirectTable(session, tableName); } + private static CollectedStatistics processComputedTableStatistics(Table table, Collection computedStatistics) + { + Map columnNameToId = table.schema().columns().stream() + .collect(toImmutableMap(nestedField -> nestedField.name().toLowerCase(ENGLISH), Types.NestedField::fieldId)); + + ImmutableMap.Builder ndvSketches = ImmutableMap.builder(); + for (ComputedStatistics computedStatistic : computedStatistics) { + verify(computedStatistic.getGroupingColumns().isEmpty() && computedStatistic.getGroupingValues().isEmpty(), "Unexpected grouping"); + verify(computedStatistic.getTableStatistics().isEmpty(), "Unexpected table statistics"); + for (Map.Entry entry : computedStatistic.getColumnStatistics().entrySet()) { + ColumnStatisticMetadata statisticMetadata = entry.getKey(); + if (statisticMetadata.getConnectorAggregationId().equals(NUMBER_OF_DISTINCT_VALUES_NAME)) { + Integer columnId = verifyNotNull( + columnNameToId.get(statisticMetadata.getColumnName()), + "Column not found in table: [%s]", + statisticMetadata.getColumnName()); + CompactSketch sketch = DataSketchStateSerializer.deserialize(entry.getValue(), 0); + ndvSketches.put(columnId, sketch); + } + else { + throw new UnsupportedOperationException("Unsupported statistic: " + statisticMetadata); + } + } + } + + return new CollectedStatistics(ndvSketches.buildOrThrow()); + } + private void beginTransaction(Table icebergTable) { verify(transaction == null, "transaction already set"); diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSessionProperties.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSessionProperties.java index a21adb312157..498acb0c9afc 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSessionProperties.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSessionProperties.java @@ -36,6 +36,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static io.trino.plugin.base.session.PropertyMetadataUtil.dataSizeProperty; import static io.trino.plugin.base.session.PropertyMetadataUtil.durationProperty; +import static io.trino.plugin.iceberg.IcebergConfig.COLLECT_EXTENDED_STATISTICS_ON_WRITE_DESCRIPTION; import static io.trino.plugin.iceberg.IcebergConfig.EXTENDED_STATISTICS_DESCRIPTION; import static io.trino.spi.StandardErrorCode.INVALID_SESSION_PROPERTY; import static io.trino.spi.session.PropertyMetadata.booleanProperty; @@ -77,6 +78,7 @@ public final class IcebergSessionProperties public static final String EXTENDED_STATISTICS_ENABLED = "extended_statistics_enabled"; private static final String PROJECTION_PUSHDOWN_ENABLED = "projection_pushdown_enabled"; private static final String TARGET_MAX_FILE_SIZE = "target_max_file_size"; + public static final String COLLECT_EXTENDED_STATISTICS_ON_WRITE = "collect_extended_statistics_on_write"; private static final String HIVE_CATALOG_NAME = "hive_catalog_name"; private static final String MINIMUM_ASSIGNED_SPLIT_WEIGHT = "minimum_assigned_split_weight"; public static final String EXPIRE_SNAPSHOTS_MIN_RETENTION = "expire_snapshots_min_retention"; @@ -257,6 +259,11 @@ public IcebergSessionProperties( "Target maximum size of written files; the actual size may be larger", icebergConfig.getTargetMaxFileSize(), false)) + .add(booleanProperty( + COLLECT_EXTENDED_STATISTICS_ON_WRITE, + COLLECT_EXTENDED_STATISTICS_ON_WRITE_DESCRIPTION, + icebergConfig.isCollectExtendedStatisticsOnWrite(), + false)) .add(stringProperty( HIVE_CATALOG_NAME, "Catalog to redirect to when a Hive table is referenced", @@ -440,6 +447,11 @@ public static boolean isExtendedStatisticsEnabled(ConnectorSession session) return session.getProperty(EXTENDED_STATISTICS_ENABLED, Boolean.class); } + public static boolean isCollectExtendedStatisticsOnWrite(ConnectorSession session) + { + return session.getProperty(COLLECT_EXTENDED_STATISTICS_ON_WRITE, Boolean.class); + } + public static boolean isProjectionPushdownEnabled(ConnectorSession session) { return session.getProperty(PROJECTION_PUSHDOWN_ENABLED, Boolean.class); diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/TableStatisticsWriter.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/TableStatisticsWriter.java index 87c22b50f2c8..9b9303b65445 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/TableStatisticsWriter.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/TableStatisticsWriter.java @@ -17,12 +17,16 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.graph.Traverser; +import io.trino.plugin.base.io.ByteBuffers; import io.trino.plugin.hive.NodeVersion; import io.trino.spi.connector.ConnectorSession; +import org.apache.datasketches.memory.Memory; import org.apache.datasketches.theta.CompactSketch; +import org.apache.datasketches.theta.SetOperation; import org.apache.iceberg.GenericBlobMetadata; import org.apache.iceberg.GenericStatisticsFile; import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.ManifestFile; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.StatisticsFile; @@ -36,6 +40,7 @@ import org.apache.iceberg.puffin.PuffinCompressionCodec; import org.apache.iceberg.puffin.PuffinReader; import org.apache.iceberg.puffin.PuffinWriter; +import org.apache.iceberg.puffin.StandardBlobTypes; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.Pair; @@ -45,11 +50,14 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.nio.ByteBuffer; +import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; +import static com.google.common.base.Preconditions.checkState; import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableSet.toImmutableSet; import static com.google.common.collect.Iterables.getOnlyElement; @@ -57,15 +65,28 @@ import static com.google.common.collect.Streams.stream; import static io.trino.plugin.base.util.Closables.closeAllSuppress; import static io.trino.plugin.iceberg.TableStatisticsReader.APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY; +import static io.trino.plugin.iceberg.TableStatisticsReader.walkStatisticsFiles; +import static io.trino.plugin.iceberg.TableStatisticsWriter.StatsUpdateMode.INCREMENTAL_UPDATE; +import static io.trino.plugin.iceberg.TableStatisticsWriter.StatsUpdateMode.REPLACE; import static java.lang.String.format; import static java.util.Map.Entry.comparingByKey; import static java.util.Objects.requireNonNull; import static java.util.UUID.randomUUID; +import static org.apache.iceberg.SnapshotSummary.TOTAL_RECORDS_PROP; import static org.apache.iceberg.puffin.PuffinCompressionCodec.ZSTD; import static org.apache.iceberg.puffin.StandardBlobTypes.APACHE_DATASKETCHES_THETA_V1; public class TableStatisticsWriter { + public enum StatsUpdateMode + { + // E.g. in ANALYZE case + REPLACE, + + // E.g. in INSERT case + INCREMENTAL_UPDATE, + } + private final String trinoVersion; @Inject @@ -78,12 +99,23 @@ public StatisticsFile writeStatisticsFile( ConnectorSession session, Table table, long snapshotId, - Map ndvSketches) + StatsUpdateMode updateMode, + CollectedStatistics collectedStatistics) { Snapshot snapshot = table.snapshot(snapshotId); + TableOperations operations = ((HasTableOperations) table).operations(); + FileIO fileIO = operations.io(); long snapshotSequenceNumber = snapshot.sequenceNumber(); Schema schema = table.schemas().get(snapshot.schemaId()); + collectedStatistics = mergeStatisticsIfNecessary( + table, + snapshotId, + fileIO, + updateMode, + collectedStatistics); + Map ndvSketches = collectedStatistics.ndvSketches(); + Set validFieldIds = stream( Traverser.forTree((Types.NestedField nestedField) -> { Type type = nestedField.type(); @@ -99,8 +131,6 @@ public StatisticsFile writeStatisticsFile( .map(Types.NestedField::fieldId) .collect(toImmutableSet()); - TableOperations operations = ((HasTableOperations) table).operations(); - FileIO fileIO = operations.io(); String path = operations.metadataFileLocation(format("%s-%s.stats", session.getQueryId(), randomUUID())); OutputFile outputFile = fileIO.newOutputFile(path); try { @@ -149,6 +179,68 @@ public StatisticsFile writeStatisticsFile( } } + private CollectedStatistics mergeStatisticsIfNecessary( + Table table, + long snapshotId, + FileIO fileIO, + StatsUpdateMode updateMode, + CollectedStatistics collectedStatistics) + { + if (updateMode == INCREMENTAL_UPDATE) { + Snapshot snapshot = table.snapshot(snapshotId); + checkState(snapshot != null, "No snapshot information for snapshotId %s in table %s", snapshotId, table); + if (snapshot.parentId() == null || !maySnapshotHaveData(table, snapshot.parentId(), fileIO)) { + // No previous snapshot, or previous snapshot empty + updateMode = REPLACE; + } + } + + return switch (updateMode) { + case REPLACE -> collectedStatistics; + case INCREMENTAL_UPDATE -> { + Map collectedNdvSketches = collectedStatistics.ndvSketches(); + ImmutableMap.Builder ndvSketches = ImmutableMap.builder(); + + Set pendingPreviousNdvSketches = new HashSet<>(collectedNdvSketches.keySet()); + Iterator statisticsFiles = walkStatisticsFiles(table, snapshotId); + while (!pendingPreviousNdvSketches.isEmpty() && statisticsFiles.hasNext()) { + StatisticsFile statisticsFile = statisticsFiles.next(); + + boolean hasUsefulData = statisticsFile.blobMetadata().stream() + .filter(blobMetadata -> blobMetadata.type().equals(StandardBlobTypes.APACHE_DATASKETCHES_THETA_V1)) + .filter(blobMetadata -> blobMetadata.fields().size() == 1) + .anyMatch(blobMetadata -> pendingPreviousNdvSketches.contains(getOnlyElement(blobMetadata.fields()))); + + if (hasUsefulData) { + try (PuffinReader reader = Puffin.read(fileIO.newInputFile(statisticsFile.path())) + .withFileSize(statisticsFile.fileSizeInBytes()) + .withFooterSize(statisticsFile.fileFooterSizeInBytes()) + .build()) { + List toRead = reader.fileMetadata().blobs().stream() + .filter(blobMetadata -> blobMetadata.type().equals(APACHE_DATASKETCHES_THETA_V1)) + .filter(blobMetadata -> blobMetadata.inputFields().size() == 1) + .filter(blobMetadata -> pendingPreviousNdvSketches.contains(getOnlyElement(blobMetadata.inputFields()))) + .collect(toImmutableList()); + for (Pair read : reader.readAll(toRead)) { + Integer fieldId = getOnlyElement(read.first().inputFields()); + checkState(pendingPreviousNdvSketches.remove(fieldId), "Unwanted read of stats for field %s", fieldId); + Memory memory = Memory.wrap(ByteBuffers.getBytes(read.second())); // Memory.wrap(ByteBuffer) results in a different deserialized state + CompactSketch previousSketch = CompactSketch.wrap(memory); + CompactSketch newSketch = requireNonNull(collectedNdvSketches.get(fieldId), "ndvSketches.get(fieldId) is null"); + ndvSketches.put(fieldId, SetOperation.builder().buildUnion().union(previousSketch, newSketch)); + } + } + catch (IOException exception) { + throw new UncheckedIOException(exception); + } + } + } + + yield new CollectedStatistics(ndvSketches.buildOrThrow()); + } + }; + } + private void copyRetainedStatistics( FileIO fileIO, StatisticsFile previousStatisticsFile, @@ -175,7 +267,7 @@ private void copyRetainedStatistics( read.first().snapshotId(), read.first().sequenceNumber(), read.second(), - // TODO Allow PuffinReader to read without decompression + // TODO (https://github.com/trinodb/trino/issues/15440) Allow PuffinReader to read without decompression compressionCodec == null ? null : tryGetCompressionCodec(compressionCodec).orElse(ZSTD), @@ -188,6 +280,21 @@ private void copyRetainedStatistics( } } + static boolean maySnapshotHaveData(Table table, long snapshotId, FileIO fileIo) + { + Snapshot snapshot = table.snapshot(snapshotId); + if (snapshot.summary().containsKey(TOTAL_RECORDS_PROP)) { + return Long.parseLong(snapshot.summary().get(TOTAL_RECORDS_PROP)) != 0; + } + + for (ManifestFile dataManifest : snapshot.dataManifests(fileIo)) { + if (dataManifest.hasExistingFiles() || dataManifest.hasAddedFiles()) { + return true; + } + } + return false; + } + @VisibleForTesting static Optional tryGetCompressionCodec(String name) { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java index f1f569cab73b..c594a80116aa 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java @@ -600,7 +600,7 @@ private void testSelectOrPartitionedByTimestampWithTimeZone(boolean partitioned) assertThat(query("SHOW STATS FOR " + tableName)) .skippingTypesCheck() .matches("VALUES " + - "('_timestamptz', NULL, NULL, 0e0, NULL, '1969-12-01 05:06:07.234 UTC', '2021-10-31 00:30:00.007 UTC'), " + + "('_timestamptz', NULL, 4e0, 0e0, NULL, '1969-12-01 05:06:07.234 UTC', '2021-10-31 00:30:00.007 UTC'), " + "(NULL, NULL, NULL, NULL, 4e0, NULL, NULL)"); } else { @@ -609,14 +609,14 @@ private void testSelectOrPartitionedByTimestampWithTimeZone(boolean partitioned) assertThat(query("SHOW STATS FOR " + tableName)) .skippingTypesCheck() .matches("VALUES " + - "('_timestamptz', NULL, NULL, 0e0, NULL, '1969-12-01 05:06:07.234 UTC', '2021-10-31 00:30:00.007 UTC'), " + + "('_timestamptz', NULL, 4e0, 0e0, NULL, '1969-12-01 05:06:07.234 UTC', '2021-10-31 00:30:00.007 UTC'), " + "(NULL, NULL, NULL, NULL, 4e0, NULL, NULL)"); } else { assertThat(query("SHOW STATS FOR " + tableName)) .skippingTypesCheck() .matches("VALUES " + - "('_timestamptz', NULL, NULL, NULL, NULL, NULL, NULL), " + + "('_timestamptz', NULL, 4e0, 0e0, NULL, NULL, NULL), " + "(NULL, NULL, NULL, NULL, 4e0, NULL, NULL)"); } } @@ -627,7 +627,7 @@ private void testSelectOrPartitionedByTimestampWithTimeZone(boolean partitioned) .skippingTypesCheck() .matches("VALUES " + // TODO (https://github.com/trinodb/trino/issues/9716) the min/max values are off by 1 millisecond - "('_timestamptz', NULL, NULL, 0e0, NULL, '2021-10-31 00:30:00.005 UTC', '2021-10-31 00:30:00.005 UTC'), " + + "('_timestamptz', NULL, 1e0, 0e0, NULL, '2021-10-31 00:30:00.005 UTC', '2021-10-31 00:30:00.005 UTC'), " + "(NULL, NULL, NULL, NULL, 1e0, NULL, NULL)"); } else { @@ -636,7 +636,7 @@ private void testSelectOrPartitionedByTimestampWithTimeZone(boolean partitioned) .skippingTypesCheck() .matches("VALUES " + "('_timestamptz', null, 1e0, 0e0, NULL, '2021-10-31 00:30:00.005 UTC', '2021-10-31 00:30:00.005 UTC'), " + - "(NULL, NULL, NULL, NULL, NULL, NULL, NULL)"); + "(NULL, NULL, NULL, NULL, 1e0, NULL, NULL)"); } assertUpdate("DROP TABLE " + tableName); @@ -836,72 +836,72 @@ public void testCreatePartitionedTable() case ORC -> { assertQuery("SHOW STATS FOR test_partitioned_table", "VALUES " + - " ('a_boolean', NULL, NULL, 0.5, NULL, 'true', 'true'), " + - " ('an_integer', NULL, NULL, 0.5, NULL, '1', '1'), " + - " ('a_bigint', NULL, NULL, 0.5, NULL, '1', '1'), " + - " ('a_real', NULL, NULL, 0.5, NULL, '1.0', '1.0'), " + - " ('a_double', NULL, NULL, 0.5, NULL, '1.0', '1.0'), " + - " ('a_short_decimal', NULL, NULL, 0.5, NULL, '1.0', '1.0'), " + - " ('a_long_decimal', NULL, NULL, 0.5, NULL, '11.0', '11.0'), " + - " ('a_varchar', NULL, NULL, 0.5, NULL, NULL, NULL), " + - " ('a_varbinary', NULL, NULL, 0.5, NULL, NULL, NULL), " + - " ('a_date', NULL, NULL, 0.5, NULL, '2021-07-24', '2021-07-24'), " + - " ('a_time', NULL, NULL, 0.5, NULL, NULL, NULL), " + - " ('a_timestamp', NULL, NULL, 0.5, NULL, '2021-07-24 03:43:57.987654', '2021-07-24 03:43:57.987654'), " + - " ('a_timestamptz', NULL, NULL, 0.5, NULL, '2021-07-24 04:43:57.987 UTC', '2021-07-24 04:43:57.987 UTC'), " + - " ('a_uuid', NULL, NULL, 0.5, NULL, NULL, NULL), " + + " ('a_boolean', NULL, 1e0, 0.5, NULL, 'true', 'true'), " + + " ('an_integer', NULL, 1e0, 0.5, NULL, '1', '1'), " + + " ('a_bigint', NULL, 1e0, 0.5, NULL, '1', '1'), " + + " ('a_real', NULL, 1e0, 0.5, NULL, '1.0', '1.0'), " + + " ('a_double', NULL, 1e0, 0.5, NULL, '1.0', '1.0'), " + + " ('a_short_decimal', NULL, 1e0, 0.5, NULL, '1.0', '1.0'), " + + " ('a_long_decimal', NULL, 1e0, 0.5, NULL, '11.0', '11.0'), " + + " ('a_varchar', NULL, 1e0, 0.5, NULL, NULL, NULL), " + + " ('a_varbinary', NULL, 1e0, 0.5, NULL, NULL, NULL), " + + " ('a_date', NULL, 1e0, 0.5, NULL, '2021-07-24', '2021-07-24'), " + + " ('a_time', NULL, 1e0, 0.5, NULL, NULL, NULL), " + + " ('a_timestamp', NULL, 1e0, 0.5, NULL, '2021-07-24 03:43:57.987654', '2021-07-24 03:43:57.987654'), " + + " ('a_timestamptz', NULL, 1e0, 0.5, NULL, '2021-07-24 04:43:57.987 UTC', '2021-07-24 04:43:57.987 UTC'), " + + " ('a_uuid', NULL, 1e0, 0.5, NULL, NULL, NULL), " + " ('a_row', NULL, NULL, 0.5, NULL, NULL, NULL), " + " ('an_array', NULL, NULL, 0.5, NULL, NULL, NULL), " + " ('a_map', NULL, NULL, 0.5, NULL, NULL, NULL), " + - " ('a quoted, field', NULL, NULL, 0.5, NULL, NULL, NULL), " + + " ('a quoted, field', NULL, 1e0, 0.5, NULL, NULL, NULL), " + " (NULL, NULL, NULL, NULL, 2e0, NULL, NULL)"); } case PARQUET -> { assertThat(query("SHOW STATS FOR test_partitioned_table")) .skippingTypesCheck() .matches("VALUES " + - " ('a_boolean', NULL, NULL, 0.5e0, NULL, 'true', 'true'), " + - " ('an_integer', NULL, NULL, 0.5e0, NULL, '1', '1'), " + - " ('a_bigint', NULL, NULL, 0.5e0, NULL, '1', '1'), " + - " ('a_real', NULL, NULL, 0.5e0, NULL, '1.0', '1.0'), " + - " ('a_double', NULL, NULL, 0.5e0, NULL, '1.0', '1.0'), " + - " ('a_short_decimal', NULL, NULL, 0.5e0, NULL, '1.0', '1.0'), " + - " ('a_long_decimal', NULL, NULL, 0.5e0, NULL, '11.0', '11.0'), " + - " ('a_varchar', 234e0, NULL, 0.5e0, NULL, NULL, NULL), " + - " ('a_varbinary', 114e0, NULL, 0.5e0, NULL, NULL, NULL), " + - " ('a_date', NULL, NULL, 0.5e0, NULL, '2021-07-24', '2021-07-24'), " + - " ('a_time', NULL, NULL, 0.5e0, NULL, NULL, NULL), " + - " ('a_timestamp', NULL, NULL, 0.5e0, NULL, '2021-07-24 03:43:57.987654', '2021-07-24 03:43:57.987654'), " + - " ('a_timestamptz', NULL, NULL, 0.5e0, NULL, '2021-07-24 04:43:57.987 UTC', '2021-07-24 04:43:57.987 UTC'), " + - " ('a_uuid', NULL, NULL, 0.5e0, NULL, NULL, NULL), " + + " ('a_boolean', NULL, 1e0, 0.5e0, NULL, 'true', 'true'), " + + " ('an_integer', NULL, 1e0, 0.5e0, NULL, '1', '1'), " + + " ('a_bigint', NULL, 1e0, 0.5e0, NULL, '1', '1'), " + + " ('a_real', NULL, 1e0, 0.5e0, NULL, '1.0', '1.0'), " + + " ('a_double', NULL, 1e0, 0.5e0, NULL, '1.0', '1.0'), " + + " ('a_short_decimal', NULL, 1e0, 0.5e0, NULL, '1.0', '1.0'), " + + " ('a_long_decimal', NULL, 1e0, 0.5e0, NULL, '11.0', '11.0'), " + + " ('a_varchar', 234e0, 1e0, 0.5e0, NULL, NULL, NULL), " + + " ('a_varbinary', 114e0, 1e0, 0.5e0, NULL, NULL, NULL), " + + " ('a_date', NULL, 1e0, 0.5e0, NULL, '2021-07-24', '2021-07-24'), " + + " ('a_time', NULL, 1e0, 0.5e0, NULL, NULL, NULL), " + + " ('a_timestamp', NULL, 1e0, 0.5e0, NULL, '2021-07-24 03:43:57.987654', '2021-07-24 03:43:57.987654'), " + + " ('a_timestamptz', NULL, 1e0, 0.5e0, NULL, '2021-07-24 04:43:57.987 UTC', '2021-07-24 04:43:57.987 UTC'), " + + " ('a_uuid', NULL, 1e0, 0.5e0, NULL, NULL, NULL), " + " ('a_row', NULL, NULL, NULL, NULL, NULL, NULL), " + " ('an_array', NULL, NULL, NULL, NULL, NULL, NULL), " + " ('a_map', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('a quoted, field', 224e0, NULL, 0.5e0, NULL, NULL, NULL), " + + " ('a quoted, field', 224e0, 1e0, 0.5e0, NULL, NULL, NULL), " + " (NULL, NULL, NULL, NULL, 2e0, NULL, NULL)"); } case AVRO -> { assertThat(query("SHOW STATS FOR test_partitioned_table")) .skippingTypesCheck() .matches("VALUES " + - " ('a_boolean', NULL, NULL, 0.5e0, NULL, 'true', 'true'), " + - " ('an_integer', NULL, NULL, 0.5e0, NULL, '1', '1'), " + - " ('a_bigint', NULL, NULL, 0.5e0, NULL, '1', '1'), " + - " ('a_real', NULL, NULL, 0.5e0, NULL, '1.0', '1.0'), " + - " ('a_double', NULL, NULL, 0.5e0, NULL, '1.0', '1.0'), " + - " ('a_short_decimal', NULL, NULL, 0.5e0, NULL, '1.0', '1.0'), " + - " ('a_long_decimal', NULL, NULL, 0.5e0, NULL, '11.0', '11.0'), " + - " ('a_varchar', NULL, NULL, 0.5e0, NULL, NULL, NULL), " + - " ('a_varbinary', NULL, NULL, 0.5e0, NULL, NULL, NULL), " + - " ('a_date', NULL, NULL, 0.5e0, NULL, '2021-07-24', '2021-07-24'), " + - " ('a_time', NULL, NULL, 0.5e0, NULL, NULL, NULL), " + - " ('a_timestamp', NULL, NULL, 0.5e0, NULL, '2021-07-24 03:43:57.987654', '2021-07-24 03:43:57.987654'), " + - " ('a_timestamptz', NULL, NULL, 0.5e0, NULL, '2021-07-24 04:43:57.987 UTC', '2021-07-24 04:43:57.987 UTC'), " + - " ('a_uuid', NULL, NULL, 0.5e0, NULL, NULL, NULL), " + + " ('a_boolean', NULL, 1e0, 0.5e0, NULL, 'true', 'true'), " + + " ('an_integer', NULL, 1e0, 0.5e0, NULL, '1', '1'), " + + " ('a_bigint', NULL, 1e0, 0.5e0, NULL, '1', '1'), " + + " ('a_real', NULL, 1e0, 0.5e0, NULL, '1.0', '1.0'), " + + " ('a_double', NULL, 1e0, 0.5e0, NULL, '1.0', '1.0'), " + + " ('a_short_decimal', NULL, 1e0, 0.5e0, NULL, '1.0', '1.0'), " + + " ('a_long_decimal', NULL, 1e0, 0.5e0, NULL, '11.0', '11.0'), " + + " ('a_varchar', NULL, 1e0, 0.5e0, NULL, NULL, NULL), " + + " ('a_varbinary', NULL, 1e0, 0.5e0, NULL, NULL, NULL), " + + " ('a_date', NULL, 1e0, 0.5e0, NULL, '2021-07-24', '2021-07-24'), " + + " ('a_time', NULL, 1e0, 0.5e0, NULL, NULL, NULL), " + + " ('a_timestamp', NULL, 1e0, 0.5e0, NULL, '2021-07-24 03:43:57.987654', '2021-07-24 03:43:57.987654'), " + + " ('a_timestamptz', NULL, 1e0, 0.5e0, NULL, '2021-07-24 04:43:57.987 UTC', '2021-07-24 04:43:57.987 UTC'), " + + " ('a_uuid', NULL, 1e0, 0.5e0, NULL, NULL, NULL), " + " ('a_row', NULL, NULL, NULL, NULL, NULL, NULL), " + " ('an_array', NULL, NULL, NULL, NULL, NULL, NULL), " + " ('a_map', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('a quoted, field', NULL, NULL, 0.5e0, NULL, NULL, NULL), " + + " ('a quoted, field', NULL, 1e0, 0.5e0, NULL, NULL, NULL), " + " (NULL, NULL, NULL, NULL, 2e0, NULL, NULL)"); } } @@ -1527,18 +1527,18 @@ public void testShowStatsAfterAddColumn() assertThat(query("SHOW STATS FOR test_show_stats_after_add_column")) .skippingTypesCheck() .matches("VALUES " + - " ('col0', NULL, NULL, 25e-2, NULL, '1', '7')," + - " ('col1', NULL, NULL, 25e-2, NULL, '2', '8'), " + - " ('col2', NULL, NULL, 25e-2, NULL, '3', '9'), " + + " ('col0', NULL, 3e0, 25e-2, NULL, '1', '7')," + + " ('col1', NULL, 3e0, 25e-2, NULL, '2', '8'), " + + " ('col2', NULL, 3e0, 25e-2, NULL, '3', '9'), " + " (NULL, NULL, NULL, NULL, 4e0, NULL, NULL)"); } else { assertThat(query("SHOW STATS FOR test_show_stats_after_add_column")) .skippingTypesCheck() .matches("VALUES " + - " ('col0', NULL, NULL, NULL, NULL, NULL, NULL)," + - " ('col1', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('col2', NULL, NULL, NULL, NULL, NULL, NULL), " + + " ('col0', NULL, 3e0, 0.1e0, NULL, NULL, NULL)," + + " ('col1', NULL, 3e0, 0.1e0, NULL, NULL, NULL), " + + " ('col2', NULL, 3e0, 0.1e0, NULL, NULL, NULL), " + " (NULL, NULL, NULL, NULL, 4e0, NULL, NULL)"); } @@ -1549,9 +1549,9 @@ public void testShowStatsAfterAddColumn() assertThat(query("SHOW STATS FOR test_show_stats_after_add_column")) .skippingTypesCheck() .matches("VALUES " + - " ('col0', NULL, NULL, 2e-1, NULL, '1', '10')," + - " ('col1', NULL, NULL, 2e-1, NULL, '2', '11'), " + - " ('col2', NULL, NULL, 2e-1, NULL, '3', '12'), " + + " ('col0', NULL, 4e0, 2e-1, NULL, '1', '10')," + + " ('col1', NULL, 4e0, 2e-1, NULL, '2', '11'), " + + " ('col2', NULL, 4e0, 2e-1, NULL, '3', '12'), " + " ('col3', NULL, NULL, NULL, NULL, NULL, NULL), " + " (NULL, NULL, NULL, NULL, 5e0, NULL, NULL)"); } @@ -1559,9 +1559,9 @@ public void testShowStatsAfterAddColumn() assertThat(query("SHOW STATS FOR test_show_stats_after_add_column")) .skippingTypesCheck() .matches("VALUES " + - " ('col0', NULL, NULL, NULL, NULL, NULL, NULL)," + - " ('col1', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('col2', NULL, NULL, NULL, NULL, NULL, NULL), " + + " ('col0', NULL, 4e0, 0.1e0, NULL, NULL, NULL)," + + " ('col1', NULL, 4e0, 0.1e0, NULL, NULL, NULL), " + + " ('col2', NULL, 4e0, 0.1e0, NULL, NULL, NULL), " + " ('col3', NULL, NULL, NULL, NULL, NULL, NULL), " + " (NULL, NULL, NULL, NULL, 5e0, NULL, NULL)"); } @@ -1707,8 +1707,8 @@ public void testHourTransformTimestamp() "(394474, 3, TIMESTAMP '2015-01-01 10:01:23.123456', TIMESTAMP '2015-01-01 10:55:00.456789', 1, 3), " + "(397692, 2, TIMESTAMP '2015-05-15 12:05:01.234567', TIMESTAMP '2015-05-15 12:21:02.345678', 4, 5), " + "(439525, 2, TIMESTAMP '2020-02-21 13:11:11.876543', TIMESTAMP '2020-02-21 13:12:12.654321', 6, 7)"; - String expectedTimestampStats = "NULL, NULL, 0.0833333e0, NULL, '1969-12-31 22:22:22.222222', '2020-02-21 13:12:12.654321'"; - String expectedBigIntStats = "NULL, NULL, 0e0, NULL, '1', '101'"; + String expectedTimestampStats = "NULL, 11e0, 0.0833333e0, NULL, '1969-12-31 22:22:22.222222', '2020-02-21 13:12:12.654321'"; + String expectedBigIntStats = "NULL, 12e0, 0e0, NULL, '1', '101'"; if (format == ORC) { expected = "VALUES " + "(NULL, 1, NULL, NULL, 101, 101), " + @@ -1718,7 +1718,7 @@ public void testHourTransformTimestamp() "(394474, 3, TIMESTAMP '2015-01-01 10:01:23.123000', TIMESTAMP '2015-01-01 10:55:00.456999', 1, 3), " + "(397692, 2, TIMESTAMP '2015-05-15 12:05:01.234000', TIMESTAMP '2015-05-15 12:21:02.345999', 4, 5), " + "(439525, 2, TIMESTAMP '2020-02-21 13:11:11.876000', TIMESTAMP '2020-02-21 13:12:12.654999', 6, 7)"; - expectedTimestampStats = "NULL, NULL, 0.0833333e0, NULL, '1969-12-31 22:22:22.222000', '2020-02-21 13:12:12.654999'"; + expectedTimestampStats = "NULL, 11e0, 0.0833333e0, NULL, '1969-12-31 22:22:22.222000', '2020-02-21 13:12:12.654999'"; } else if (format == AVRO) { expected = "VALUES " + @@ -1729,8 +1729,8 @@ else if (format == AVRO) { "(394474, 3, NULL, NULL, NULL, NULL), " + "(397692, 2, NULL, NULL, NULL, NULL), " + "(439525, 2, NULL, NULL, NULL, NULL)"; - expectedTimestampStats = "NULL, NULL, NULL, NULL, NULL, NULL"; - expectedBigIntStats = "NULL, NULL, NULL, NULL, NULL, NULL"; + expectedTimestampStats = "NULL, 11e0, 0.0833333e0, NULL, NULL, NULL"; + expectedBigIntStats = "NULL, 12e0, 0e0, NULL, NULL, NULL"; } assertQuery("SELECT partition.d_hour, record_count, data.d.min, data.d.max, data.b.min, data.b.max FROM \"test_hour_transform_timestamp$partitions\"", expected); @@ -1813,8 +1813,8 @@ public void testHourTransformTimestampWithTimeZone() "(394474, 3, TIMESTAMP '2015-01-01 10:01:23.123456 UTC', TIMESTAMP '2015-01-01 10:55:00.456789 UTC', 1, 3), " + "(397692, 2, TIMESTAMP '2015-05-15 12:05:01.234567 UTC', TIMESTAMP '2015-05-15 12:21:02.345678 UTC', 4, 5), " + "(439525, 2, TIMESTAMP '2020-02-21 13:11:11.876543 UTC', TIMESTAMP '2020-02-21 13:12:12.654321 UTC', 6, 7)"; - String expectedTimestampStats = "NULL, NULL, 0.0833333e0, NULL, '1969-12-31 22:22:22.222 UTC', '2020-02-21 13:12:12.654 UTC'"; - String expectedBigIntStats = "NULL, NULL, 0e0, NULL, '1', '101'"; + String expectedTimestampStats = "NULL, 11e0, 0.0833333e0, NULL, '1969-12-31 22:22:22.222 UTC', '2020-02-21 13:12:12.654 UTC'"; + String expectedBigIntStats = "NULL, 12e0, 0e0, NULL, '1', '101'"; if (format == ORC) { expected = "VALUES " + "(NULL, BIGINT '1', NULL, NULL, 101, 101), " + @@ -1824,7 +1824,7 @@ public void testHourTransformTimestampWithTimeZone() "(394474, 3, TIMESTAMP '2015-01-01 10:01:23.123000 UTC', TIMESTAMP '2015-01-01 10:55:00.456999 UTC', 1, 3), " + "(397692, 2, TIMESTAMP '2015-05-15 12:05:01.234000 UTC', TIMESTAMP '2015-05-15 12:21:02.345999 UTC', 4, 5), " + "(439525, 2, TIMESTAMP '2020-02-21 13:11:11.876000 UTC', TIMESTAMP '2020-02-21 13:12:12.654999 UTC', 6, 7)"; - expectedTimestampStats = "NULL, NULL, 0.0833333e0, NULL, '1969-12-31 22:22:22.222 UTC', '2020-02-21 13:12:12.654 UTC'"; + expectedTimestampStats = "NULL, 11e0, 0.0833333e0, NULL, '1969-12-31 22:22:22.222 UTC', '2020-02-21 13:12:12.654 UTC'"; } else if (format == AVRO) { expected = "VALUES " + @@ -1835,8 +1835,8 @@ else if (format == AVRO) { "(394474, 3, NULL, NULL, NULL, NULL), " + "(397692, 2, NULL, NULL, NULL, NULL), " + "(439525, 2, NULL, NULL, NULL, NULL)"; - expectedTimestampStats = "NULL, NULL, NULL, NULL, NULL, NULL"; - expectedBigIntStats = "NULL, NULL, NULL, NULL, NULL, NULL"; + expectedTimestampStats = "NULL, 11e0, 0.0833333e0, NULL, NULL, NULL"; + expectedBigIntStats = "NULL, 12e0, 0e0, NULL, NULL, NULL"; } assertThat(query("SELECT partition.d_hour, record_count, data.d.min, data.d.max, data.b.min, data.b.max FROM \"test_hour_transform_timestamptz$partitions\"")) @@ -1980,13 +1980,13 @@ public void testDayTransformDate() "VALUES (DATE '1969-01-01', 10)"); String expectedTransformed = "VALUES " + - " ('d', NULL, NULL, 0.0833333e0, NULL, '1969-01-01', '2020-02-21'), " + - " ('b', NULL, NULL, 0e0, NULL, '1', '101'), " + + " ('d', NULL, 8e0, 0.0833333e0, NULL, '1969-01-01', '2020-02-21'), " + + " ('b', NULL, 12e0, 0e0, NULL, '1', '101'), " + " (NULL, NULL, NULL, NULL, 12e0, NULL, NULL)"; if (format == AVRO) { expectedTransformed = "VALUES " + - " ('d', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('b', NULL, NULL, NULL, NULL, NULL, NULL), " + + " ('d', NULL, 8e0, 0.1e0, NULL, NULL, NULL), " + + " ('b', NULL, 12e0, 0e0, NULL, NULL, NULL), " + " (NULL, NULL, NULL, NULL, 12e0, NULL, NULL)"; } assertThat(query("SHOW STATS FOR test_day_transform_date")) @@ -2060,8 +2060,8 @@ public void testDayTransformTimestamp() "(DATE '2015-05-15', 2, TIMESTAMP '2015-05-15 13:05:01.234567', TIMESTAMP '2015-05-15 14:21:02.345678', 4, 5), " + "(DATE '2020-02-21', 2, TIMESTAMP '2020-02-21 15:11:11.876543', TIMESTAMP '2020-02-21 16:12:12.654321', 6, 7)"; String expectedTimestampStats = "VALUES " + - " ('d', NULL, NULL, 0.0769231e0, NULL, '1969-12-25 15:13:12.876543', '2020-02-21 16:12:12.654321'), " + - " ('b', NULL, NULL, 0e0, NULL, '1', '101'), " + + " ('d', NULL, 12e0, 0.0769231e0, NULL, '1969-12-25 15:13:12.876543', '2020-02-21 16:12:12.654321'), " + + " ('b', NULL, 13e0, 0e0, NULL, '1', '101'), " + " (NULL, NULL, NULL, NULL, 13e0, NULL, NULL)"; if (format == ORC) { @@ -2075,8 +2075,8 @@ public void testDayTransformTimestamp() "(DATE '2015-05-15', 2, TIMESTAMP '2015-05-15 13:05:01.234000', TIMESTAMP '2015-05-15 14:21:02.345999', 4, 5), " + "(DATE '2020-02-21', 2, TIMESTAMP '2020-02-21 15:11:11.876000', TIMESTAMP '2020-02-21 16:12:12.654999', 6, 7)"; expectedTimestampStats = "VALUES " + - " ('d', NULL, NULL, 0.0769231e0, NULL, '1969-12-25 15:13:12.876000', '2020-02-21 16:12:12.654999'), " + - " ('b', NULL, NULL, 0e0, NULL, '1', '101'), " + + " ('d', NULL, 12e0, 0.0769231e0, NULL, '1969-12-25 15:13:12.876000', '2020-02-21 16:12:12.654999'), " + + " ('b', NULL, 13e0, 0e0, NULL, '1', '101'), " + " (NULL, NULL, NULL, NULL, 13e0, NULL, NULL)"; } else if (format == AVRO) { @@ -2090,8 +2090,8 @@ else if (format == AVRO) { "(DATE '2015-05-15', 2, NULL, NULL, NULL, NULL), " + "(DATE '2020-02-21', 2, NULL, NULL, NULL, NULL)"; expectedTimestampStats = "VALUES " + - " ('d', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('b', NULL, NULL, NULL, NULL, NULL, NULL), " + + " ('d', NULL, 12e0, 0.076923e0, NULL, NULL, NULL), " + + " ('b', NULL, 13e0, 0e0, NULL, NULL, NULL), " + " (NULL, NULL, NULL, NULL, 13e0, NULL, NULL)"; } @@ -2172,8 +2172,8 @@ public void testDayTransformTimestampWithTimeZone() "(DATE '2015-01-01', 3, TIMESTAMP '2015-01-01 10:01:23.123456 UTC', TIMESTAMP '2015-01-01 12:55:00.456789 UTC', 1, 3), " + "(DATE '2015-05-15', 2, TIMESTAMP '2015-05-15 13:05:01.234567 UTC', TIMESTAMP '2015-05-15 14:21:02.345678 UTC', 4, 5), " + "(DATE '2020-02-21', 2, TIMESTAMP '2020-02-21 15:11:11.876543 UTC', TIMESTAMP '2020-02-21 16:12:12.654321 UTC', 6, 7)"; - String expectedTimestampStats = "NULL, NULL, 0.0769231e0, NULL, '1969-12-25 15:13:12.876 UTC', '2020-02-21 16:12:12.654 UTC'"; - String expectedIntegerStats = "NULL, NULL, 0e0, NULL, '1', '101'"; + String expectedTimestampStats = "NULL, 12e0, 0.0769231e0, NULL, '1969-12-25 15:13:12.876 UTC', '2020-02-21 16:12:12.654 UTC'"; + String expectedIntegerStats = "NULL, 13e0, 0e0, NULL, '1', '101'"; if (format == ORC) { expected = "VALUES " + "(NULL, BIGINT '1', NULL, NULL, 101, 101), " + @@ -2195,8 +2195,8 @@ else if (format == AVRO) { "(DATE '2015-01-01', 3, NULL, NULL, NULL, NULL), " + "(DATE '2015-05-15', 2, NULL, NULL, NULL, NULL), " + "(DATE '2020-02-21', 2, NULL, NULL, NULL, NULL)"; - expectedTimestampStats = "NULL, NULL, NULL, NULL, NULL, NULL"; - expectedIntegerStats = "NULL, NULL, NULL, NULL, NULL, NULL"; + expectedTimestampStats = "NULL, 12e0, 0.0769231e0, NULL, NULL, NULL"; + expectedIntegerStats = "NULL, 13e0, 0e0, NULL, NULL, NULL"; } assertThat(query("SELECT partition.d_day, record_count, data.d.min, data.d.max, data.b.min, data.b.max FROM \"test_day_transform_timestamptz$partitions\"")) @@ -2278,8 +2278,8 @@ public void testMonthTransformDate() assertUpdate("INSERT INTO test_month_transform_date " + values, 15); assertQuery("SELECT * FROM test_month_transform_date", values); - String expectedDateStats = "NULL, NULL, 0.0666667e0, NULL, '1969-11-13', '2020-12-31'"; - String expectedBigIntStats = "NULL, NULL, 0e0, NULL, '1', '101'"; + String expectedDateStats = "NULL, 14e0, 0.0666667e0, NULL, '1969-11-13', '2020-12-31'"; + String expectedBigIntStats = "NULL, 15e0, 0e0, NULL, '1', '101'"; if (format != AVRO) { assertQuery( "SELECT partition.d_month, record_count, data.d.min, data.d.max, data.b.min, data.b.max FROM \"test_month_transform_date$partitions\"", @@ -2309,8 +2309,8 @@ public void testMonthTransformDate() "(605, 3, NULL, NULL, NULL, NULL), " + "(606, 2, NULL, NULL, NULL, NULL), " + "(611, 1, NULL, NULL, NULL, NULL)"); - expectedDateStats = "NULL, NULL, NULL, NULL, NULL, NULL"; - expectedBigIntStats = "NULL, NULL, NULL, NULL, NULL, NULL"; + expectedDateStats = "NULL, 14e0, 0.0666667e0, NULL, NULL, NULL"; + expectedBigIntStats = "NULL, 15e0, 0e0, NULL, NULL, NULL"; } // Exercise IcebergMetadata.applyFilter with non-empty Constraint.predicate, via non-pushdownable predicates @@ -2359,16 +2359,16 @@ public void testMonthTransformDate() assertThat(query("SHOW STATS FOR test_month_transform_date")) .skippingTypesCheck() .matches("VALUES " + - " ('d', NULL, NULL, 0.0666667e0, NULL, '1969-11-13', '2020-12-31'), " + - " ('b', NULL, NULL, 0e0, NULL, '1', '101'), " + + " ('d', NULL, 14e0, 0.0666667e0, NULL, '1969-11-13', '2020-12-31'), " + + " ('b', NULL, 15e0, 0e0, NULL, '1', '101'), " + " (NULL, NULL, NULL, NULL, 15e0, NULL, NULL)"); } else { assertThat(query("SHOW STATS FOR test_month_transform_date")) .skippingTypesCheck() .matches("VALUES " + - " ('d', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('b', NULL, NULL, NULL, NULL, NULL, NULL), " + + " ('d', NULL, 14e0, 0.0666667e0, NULL, NULL, NULL), " + + " ('b', NULL, 15e0, 0e0, NULL, NULL, NULL), " + " (NULL, NULL, NULL, NULL, 15e0, NULL, NULL)"); } @@ -2406,8 +2406,8 @@ public void testMonthTransformTimestamp() "(544, 2, TIMESTAMP '2015-05-15 13:05:01.234567', TIMESTAMP '2015-05-15 14:21:02.345678', 4, 5), " + "(601, 2, TIMESTAMP '2020-02-21 15:11:11.876543', TIMESTAMP '2020-02-21 16:12:12.654321', 6, 7)"; String expectedTimestampStats = "VALUES " + - " ('d', NULL, NULL, 0.0769231e0, NULL, '1969-11-15 15:13:12.876543', '2020-02-21 16:12:12.654321'), " + - " ('b', NULL, NULL, 0e0, NULL, '1', '101'), " + + " ('d', NULL, 12e0, 0.0769231e0, NULL, '1969-11-15 15:13:12.876543', '2020-02-21 16:12:12.654321'), " + + " ('b', NULL, 13e0, 0e0, NULL, '1', '101'), " + " (NULL, NULL, NULL, NULL, 13e0, NULL, NULL)"; if (format == ORC) { @@ -2420,8 +2420,8 @@ public void testMonthTransformTimestamp() "(544, 2, TIMESTAMP '2015-05-15 13:05:01.234000', TIMESTAMP '2015-05-15 14:21:02.345999', 4, 5), " + "(601, 2, TIMESTAMP '2020-02-21 15:11:11.876000', TIMESTAMP '2020-02-21 16:12:12.654999', 6, 7)"; expectedTimestampStats = "VALUES " + - " ('d', NULL, NULL, 0.0769231e0, NULL, '1969-11-15 15:13:12.876000', '2020-02-21 16:12:12.654999'), " + - " ('b', NULL, NULL, 0e0, NULL, '1', '101'), " + + " ('d', NULL, 12e0, 0.0769231e0, NULL, '1969-11-15 15:13:12.876000', '2020-02-21 16:12:12.654999'), " + + " ('b', NULL, 13e0, 0e0, NULL, '1', '101'), " + " (NULL, NULL, NULL, NULL, 13e0, NULL, NULL)"; } else if (format == AVRO) { @@ -2434,8 +2434,8 @@ else if (format == AVRO) { "(544, 2, NULL, NULL, NULL, NULL), " + "(601, 2, NULL, NULL, NULL, NULL)"; expectedTimestampStats = "VALUES " + - " ('d', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('b', NULL, NULL, NULL, NULL, NULL, NULL), " + + " ('d', NULL, 12e0, 0.0769231e0, NULL, NULL, NULL), " + + " ('b', NULL, 13e0, 0e0, NULL, NULL, NULL), " + " (NULL, NULL, NULL, NULL, 13e0, NULL, NULL)"; } @@ -2513,8 +2513,8 @@ public void testMonthTransformTimestampWithTimeZone() "(540, 3, TIMESTAMP '2015-01-01 10:01:23.123456 UTC', TIMESTAMP '2015-01-01 12:55:00.456789 UTC', 1, 3), " + "(544, 2, TIMESTAMP '2015-05-15 13:05:01.234567 UTC', TIMESTAMP '2015-05-15 14:21:02.345678 UTC', 4, 5), " + "(601, 2, TIMESTAMP '2020-02-21 15:11:11.876543 UTC', TIMESTAMP '2020-02-21 16:12:12.654321 UTC', 6, 7)"; - String expectedTimestampStats = "NULL, NULL, 0.0769231e0, NULL, '1969-11-15 15:13:12.876 UTC', '2020-02-21 16:12:12.654 UTC'"; - String expectedIntegerStats = "NULL, NULL, 0e0, NULL, '1', '101'"; + String expectedTimestampStats = "NULL, 12e0, 0.0769231e0, NULL, '1969-11-15 15:13:12.876 UTC', '2020-02-21 16:12:12.654 UTC'"; + String expectedIntegerStats = "NULL, 13e0, 0e0, NULL, '1', '101'"; if (format == ORC) { expected = "VALUES " + "(NULL, BIGINT '1', NULL, NULL, 101, 101), " + @@ -2534,8 +2534,8 @@ else if (format == AVRO) { "(540, 3, NULL, NULL, NULL, NULL), " + "(544, 2, NULL, NULL, NULL, NULL), " + "(601, 2, NULL, NULL, NULL, NULL)"; - expectedTimestampStats = "NULL, NULL, NULL, NULL, NULL, NULL"; - expectedIntegerStats = "NULL, NULL, NULL, NULL, NULL, NULL"; + expectedTimestampStats = "NULL, 12e0, 0.0769231e0, NULL, NULL, NULL"; + expectedIntegerStats = "NULL, 13e0, 0e0, NULL, NULL, NULL"; } assertThat(query("SELECT partition.d_month, record_count, data.d.min, data.d.max, data.b.min, data.b.max FROM \"test_month_transform_timestamptz$partitions\"")) @@ -2647,16 +2647,16 @@ public void testYearTransformDate() assertThat(query("SHOW STATS FOR test_year_transform_date")) .skippingTypesCheck() .matches("VALUES " + - " ('d', NULL, NULL, 0.0769231e0, NULL, '1968-10-13', '2020-11-10'), " + - " ('b', NULL, NULL, 0e0, NULL, '1', '101'), " + + " ('d', NULL, 12e0, 0.0769231e0, NULL, '1968-10-13', '2020-11-10'), " + + " ('b', NULL, 13e0, 0e0, NULL, '1', '101'), " + " (NULL, NULL, NULL, NULL, 13e0, NULL, NULL)"); } else { assertThat(query("SHOW STATS FOR test_year_transform_date")) .skippingTypesCheck() .matches("VALUES " + - " ('d', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('b', NULL, NULL, NULL, NULL, NULL, NULL), " + + " ('d', NULL, 12e0, 0.0769231e0, NULL, NULL, NULL), " + + " ('b', NULL, 13e0, 0e0, NULL, NULL, NULL), " + " (NULL, NULL, NULL, NULL, 13e0, NULL, NULL)"); } @@ -2692,16 +2692,16 @@ public void testYearTransformDate() assertThat(query("SHOW STATS FOR test_year_transform_date")) .skippingTypesCheck() .matches("VALUES " + - " ('d', NULL, NULL, 0.0769231e0, NULL, '1968-10-13', '2020-11-10'), " + - " ('b', NULL, NULL, 0e0, NULL, '1', '101'), " + + " ('d', NULL, 12e0, 0.0769231e0, NULL, '1968-10-13', '2020-11-10'), " + + " ('b', NULL, 13e0, 0e0, NULL, '1', '101'), " + " (NULL, NULL, NULL, NULL, 13e0, NULL, NULL)"); } else { assertThat(query("SHOW STATS FOR test_year_transform_date")) .skippingTypesCheck() .matches("VALUES " + - " ('d', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('b', NULL, NULL, NULL, NULL, NULL, NULL), " + + " ('d', NULL, 12e0, 0.0769231e0, NULL, NULL, NULL), " + + " ('b', NULL, 13e0, 0e0, NULL, NULL, NULL), " + " (NULL, NULL, NULL, NULL, 13e0, NULL, NULL)"); } @@ -2738,8 +2738,8 @@ public void testYearTransformTimestamp() "(45, 2, TIMESTAMP '2015-05-15 13:05:01.234567', TIMESTAMP '2015-09-15 14:21:02.345678', 9, 10), " + "(50, 2, TIMESTAMP '2020-02-21 15:11:11.876543', TIMESTAMP '2020-08-21 16:12:12.654321', 11, 12)"; String expectedTimestampStats = "VALUES " + - " ('d', NULL, NULL, 0.0769231e0, NULL, '1968-03-15 15:13:12.876543', '2020-08-21 16:12:12.654321'), " + - " ('b', NULL, NULL, 0e0, NULL, '1', '101'), " + + " ('d', NULL, 12e0, 0.0769231e0, NULL, '1968-03-15 15:13:12.876543', '2020-08-21 16:12:12.654321'), " + + " ('b', NULL, 13e0, 0e0, NULL, '1', '101'), " + " (NULL, NULL, NULL, NULL, 13e0, NULL, NULL)"; if (format == ORC) { @@ -2751,8 +2751,8 @@ public void testYearTransformTimestamp() "(45, 2, TIMESTAMP '2015-05-15 13:05:01.234000', TIMESTAMP '2015-09-15 14:21:02.345999', 9, 10), " + "(50, 2, TIMESTAMP '2020-02-21 15:11:11.876000', TIMESTAMP '2020-08-21 16:12:12.654999', 11, 12)"; expectedTimestampStats = "VALUES " + - " ('d', NULL, NULL, 0.0769231e0, NULL, '1968-03-15 15:13:12.876000', '2020-08-21 16:12:12.654999'), " + - " ('b', NULL, NULL, 0e0, NULL, '1', '101'), " + + " ('d', NULL, 12e0, 0.0769231e0, NULL, '1968-03-15 15:13:12.876000', '2020-08-21 16:12:12.654999'), " + + " ('b', NULL, 13e0, 0e0, NULL, '1', '101'), " + " (NULL, NULL, NULL, NULL, 13e0, NULL, NULL)"; } else if (format == AVRO) { @@ -2764,8 +2764,8 @@ else if (format == AVRO) { "(45, 2, NULL, NULL, NULL, NULL), " + "(50, 2, NULL, NULL, NULL, NULL)"; expectedTimestampStats = "VALUES " + - " ('d', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('b', NULL, NULL, NULL, NULL, NULL, NULL), " + + " ('d', NULL, 12e0, 0.0769231e0, NULL, NULL, NULL), " + + " ('b', NULL, 13e0, 0e0, NULL, NULL, NULL), " + " (NULL, NULL, NULL, NULL, 13e0, NULL, NULL)"; } @@ -2840,8 +2840,8 @@ public void testYearTransformTimestampWithTimeZone() "(0, 4, TIMESTAMP '1970-01-18 12:03:08.456789 UTC', TIMESTAMP '1970-12-31 12:55:00.456789 UTC', 5, 8), " + "(45, 2, TIMESTAMP '2015-05-15 13:05:01.234567 UTC', TIMESTAMP '2015-09-15 14:21:02.345678 UTC', 9, 10), " + "(50, 2, TIMESTAMP '2020-02-21 15:11:11.876543 UTC', TIMESTAMP '2020-08-21 16:12:12.654321 UTC', 11, 12)"; - String expectedTimestampStats = "NULL, NULL, 0.0769231e0, NULL, '1968-03-15 15:13:12.876 UTC', '2020-08-21 16:12:12.654 UTC'"; - String expectedIntegerStats = "NULL, NULL, 0e0, NULL, '1', '101'"; + String expectedTimestampStats = "NULL, 12e0, 0.0769231e0, NULL, '1968-03-15 15:13:12.876 UTC', '2020-08-21 16:12:12.654 UTC'"; + String expectedIntegerStats = "NULL, 13e0, 0e0, NULL, '1', '101'"; if (format == ORC) { expected = "VALUES " + "(NULL, BIGINT '1', NULL, NULL, 101, 101), " + @@ -2859,8 +2859,8 @@ else if (format == AVRO) { "(0, 4, NULL, NULL, NULL, NULL), " + "(45, 2, NULL, NULL, NULL, NULL), " + "(50, 2, NULL, NULL, NULL, NULL)"; - expectedTimestampStats = "NULL, NULL, NULL, NULL, NULL, NULL"; - expectedIntegerStats = "NULL, NULL, NULL, NULL, NULL, NULL"; + expectedTimestampStats = "NULL, 12e0, 0.0769231e0, NULL, NULL, NULL"; + expectedIntegerStats = "NULL, 13e0, 0e0, NULL, NULL, NULL"; } assertThat(query("SELECT partition.d_year, record_count, data.d.min, data.d.max, data.b.min, data.b.max FROM \"test_year_transform_timestamptz$partitions\"")) @@ -2955,8 +2955,8 @@ public void testTruncateTextTransform() assertThat(query("SHOW STATS FOR test_truncate_text_transform")) .skippingTypesCheck() .matches("VALUES " + - " ('d', " + (format == PARQUET ? "550e0" : "NULL") + ", NULL, " + (format == AVRO ? "NULL" : "0.125e0") + ", NULL, NULL, NULL), " + - (format == AVRO ? " ('b', NULL, NULL, NULL, NULL, NULL, NULL), " : " ('b', NULL, NULL, 0e0, NULL, '1', '101'), ") + + " ('d', " + (format == PARQUET ? "550e0" : "NULL") + ", 7e0, " + (format == AVRO ? "0.1e0" : "0.125e0") + ", NULL, NULL, NULL), " + + " ('b', NULL, 8e0, 0e0, NULL, " + (format == AVRO ? "NULL, NULL" : "'1', '101'") + "), " + " (NULL, NULL, NULL, NULL, 8e0, NULL, NULL)"); assertThat(query("SELECT * FROM test_truncate_text_transform WHERE d IS NOT NULL")) @@ -3045,16 +3045,16 @@ public void testTruncateIntegerTransform(String dataType) assertThat(query("SHOW STATS FOR " + table)) .skippingTypesCheck() .matches("VALUES " + - " ('d', NULL, NULL, 0.0625e0, NULL, '-130', '123'), " + - " ('b', NULL, NULL, 0e0, NULL, '1', '101'), " + + " ('d', NULL, 15e0, 0.0625e0, NULL, '-130', '123'), " + + " ('b', NULL, 16e0, 0e0, NULL, '1', '101'), " + " (NULL, NULL, NULL, NULL, 16e0, NULL, NULL)"); } else { assertThat(query("SHOW STATS FOR " + table)) .skippingTypesCheck() .matches("VALUES " + - " ('d', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('b', NULL, NULL, NULL, NULL, NULL, NULL), " + + " ('d', NULL, 15e0, 0.0625e0, NULL, NULL, NULL), " + + " ('b', NULL, 16e0, 0e0, NULL, NULL, NULL), " + " (NULL, NULL, NULL, NULL, 16e0, NULL, NULL)"); } @@ -3127,16 +3127,16 @@ public void testTruncateDecimalTransform() assertThat(query("SHOW STATS FOR test_truncate_decimal_transform")) .skippingTypesCheck() .matches("VALUES " + - " ('d', NULL, NULL, 0.166667e0, NULL, '-0.05', '12.34'), " + - " ('b', NULL, NULL, 0e0, NULL, '1', '101'), " + + " ('d', NULL, 5e0, 0.166667e0, NULL, '-0.05', '12.34'), " + + " ('b', NULL, 6e0, 0e0, NULL, '1', '101'), " + " (NULL, NULL, NULL, NULL, 6e0, NULL, NULL)"); } else if (format == AVRO) { assertThat(query("SHOW STATS FOR test_truncate_decimal_transform")) .skippingTypesCheck() .matches("VALUES " + - " ('d', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('b', NULL, NULL, NULL, NULL, NULL, NULL), " + + " ('d', NULL, 5e0, 0.1e0, NULL, NULL, NULL), " + + " ('b', NULL, 6e0, 0e0, NULL, NULL, NULL), " + " (NULL, NULL, NULL, NULL, 6e0, NULL, NULL)"); } @@ -3200,7 +3200,7 @@ protected void testBucketTransformForType( .skippingTypesCheck() .projected(0, 2, 3, 4) // data size, min and max may vary between types .matches("VALUES " + - " ('d', NULL, " + (format == AVRO ? "NULL" : "0.25e0") + ", NULL), " + + " ('d', 3e0, " + (format == AVRO ? "0.1e0" : "0.25e0") + ", NULL), " + " (NULL, NULL, NULL, 4e0)"); assertThat(query("SELECT * FROM " + tableName + " WHERE d IS NULL")) @@ -3240,16 +3240,16 @@ public void testApplyFilterWithNonEmptyConstraintPredicate() String expected = switch (format) { case ORC -> "VALUES " + - " ('d', NULL, NULL, 0e0, NULL, NULL, NULL), " + - " ('b', NULL, NULL, 0e0, NULL, '1', '7'), " + + " ('d', NULL, 7e0, 0e0, NULL, NULL, NULL), " + + " ('b', NULL, 7e0, 0e0, NULL, '1', '7'), " + " (NULL, NULL, NULL, NULL, 7e0, NULL, NULL)"; case PARQUET -> "VALUES " + - " ('d', 364e0, NULL, 0e0, NULL, NULL, NULL), " + - " ('b', NULL, NULL, 0e0, NULL, '1', '7'), " + + " ('d', 364e0, 7e0, 0e0, NULL, NULL, NULL), " + + " ('b', NULL, 7e0, 0e0, NULL, '1', '7'), " + " (NULL, NULL, NULL, NULL, 7e0, NULL, NULL)"; case AVRO -> "VALUES " + - " ('d', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('b', NULL, NULL, NULL, NULL, NULL, NULL), " + + " ('d', NULL, 7e0, 0e0, NULL, NULL, NULL), " + + " ('b', NULL, 7e0, 0e0, NULL, NULL, NULL), " + " (NULL, NULL, NULL, NULL, 7e0, NULL, NULL)"; }; assertThat(query("SHOW STATS FOR test_apply_functional_constraint")) @@ -3301,16 +3301,16 @@ public void testVoidTransform() assertThat(query("SHOW STATS FOR test_void_transform")) .skippingTypesCheck() .matches("VALUES " + - " ('d', " + (format == PARQUET ? "205e0" : "NULL") + ", NULL, 0.2857142857142857, NULL, NULL, NULL), " + - " ('b', NULL, NULL, 0e0, NULL, '1', '7'), " + + " ('d', " + (format == PARQUET ? "205e0" : "NULL") + ", 5e0, 0.2857142857142857, NULL, NULL, NULL), " + + " ('b', NULL, 7e0, 0e0, NULL, '1', '7'), " + " (NULL, NULL, NULL, NULL, 7e0, NULL, NULL)"); } else { assertThat(query("SHOW STATS FOR test_void_transform")) .skippingTypesCheck() .matches("VALUES " + - " ('d', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('b', NULL, NULL, NULL, NULL, NULL, NULL), " + + " ('d', NULL, 5e0, 0.1e0, NULL, NULL, NULL), " + + " ('b', NULL, 7e0, 0e0, NULL, NULL, NULL), " + " (NULL, NULL, NULL, NULL, 7e0, NULL, NULL)"); } @@ -3411,13 +3411,13 @@ public void testBasicTableStatistics() MaterializedResult result = computeActual("SHOW STATS FOR " + tableName); MaterializedResult expectedStatistics = resultBuilder(getSession(), VARCHAR, DOUBLE, DOUBLE, DOUBLE, DOUBLE, VARCHAR, VARCHAR) - .row("col", null, null, 0.0, null, "-10.0", "100.0") + .row("col", null, 2.0, 0.0, null, "-10.0", "100.0") .row(null, null, null, null, 2.0, null, null) .build(); if (format == AVRO) { expectedStatistics = resultBuilder(getSession(), VARCHAR, DOUBLE, DOUBLE, DOUBLE, DOUBLE, VARCHAR, VARCHAR) - .row("col", null, null, null, null, null, null) + .row("col", null, 2.0, 0.0, null, null, null) .row(null, null, null, null, 2.0, null, null) .build(); } @@ -3428,13 +3428,13 @@ public void testBasicTableStatistics() result = computeActual("SHOW STATS FOR " + tableName); expectedStatistics = resultBuilder(getSession(), VARCHAR, DOUBLE, DOUBLE, DOUBLE, DOUBLE, VARCHAR, VARCHAR) - .row("col", null, null, 0.0, null, "-10.0", "200.0") + .row("col", null, 3.0, 0.0, null, "-10.0", "200.0") .row(null, null, null, null, 3.0, null, null) .build(); if (format == AVRO) { expectedStatistics = resultBuilder(getSession(), VARCHAR, DOUBLE, DOUBLE, DOUBLE, DOUBLE, VARCHAR, VARCHAR) - .row("col", null, null, null, null, null, null) + .row("col", null, 3.0, 0.0, null, null, null) .row(null, null, null, null, 3.0, null, null) .build(); } @@ -3482,8 +3482,7 @@ public void testBasicAnalyze() " ('comment', " + (format == PARQUET ? "639e0" : "NULL") + ", 5e0, 0e0, NULL, NULL, NULL), " + " (NULL, NULL, NULL, NULL, 5e0, NULL, NULL)"); - // initially, no NDV information - assertThat(query(defaultSession, "SHOW STATS FOR " + tableName)).skippingTypesCheck().matches(statsWithoutNdv); + assertThat(query(defaultSession, "SHOW STATS FOR " + tableName)).skippingTypesCheck().matches(statsWithNdv); assertThat(query(extendedStatisticsDisabled, "SHOW STATS FOR " + tableName)).skippingTypesCheck().matches(statsWithoutNdv); // ANALYZE can be disabled. @@ -3518,16 +3517,16 @@ public void testMultipleColumnTableStatistics() MaterializedResult expectedStatistics = resultBuilder(getSession(), VARCHAR, DOUBLE, DOUBLE, DOUBLE, DOUBLE, VARCHAR, VARCHAR) - .row("col1", null, null, 0.0, null, "-10.0", "100.0") - .row("col2", null, null, 0.0, null, "-1", "10") - .row("col3", null, null, 0.0, null, "2019-06-28", "2020-01-01") + .row("col1", null, 2.0, 0.0, null, "-10.0", "100.0") + .row("col2", null, 2.0, 0.0, null, "-1", "10") + .row("col3", null, 2.0, 0.0, null, "2019-06-28", "2020-01-01") .row(null, null, null, null, 2.0, null, null) .build(); if (format == AVRO) { expectedStatistics = resultBuilder(getSession(), VARCHAR, DOUBLE, DOUBLE, DOUBLE, DOUBLE, VARCHAR, VARCHAR) - .row("col1", null, null, null, null, null, null) - .row("col2", null, null, null, null, null, null) - .row("col3", null, null, null, null, null, null) + .row("col1", null, 2.0, 0.0, null, null, null) + .row("col2", null, 2.0, 0.0, null, null, null) + .row("col3", null, 2.0, 0.0, null, null, null) .row(null, null, null, null, 2.0, null, null) .build(); } @@ -3537,17 +3536,17 @@ public void testMultipleColumnTableStatistics() result = computeActual("SHOW STATS FOR " + tableName); expectedStatistics = resultBuilder(getSession(), VARCHAR, DOUBLE, DOUBLE, DOUBLE, DOUBLE, VARCHAR, VARCHAR) - .row("col1", null, null, 0.0, null, "-10.0", "200.0") - .row("col2", null, null, 0.0, null, "-1", "20") - .row("col3", null, null, 0.0, null, "2019-06-28", "2020-06-28") + .row("col1", null, 3.0, 0.0, null, "-10.0", "200.0") + .row("col2", null, 3.0, 0.0, null, "-1", "20") + .row("col3", null, 3.0, 0.0, null, "2019-06-28", "2020-06-28") .row(null, null, null, null, 3.0, null, null) .build(); if (format == AVRO) { expectedStatistics = resultBuilder(getSession(), VARCHAR, DOUBLE, DOUBLE, DOUBLE, DOUBLE, VARCHAR, VARCHAR) - .row("col1", null, null, null, null, null, null) - .row("col2", null, null, null, null, null, null) - .row("col3", null, null, null, null, null, null) + .row("col1", null, 3.0, 0.0, null, null, null) + .row("col2", null, 3.0, 0.0, null, null, null) + .row("col3", null, 3.0, 0.0, null, null, null) .row(null, null, null, null, 3.0, null, null) .build(); } @@ -3565,17 +3564,17 @@ public void testMultipleColumnTableStatistics() expectedStatistics = resultBuilder(getSession(), VARCHAR, DOUBLE, DOUBLE, DOUBLE, DOUBLE, VARCHAR, VARCHAR) - .row("col1", null, null, 5.0 / 13.0, null, "-10.0", "200.0") - .row("col2", null, null, 0.0, null, "-1", "30") - .row("col3", null, null, 0.0, null, "2019-06-28", "2020-07-25") + .row("col1", null, 3.0, 5.0 / 13.0, null, "-10.0", "200.0") + .row("col2", null, 13.0, 0.0, null, "-1", "30") + .row("col3", null, 12.0, 0.0, null, "2019-06-28", "2020-07-25") .row(null, null, null, null, 13.0, null, null) .build(); if (format == AVRO) { expectedStatistics = resultBuilder(getSession(), VARCHAR, DOUBLE, DOUBLE, DOUBLE, DOUBLE, VARCHAR, VARCHAR) - .row("col1", null, null, null, null, null, null) - .row("col2", null, null, null, null, null, null) - .row("col3", null, null, null, null, null, null) + .row("col1", null, 3.0, 0.1, null, null, null) + .row("col2", null, 13.0, 0.0, null, null, null) + .row("col3", null, 12.0, 1.0 / 13.0, null, null, null) .row(null, null, null, null, 13.0, null, null) .build(); } @@ -3597,26 +3596,24 @@ public void testPartitionedTableStatistics() MaterializedRow row0 = result.getMaterializedRows().get(0); assertEquals(row0.getField(0), "col1"); + assertEquals(row0.getField(3), 0.0); if (format != AVRO) { - assertEquals(row0.getField(3), 0.0); assertEquals(row0.getField(5), "-10.0"); assertEquals(row0.getField(6), "100.0"); } else { - assertNull(row0.getField(3)); assertNull(row0.getField(5)); assertNull(row0.getField(6)); } MaterializedRow row1 = result.getMaterializedRows().get(1); assertEquals(row1.getField(0), "col2"); + assertEquals(row1.getField(3), 0.0); if (format != AVRO) { - assertEquals(row1.getField(3), 0.0); assertEquals(row1.getField(5), "-1"); assertEquals(row1.getField(6), "10"); } else { - assertNull(row0.getField(3)); assertNull(row0.getField(5)); assertNull(row0.getField(6)); } @@ -3637,12 +3634,12 @@ public void testPartitionedTableStatistics() row0 = result.getMaterializedRows().get(0); assertEquals(row0.getField(0), "col1"); if (format != AVRO) { - assertEquals(row0.getField(3), 5.0 / 12.0); + assertEquals((double) row0.getField(3), 5.0 / 12.0, 1e-10); assertEquals(row0.getField(5), "-10.0"); assertEquals(row0.getField(6), "105.0"); } else { - assertNull(row0.getField(3)); + assertEquals(row0.getField(3), 0.1); assertNull(row0.getField(5)); assertNull(row0.getField(6)); } @@ -3655,7 +3652,7 @@ public void testPartitionedTableStatistics() assertEquals(row1.getField(6), "10"); } else { - assertNull(row0.getField(3)); + assertEquals(row0.getField(3), 0.1); assertNull(row0.getField(5)); assertNull(row0.getField(6)); } @@ -3676,7 +3673,7 @@ public void testPartitionedTableStatistics() assertEquals(row0.getField(6), "105.0"); } else { - assertNull(row0.getField(3)); + assertEquals(row0.getField(3), 0.1); assertNull(row0.getField(5)); assertNull(row0.getField(6)); } @@ -3689,7 +3686,7 @@ public void testPartitionedTableStatistics() assertEquals(row1.getField(6), "10"); } else { - assertNull(row0.getField(3)); + assertEquals(row0.getField(3), 0.1); assertNull(row0.getField(5)); assertNull(row0.getField(6)); } @@ -3910,40 +3907,40 @@ public void testCreateNestedPartitionedTable() assertThat(query("SHOW STATS FOR test_nested_table_1")) .skippingTypesCheck() .matches("VALUES " + - " ('bool', NULL, NULL, 0e0, NULL, 'true', 'true'), " + - " ('int', NULL, NULL, 0e0, NULL, '1', '1'), " + + " ('bool', NULL, 1e0, 0e0, NULL, 'true', 'true'), " + + " ('int', NULL, 1e0, 0e0, NULL, '1', '1'), " + " ('arr', NULL, NULL, " + (format == ORC ? "0e0" : "NULL") + ", NULL, NULL, NULL), " + - " ('big', NULL, NULL, 0e0, NULL, '1', '1'), " + - " ('rl', NULL, NULL, 0e0, NULL, '1.0', '1.0'), " + - " ('dbl', NULL, NULL, 0e0, NULL, '1.0', '1.0'), " + + " ('big', NULL, 1e0, 0e0, NULL, '1', '1'), " + + " ('rl', NULL, 1e0, 0e0, NULL, '1.0', '1.0'), " + + " ('dbl', NULL, 1e0, 0e0, NULL, '1.0', '1.0'), " + " ('mp', NULL, NULL, " + (format == ORC ? "0e0" : "NULL") + ", NULL, NULL, NULL), " + - " ('dec', NULL, NULL, 0e0, NULL, '1.0', '1.0'), " + - " ('vc', " + (format == PARQUET ? "116e0" : "NULL") + ", NULL, 0e0, NULL, NULL, NULL), " + - " ('vb', " + (format == PARQUET ? "77e0" : "NULL") + ", NULL, 0e0, NULL, NULL, NULL), " + - " ('ts', NULL, NULL, 0e0, NULL, '2021-07-24 02:43:57.348000', " + (format == ORC ? "'2021-07-24 02:43:57.348999'" : "'2021-07-24 02:43:57.348000'") + "), " + - " ('tstz', NULL, NULL, 0e0, NULL, '2021-07-24 02:43:57.348 UTC', '2021-07-24 02:43:57.348 UTC'), " + + " ('dec', NULL, 1e0, 0e0, NULL, '1.0', '1.0'), " + + " ('vc', " + (format == PARQUET ? "116e0" : "NULL") + ", 1e0, 0e0, NULL, NULL, NULL), " + + " ('vb', " + (format == PARQUET ? "77e0" : "NULL") + ", 1e0, 0e0, NULL, NULL, NULL), " + + " ('ts', NULL, 1e0, 0e0, NULL, '2021-07-24 02:43:57.348000', " + (format == ORC ? "'2021-07-24 02:43:57.348999'" : "'2021-07-24 02:43:57.348000'") + "), " + + " ('tstz', NULL, 1e0, 0e0, NULL, '2021-07-24 02:43:57.348 UTC', '2021-07-24 02:43:57.348 UTC'), " + " ('str', NULL, NULL, " + (format == ORC ? "0e0" : "NULL") + ", NULL, NULL, NULL), " + - " ('dt', NULL, NULL, 0e0, NULL, '2021-07-24', '2021-07-24'), " + + " ('dt', NULL, 1e0, 0e0, NULL, '2021-07-24', '2021-07-24'), " + " (NULL, NULL, NULL, NULL, 1e0, NULL, NULL)"); } else { assertThat(query("SHOW STATS FOR test_nested_table_1")) .skippingTypesCheck() .matches("VALUES " + - " ('bool', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('int', NULL, NULL, 0e0, NULL, '1', '1'), " + + " ('bool', NULL, 1e0, 0e0, NULL, NULL, NULL), " + + " ('int', NULL, 1e0, 0e0, NULL, '1', '1'), " + " ('arr', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('big', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('rl', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('dbl', NULL, NULL, NULL, NULL, NULL, NULL), " + + " ('big', NULL, 1e0, 0e0, NULL, NULL, NULL), " + + " ('rl', NULL, 1e0, 0e0, NULL, NULL, NULL), " + + " ('dbl', NULL, 1e0, 0e0, NULL, NULL, NULL), " + " ('mp', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('dec', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('vc', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('vb', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('ts', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('tstz', NULL, NULL, NULL, NULL, NULL, NULL), " + + " ('dec', NULL, 1e0, 0e0, NULL, NULL, NULL), " + + " ('vc', NULL, 1e0, 0e0, NULL, NULL, NULL), " + + " ('vb', NULL, 1e0, 0e0, NULL, NULL, NULL), " + + " ('ts', NULL, 1e0, 0e0, NULL, NULL, NULL), " + + " ('tstz', NULL, 1e0, 0e0, NULL, NULL, NULL), " + " ('str', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('dt', NULL, NULL, NULL, NULL, NULL, NULL), " + + " ('dt', NULL, 1e0, 0e0, NULL, NULL, NULL), " + " (NULL, NULL, NULL, NULL, 1e0, NULL, NULL)"); } @@ -3974,14 +3971,14 @@ public void testCreateNestedPartitionedTable() assertThat(query("SHOW STATS FOR test_nested_table_2")) .skippingTypesCheck() .matches("VALUES " + - " ('int', NULL, NULL, 0e0, NULL, '1', '1'), " + + " ('int', NULL, 1e0, 0e0, NULL, '1', '1'), " + " ('arr', NULL, NULL, " + (format == ORC ? "0e0" : "NULL") + ", NULL, NULL, NULL), " + - " ('big', NULL, NULL, 0e0, NULL, '1', '1'), " + - " ('rl', NULL, NULL, 0e0, NULL, '1.0', '1.0'), " + - " ('dbl', NULL, NULL, 0e0, NULL, '1.0', '1.0'), " + + " ('big', NULL, 1e0, 0e0, NULL, '1', '1'), " + + " ('rl', NULL, 1e0, 0e0, NULL, '1.0', '1.0'), " + + " ('dbl', NULL, 1e0, 0e0, NULL, '1.0', '1.0'), " + " ('mp', NULL, NULL, " + (format == ORC ? "0e0" : "NULL") + ", NULL, NULL, NULL), " + - " ('dec', NULL, NULL, 0e0, NULL, '1.0', '1.0'), " + - " ('vc', " + (format == PARQUET ? "116e0" : "NULL") + ", NULL, 0e0, NULL, NULL, NULL), " + + " ('dec', NULL, 1e0, 0e0, NULL, '1.0', '1.0'), " + + " ('vc', " + (format == PARQUET ? "116e0" : "NULL") + ", 1e0, 0e0, NULL, NULL, NULL), " + " ('str', NULL, NULL, " + (format == ORC ? "0e0" : "NULL") + ", NULL, NULL, NULL), " + " (NULL, NULL, NULL, NULL, 1e0, NULL, NULL)"); } @@ -3989,14 +3986,14 @@ public void testCreateNestedPartitionedTable() assertThat(query("SHOW STATS FOR test_nested_table_2")) .skippingTypesCheck() .matches("VALUES " + - " ('int', NULL, NULL, 0e0, NULL, '1', '1'), " + + " ('int', NULL, 1e0, 0e0, NULL, '1', '1'), " + " ('arr', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('big', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('rl', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('dbl', NULL, NULL, NULL, NULL, NULL, NULL), " + + " ('big', NULL, 1e0, 0e0, NULL, NULL, NULL), " + + " ('rl', NULL, 1e0, 0e0, NULL, NULL, NULL), " + + " ('dbl', NULL, 1e0, 0e0, NULL, NULL, NULL), " + " ('mp', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('dec', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('vc', NULL, NULL, NULL, NULL, NULL, NULL), " + + " ('dec', NULL, 1e0, 0e0, NULL, NULL, NULL), " + + " ('vc', NULL, 1e0, 0e0, NULL, NULL, NULL), " + " ('str', NULL, NULL, NULL, NULL, NULL, NULL), " + " (NULL, NULL, NULL, NULL, 1e0, NULL, NULL)"); } @@ -4286,20 +4283,20 @@ public void testAllAvailableTypes() assertThat(query("SHOW STATS FOR test_all_types")) .skippingTypesCheck() .matches("VALUES " + - " ('a_boolean', NULL, NULL, 0.5e0, NULL, 'true', 'true'), " + - " ('an_integer', NULL, NULL, 0.5e0, NULL, '1', '1'), " + - " ('a_bigint', NULL, NULL, 0.5e0, NULL, '1', '1'), " + - " ('a_real', NULL, NULL, 0.5e0, NULL, '1.0', '1.0'), " + - " ('a_double', NULL, NULL, 0.5e0, NULL, '1.0', '1.0'), " + - " ('a_short_decimal', NULL, NULL, 0.5e0, NULL, '1.0', '1.0'), " + - " ('a_long_decimal', NULL, NULL, 0.5e0, NULL, '11.0', '11.0'), " + - " ('a_varchar', " + (format == PARQUET ? "234e0" : "NULL") + ", NULL, 0.5e0, NULL, NULL, NULL), " + - " ('a_varbinary', " + (format == PARQUET ? "114e0" : "NULL") + ", NULL, 0.5e0, NULL, NULL, NULL), " + - " ('a_date', NULL, NULL, 0.5e0, NULL, '2021-07-24', '2021-07-24'), " + - " ('a_time', NULL, NULL, 0.5e0, NULL, NULL, NULL), " + - " ('a_timestamp', NULL, NULL, 0.5e0, NULL, " + (format == ORC ? "'2021-07-24 03:43:57.987000', '2021-07-24 03:43:57.987999'" : "'2021-07-24 03:43:57.987654', '2021-07-24 03:43:57.987654'") + "), " + - " ('a_timestamptz', NULL, NULL, 0.5e0, NULL, '2021-07-24 04:43:57.987 UTC', '2021-07-24 04:43:57.987 UTC'), " + - " ('a_uuid', NULL, NULL, 0.5e0, NULL, NULL, NULL), " + + " ('a_boolean', NULL, 1e0, 0.5e0, NULL, 'true', 'true'), " + + " ('an_integer', NULL, 1e0, 0.5e0, NULL, '1', '1'), " + + " ('a_bigint', NULL, 1e0, 0.5e0, NULL, '1', '1'), " + + " ('a_real', NULL, 1e0, 0.5e0, NULL, '1.0', '1.0'), " + + " ('a_double', NULL, 1e0, 0.5e0, NULL, '1.0', '1.0'), " + + " ('a_short_decimal', NULL, 1e0, 0.5e0, NULL, '1.0', '1.0'), " + + " ('a_long_decimal', NULL, 1e0, 0.5e0, NULL, '11.0', '11.0'), " + + " ('a_varchar', " + (format == PARQUET ? "234e0" : "NULL") + ", 1e0, 0.5e0, NULL, NULL, NULL), " + + " ('a_varbinary', " + (format == PARQUET ? "114e0" : "NULL") + ", 1e0, 0.5e0, NULL, NULL, NULL), " + + " ('a_date', NULL, 1e0, 0.5e0, NULL, '2021-07-24', '2021-07-24'), " + + " ('a_time', NULL, 1e0, 0.5e0, NULL, NULL, NULL), " + + " ('a_timestamp', NULL, 1e0, 0.5e0, NULL, " + (format == ORC ? "'2021-07-24 03:43:57.987000', '2021-07-24 03:43:57.987999'" : "'2021-07-24 03:43:57.987654', '2021-07-24 03:43:57.987654'") + "), " + + " ('a_timestamptz', NULL, 1e0, 0.5e0, NULL, '2021-07-24 04:43:57.987 UTC', '2021-07-24 04:43:57.987 UTC'), " + + " ('a_uuid', NULL, 1e0, 0.5e0, NULL, NULL, NULL), " + " ('a_row', NULL, NULL, " + (format == ORC ? "0.5" : "NULL") + ", NULL, NULL, NULL), " + " ('an_array', NULL, NULL, " + (format == ORC ? "0.5" : "NULL") + ", NULL, NULL, NULL), " + " ('a_map', NULL, NULL, " + (format == ORC ? "0.5" : "NULL") + ", NULL, NULL, NULL), " + @@ -4309,20 +4306,20 @@ public void testAllAvailableTypes() assertThat(query("SHOW STATS FOR test_all_types")) .skippingTypesCheck() .matches("VALUES " + - " ('a_boolean', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('an_integer', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('a_bigint', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('a_real', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('a_double', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('a_short_decimal', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('a_long_decimal', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('a_varchar', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('a_varbinary', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('a_date', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('a_time', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('a_timestamp', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('a_timestamptz', NULL, NULL, NULL, NULL, NULL, NULL), " + - " ('a_uuid', NULL, NULL, NULL, NULL, NULL, NULL), " + + " ('a_boolean', NULL, 1e0, 0.1e0, NULL, NULL, NULL), " + + " ('an_integer', NULL, 1e0, 0.1e0, NULL, NULL, NULL), " + + " ('a_bigint', NULL, 1e0, 0.1e0, NULL, NULL, NULL), " + + " ('a_real', NULL, 1e0, 0.1e0, NULL, NULL, NULL), " + + " ('a_double', NULL, 1e0, 0.1e0, NULL, NULL, NULL), " + + " ('a_short_decimal', NULL, 1e0, 0.1e0, NULL, NULL, NULL), " + + " ('a_long_decimal', NULL, 1e0, 0.1e0, NULL, NULL, NULL), " + + " ('a_varchar', NULL, 1e0, 0.1e0, NULL, NULL, NULL), " + + " ('a_varbinary', NULL, 1e0, 0.1e0, NULL, NULL, NULL), " + + " ('a_date', NULL, 1e0, 0.1e0, NULL, NULL, NULL), " + + " ('a_time', NULL, 1e0, 0.1e0, NULL, NULL, NULL), " + + " ('a_timestamp', NULL, 1e0, 0.1e0, NULL, NULL, NULL), " + + " ('a_timestamptz', NULL, 1e0, 0.1e0, NULL, NULL, NULL), " + + " ('a_uuid', NULL, 1e0, 0.1e0, NULL, NULL, NULL), " + " ('a_row', NULL, NULL, NULL, NULL, NULL, NULL), " + " ('an_array', NULL, NULL, NULL, NULL, NULL, NULL), " + " ('a_map', NULL, NULL, NULL, NULL, NULL, NULL), " + @@ -5235,7 +5232,7 @@ private List getActiveFiles(String tableName) .collect(toImmutableList()); } - private String getTableLocation(String tableName) + protected String getTableLocation(String tableName) { Pattern locationPattern = Pattern.compile(".*location = '(.*?)'.*", Pattern.DOTALL); Matcher m = locationPattern.matcher((String) computeActual("SHOW CREATE TABLE " + tableName).getOnlyValue()); @@ -5715,7 +5712,7 @@ private void testCleaningUpWithTableWithSpecifiedLocation(String suffix) List prunedMetadataFiles = getAllMetadataFilesFromTableDirectory(tableDirectory); List prunedSnapshots = getSnapshotIds(tableName); assertThat(prunedMetadataFiles).as("prunedMetadataFiles") - .hasSize(initialMetadataFiles.size() - 1); + .hasSize(initialMetadataFiles.size() - 3); assertThat(prunedSnapshots).as("prunedSnapshots") .hasSizeLessThan(initialSnapshots.size()) .hasSize(1); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConfig.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConfig.java index e5688ce70007..822337eb3892 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConfig.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConfig.java @@ -50,6 +50,7 @@ public void testDefaults() .setDynamicFilteringWaitTimeout(new Duration(0, MINUTES)) .setTableStatisticsEnabled(true) .setExtendedStatisticsEnabled(true) + .setCollectExtendedStatisticsOnWrite(true) .setProjectionPushdownEnabled(true) .setHiveCatalogName(null) .setFormatVersion(2) @@ -76,6 +77,7 @@ public void testExplicitPropertyMappings() .put("iceberg.dynamic-filtering.wait-timeout", "1h") .put("iceberg.table-statistics-enabled", "false") .put("iceberg.extended-statistics.enabled", "false") + .put("iceberg.extended-statistics.collect-on-write", "false") .put("iceberg.projection-pushdown-enabled", "false") .put("iceberg.hive-catalog-name", "hive") .put("iceberg.format-version", "1") @@ -99,6 +101,7 @@ public void testExplicitPropertyMappings() .setDynamicFilteringWaitTimeout(Duration.valueOf("1h")) .setTableStatisticsEnabled(false) .setExtendedStatisticsEnabled(false) + .setCollectExtendedStatisticsOnWrite(false) .setProjectionPushdownEnabled(false) .setHiveCatalogName("hive") .setFormatVersion(1) diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergGetTableStatisticsOperations.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergGetTableStatisticsOperations.java index 9780aa764cdd..e1984e9795b4 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergGetTableStatisticsOperations.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergGetTableStatisticsOperations.java @@ -17,6 +17,7 @@ import com.google.common.collect.ImmutableMultiset; import io.trino.execution.warnings.WarningCollector; import io.trino.metadata.CountingAccessMetadata; +import io.trino.metadata.InternalFunctionBundle; import io.trino.metadata.MetadataManager; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.HiveMetastore; @@ -67,6 +68,10 @@ protected QueryRunner createQueryRunner() localQueryRunner.installPlugin(new TpchPlugin()); localQueryRunner.createCatalog("tpch", "tpch", ImmutableMap.of()); + InternalFunctionBundle.InternalFunctionBundleBuilder functions = InternalFunctionBundle.builder(); + new IcebergPlugin().getFunctions().forEach(functions::functions); + localQueryRunner.addFunctions(functions.build()); + metastoreDir = Files.createTempDirectory("test_iceberg_get_table_statistics_operations").toFile(); HiveMetastore metastore = createTestingFileHiveMetastore(metastoreDir); localQueryRunner.createCatalog( diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataFileOperations.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataFileOperations.java index 0df06a5ff935..2a97dfa4709f 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataFileOperations.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataFileOperations.java @@ -38,10 +38,12 @@ import static io.trino.SystemSessionProperties.MIN_INPUT_SIZE_PER_TASK; import static io.trino.plugin.hive.HiveTestUtils.HDFS_ENVIRONMENT; import static io.trino.plugin.hive.metastore.file.FileHiveMetastore.createTestingFileHiveMetastore; +import static io.trino.plugin.iceberg.IcebergSessionProperties.COLLECT_EXTENDED_STATISTICS_ON_WRITE; import static io.trino.plugin.iceberg.TestIcebergMetadataFileOperations.FileType.DATA; import static io.trino.plugin.iceberg.TestIcebergMetadataFileOperations.FileType.MANIFEST; import static io.trino.plugin.iceberg.TestIcebergMetadataFileOperations.FileType.METADATA_JSON; import static io.trino.plugin.iceberg.TestIcebergMetadataFileOperations.FileType.SNAPSHOT; +import static io.trino.plugin.iceberg.TestIcebergMetadataFileOperations.FileType.STATS; import static io.trino.plugin.iceberg.TestIcebergMetadataFileOperations.FileType.fromFilePath; import static io.trino.plugin.iceberg.TrackingFileSystemFactory.OperationType.INPUT_FILE_GET_LENGTH; import static io.trino.plugin.iceberg.TrackingFileSystemFactory.OperationType.INPUT_FILE_NEW_STREAM; @@ -120,7 +122,9 @@ public void testCreateTable() @Test public void testCreateTableAsSelect() { - assertFileSystemAccesses("CREATE TABLE test_create_as_select AS SELECT 1 col_name", + assertFileSystemAccesses( + withStatsOnWrite(getSession(), false), + "CREATE TABLE test_create_as_select AS SELECT 1 col_name", ImmutableMultiset.builder() .addCopies(new FileOperation(MANIFEST, OUTPUT_FILE_CREATE_OR_OVERWRITE), 1) .addCopies(new FileOperation(MANIFEST, OUTPUT_FILE_LOCATION), 1) @@ -131,6 +135,22 @@ public void testCreateTableAsSelect() .addCopies(new FileOperation(SNAPSHOT, OUTPUT_FILE_CREATE_OR_OVERWRITE), 1) .addCopies(new FileOperation(SNAPSHOT, OUTPUT_FILE_LOCATION), 2) .build()); + + assertFileSystemAccesses( + withStatsOnWrite(getSession(), true), + "CREATE TABLE test_create_as_select_with_stats AS SELECT 1 col_name", + ImmutableMultiset.builder() + .addCopies(new FileOperation(MANIFEST, OUTPUT_FILE_CREATE_OR_OVERWRITE), 1) + .addCopies(new FileOperation(MANIFEST, OUTPUT_FILE_LOCATION), 1) + .addCopies(new FileOperation(METADATA_JSON, INPUT_FILE_NEW_STREAM), 2) + .addCopies(new FileOperation(METADATA_JSON, OUTPUT_FILE_CREATE), 2) // TODO (https://github.com/trinodb/trino/issues/15439): it would be good to publish data and stats in one commit + .addCopies(new FileOperation(METADATA_JSON, OUTPUT_FILE_LOCATION), 2) + .addCopies(new FileOperation(SNAPSHOT, INPUT_FILE_GET_LENGTH), 2) + .addCopies(new FileOperation(SNAPSHOT, INPUT_FILE_NEW_STREAM), 2) + .addCopies(new FileOperation(SNAPSHOT, OUTPUT_FILE_CREATE_OR_OVERWRITE), 1) + .addCopies(new FileOperation(SNAPSHOT, OUTPUT_FILE_LOCATION), 2) + .addCopies(new FileOperation(STATS, OUTPUT_FILE_CREATE), 1) + .build()); } @Test @@ -449,6 +469,14 @@ private long getLatestSnapshotId(String tableName) return (long) computeScalar(format("SELECT snapshot_id FROM \"%s$snapshots\" ORDER BY committed_at DESC FETCH FIRST 1 ROW WITH TIES", tableName)); } + private static Session withStatsOnWrite(Session session, boolean enabled) + { + String catalog = session.getCatalog().orElseThrow(); + return Session.builder(session) + .setCatalogSessionProperty(catalog, COLLECT_EXTENDED_STATISTICS_ON_WRITE, Boolean.toString(enabled)) + .build(); + } + static class FileOperation { private final FileType fileType; @@ -505,6 +533,7 @@ enum FileType METADATA_JSON, MANIFEST, SNAPSHOT, + STATS, DATA, /**/; @@ -519,6 +548,9 @@ public static FileType fromFilePath(String path) if (path.endsWith("-m0.avro")) { return MANIFEST; } + if (path.endsWith(".stats")) { + return STATS; + } if (path.contains("/data/") && path.endsWith(".orc")) { return DATA; } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetastoreAccessOperations.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetastoreAccessOperations.java index 87ab9cc63a58..b8f2a0075347 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetastoreAccessOperations.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetastoreAccessOperations.java @@ -37,6 +37,7 @@ import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Methods.GET_TABLE; import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Methods.REPLACE_TABLE; import static io.trino.plugin.hive.metastore.file.FileHiveMetastore.createTestingFileHiveMetastore; +import static io.trino.plugin.iceberg.IcebergSessionProperties.COLLECT_EXTENDED_STATISTICS_ON_WRITE; import static io.trino.plugin.iceberg.TableType.DATA; import static io.trino.plugin.iceberg.TableType.FILES; import static io.trino.plugin.iceberg.TableType.HISTORY; @@ -106,12 +107,24 @@ public void testCreateTable() @Test public void testCreateTableAsSelect() { - assertMetastoreInvocations("CREATE TABLE test_ctas AS SELECT 1 AS age", + assertMetastoreInvocations( + withStatsOnWrite(getSession(), false), + "CREATE TABLE test_ctas AS SELECT 1 AS age", ImmutableMultiset.builder() .add(GET_DATABASE) .add(CREATE_TABLE) .add(GET_TABLE) .build()); + + assertMetastoreInvocations( + withStatsOnWrite(getSession(), true), + "CREATE TABLE test_ctas_with_stats AS SELECT 1 AS age", + ImmutableMultiset.builder() + .add(GET_DATABASE) + .add(CREATE_TABLE) + .addCopies(GET_TABLE, 5) + .add(REPLACE_TABLE) + .build()); } @Test @@ -344,4 +357,12 @@ private void assertMetastoreInvocations(Session session, @Language("SQL") String fail("Expected: \n\t\t" + join(",\n\t\t", mismatchReport)); } + + private static Session withStatsOnWrite(Session session, boolean enabled) + { + String catalog = session.getCatalog().orElseThrow(); + return Session.builder(session) + .setCatalogSessionProperty(catalog, COLLECT_EXTENDED_STATISTICS_ON_WRITE, Boolean.toString(enabled)) + .build(); + } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergProjectionPushdownPlans.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergProjectionPushdownPlans.java index 59eefbdb779b..a488779f52d7 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergProjectionPushdownPlans.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergProjectionPushdownPlans.java @@ -17,6 +17,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import io.trino.Session; +import io.trino.metadata.InternalFunctionBundle; import io.trino.metadata.QualifiedObjectName; import io.trino.metadata.TableHandle; import io.trino.plugin.hive.metastore.Database; @@ -82,6 +83,10 @@ protected LocalQueryRunner createLocalQueryRunner() HiveMetastore metastore = createTestingFileHiveMetastore(metastoreDir); LocalQueryRunner queryRunner = LocalQueryRunner.create(session); + InternalFunctionBundle.InternalFunctionBundleBuilder functions = InternalFunctionBundle.builder(); + new IcebergPlugin().getFunctions().forEach(functions::functions); + queryRunner.addFunctions(functions.build()); + queryRunner.createCatalog( CATALOG, new TestingIcebergConnectorFactory(Optional.of(new TestingIcebergFileMetastoreCatalogModule(metastore)), Optional.empty(), EMPTY_MODULE), diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergStatistics.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergStatistics.java index 5bd25797dd6d..0a30bb9582cb 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergStatistics.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergStatistics.java @@ -15,9 +15,14 @@ import io.trino.Session; import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.DataProviders; import io.trino.testing.QueryRunner; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; +import static io.trino.plugin.iceberg.IcebergSessionProperties.COLLECT_EXTENDED_STATISTICS_ON_WRITE; +import static io.trino.testing.DataProviders.cartesianProduct; +import static io.trino.testing.DataProviders.trueFalse; import static io.trino.testing.TestingAccessControlManager.TestingPrivilegeType.EXECUTE_TABLE_PROCEDURE; import static io.trino.testing.TestingAccessControlManager.privilege; import static io.trino.testing.TestingNames.randomNameSuffix; @@ -40,84 +45,84 @@ protected QueryRunner createQueryRunner() .build(); } - @Test - public void testAnalyze() + @Test(dataProviderClass = DataProviders.class, dataProvider = "trueFalse") + public void testAnalyze(boolean collectOnStatsOnWrites) { - String tableName = "test_analyze"; - assertUpdate("CREATE TABLE " + tableName + " AS SELECT * FROM tpch.sf1.nation", 25); + Session writeSession = withStatsOnWrite(getSession(), collectOnStatsOnWrites); + String tableName = "test_analyze_" + collectOnStatsOnWrites; - assertQuery( - "SHOW STATS FOR " + tableName, - """ - VALUES - ('nationkey', null, null, 0, null, '0', '24'), - ('regionkey', null, null, 0, null, '0', '4'), - ('comment', null, null, 0, null, null, null), - ('name', null, null, 0, null, null, null), - (null, null, null, null, 25, null, null)"""); + assertUpdate(writeSession, "CREATE TABLE " + tableName + " AS SELECT * FROM tpch.sf1.nation", 25); + String goodStatsInitial = """ + VALUES + ('nationkey', null, 25, 0, null, '0', '24'), + ('regionkey', null, 5, 0, null, '0', '4'), + ('comment', null, 25, 0, null, null, null), + ('name', null, 25, 0, null, null, null), + (null, null, null, null, 25, null, null)"""; + + if (collectOnStatsOnWrites) { + assertQuery("SHOW STATS FOR " + tableName, goodStatsInitial); + } + else { + assertQuery( + "SHOW STATS FOR " + tableName, + """ + VALUES + ('nationkey', null, null, 0, null, '0', '24'), + ('regionkey', null, null, 0, null, '0', '4'), + ('comment', null, null, 0, null, null, null), + ('name', null, null, 0, null, null, null), + (null, null, null, null, 25, null, null)"""); + } assertUpdate("ANALYZE " + tableName); - assertQuery( - "SHOW STATS FOR " + tableName, - """ - VALUES - ('nationkey', null, 25, 0, null, '0', '24'), - ('regionkey', null, 5, 0, null, '0', '4'), - ('comment', null, 25, 0, null, null, null), - ('name', null, 25, 0, null, null, null), - (null, null, null, null, 25, null, null)"""); + assertQuery("SHOW STATS FOR " + tableName, goodStatsInitial); // reanalyze data assertUpdate("ANALYZE " + tableName); - assertQuery( - "SHOW STATS FOR " + tableName, - """ - VALUES - ('nationkey', null, 25, 0, null, '0', '24'), - ('regionkey', null, 5, 0, null, '0', '4'), - ('comment', null, 25, 0, null, null, null), - ('name', null, 25, 0, null, null, null), - (null, null, null, null, 25, null, null)"""); + assertQuery("SHOW STATS FOR " + tableName, goodStatsInitial); // insert one more copy; should not influence stats other than rowcount - assertUpdate("INSERT INTO " + tableName + " SELECT * FROM tpch.sf1.nation", 25); - + assertUpdate(writeSession, "INSERT INTO " + tableName + " SELECT * FROM tpch.sf1.nation", 25); + String goodStatsAfterFirstInsert = """ + VALUES + ('nationkey', null, 25, 0, null, '0', '24'), + ('regionkey', null, 5, 0, null, '0', '4'), + ('comment', null, 25, 0, null, null, null), + ('name', null, 25, 0, null, null, null), + (null, null, null, null, 50, null, null)"""; assertUpdate("ANALYZE " + tableName); - assertQuery( - "SHOW STATS FOR " + tableName, - """ - VALUES - ('nationkey', null, 25, 0, null, '0', '24'), - ('regionkey', null, 5, 0, null, '0', '4'), - ('comment', null, 25, 0, null, null, null), - ('name', null, 25, 0, null, null, null), - (null, null, null, null, 50, null, null)"""); + assertQuery("SHOW STATS FOR " + tableName, goodStatsAfterFirstInsert); // insert modified rows - assertUpdate("INSERT INTO " + tableName + " SELECT nationkey + 25, reverse(name), regionkey + 5, reverse(comment) FROM tpch.sf1.nation", 25); - - // without ANALYZE all stats but NDV should be updated - assertQuery( - "SHOW STATS FOR " + tableName, - """ - VALUES - ('nationkey', null, 25, 0, null, '0', '49'), - ('regionkey', null, 5, 0, null, '0', '9'), - ('comment', null, 25, 0, null, null, null), - ('name', null, 25, 0, null, null, null), - (null, null, null, null, 75, null, null)"""); + assertUpdate(writeSession, "INSERT INTO " + tableName + " SELECT nationkey + 25, reverse(name), regionkey + 5, reverse(comment) FROM tpch.sf1.nation", 25); + String goodStatsAfterSecondInsert = """ + VALUES + ('nationkey', null, 50, 0, null, '0', '49'), + ('regionkey', null, 10, 0, null, '0', '9'), + ('comment', null, 50, 0, null, null, null), + ('name', null, 50, 0, null, null, null), + (null, null, null, null, 75, null, null)"""; + + if (collectOnStatsOnWrites) { + assertQuery("SHOW STATS FOR " + tableName, goodStatsAfterSecondInsert); + } + else { + // without ANALYZE all stats but NDV should be updated + assertQuery( + "SHOW STATS FOR " + tableName, + """ + VALUES + ('nationkey', null, 25, 0, null, '0', '49'), + ('regionkey', null, 5, 0, null, '0', '9'), + ('comment', null, 25, 0, null, null, null), + ('name', null, 25, 0, null, null, null), + (null, null, null, null, 75, null, null)"""); + } // with analyze we should get new NDV assertUpdate("ANALYZE " + tableName); - assertQuery( - "SHOW STATS FOR " + tableName, - """ - VALUES - ('nationkey', null, 50, 0, null, '0', '49'), - ('regionkey', null, 10, 0, null, '0', '9'), - ('comment', null, 50, 0, null, null, null), - ('name', null, 50, 0, null, null, null), - (null, null, null, null, 75, null, null)"""); + assertQuery("SHOW STATS FOR " + tableName, goodStatsAfterSecondInsert); assertUpdate("DROP TABLE " + tableName); } @@ -159,35 +164,40 @@ public void testAnalyzeWithSchemaEvolution() assertUpdate("DROP TABLE " + tableName); } - @Test - public void testAnalyzePartitioned() + @Test(dataProviderClass = DataProviders.class, dataProvider = "trueFalse") + public void testAnalyzePartitioned(boolean collectOnStatsOnWrites) { - String tableName = "test_analyze_partitioned"; - assertUpdate("CREATE TABLE " + tableName + " WITH (partitioning = ARRAY['regionkey']) AS SELECT * FROM tpch.sf1.nation", 25); - - assertQuery( - "SHOW STATS FOR " + tableName, - """ - VALUES - ('nationkey', null, null, 0, null, '0', '24'), - ('regionkey', null, null, 0, null, '0', '4'), - ('comment', null, null, 0, null, null, null), - ('name', null, null, 0, null, null, null), - (null, null, null, null, 25, null, null)"""); + Session writeSession = withStatsOnWrite(getSession(), collectOnStatsOnWrites); + String tableName = "test_analyze_partitioned_" + collectOnStatsOnWrites; + assertUpdate(writeSession, "CREATE TABLE " + tableName + " WITH (partitioning = ARRAY['regionkey']) AS SELECT * FROM tpch.sf1.nation", 25); + String goodStatsInitial = """ + VALUES + ('nationkey', null, 25, 0, null, '0', '24'), + ('regionkey', null, 5, 0, null, '0', '4'), + ('comment', null, 25, 0, null, null, null), + ('name', null, 25, 0, null, null, null), + (null, null, null, null, 25, null, null)"""; + + if (collectOnStatsOnWrites) { + assertQuery("SHOW STATS FOR " + tableName, goodStatsInitial); + } + else { + assertQuery( + "SHOW STATS FOR " + tableName, + """ + VALUES + ('nationkey', null, null, 0, null, '0', '24'), + ('regionkey', null, null, 0, null, '0', '4'), + ('comment', null, null, 0, null, null, null), + ('name', null, null, 0, null, null, null), + (null, null, null, null, 25, null, null)"""); + } assertUpdate("ANALYZE " + tableName); - assertQuery( - "SHOW STATS FOR " + tableName, - """ - VALUES - ('nationkey', null, 25, 0, null, '0', '24'), - ('regionkey', null, 5, 0, null, '0', '4'), - ('comment', null, 25, 0, null, null, null), - ('name', null, 25, 0, null, null, null), - (null, null, null, null, 25, null, null)"""); + assertQuery("SHOW STATS FOR " + tableName, goodStatsInitial); // insert one more copy; should not influence stats other than rowcount - assertUpdate("INSERT INTO " + tableName + " SELECT * FROM tpch.sf1.nation", 25); + assertUpdate(writeSession, "INSERT INTO " + tableName + " SELECT * FROM tpch.sf1.nation", 25); assertUpdate("ANALYZE " + tableName); assertQuery( @@ -201,30 +211,34 @@ public void testAnalyzePartitioned() (null, null, null, null, 50, null, null)"""); // insert modified rows - assertUpdate("INSERT INTO " + tableName + " SELECT nationkey + 25, reverse(name), regionkey + 5, reverse(comment) FROM tpch.sf1.nation", 25); - - // without ANALYZE all stats but NDV should be updated - assertQuery( - "SHOW STATS FOR " + tableName, - """ - VALUES - ('nationkey', null, 25, 0, null, '0', '49'), - ('regionkey', null, 5, 0, null, '0', '9'), - ('comment', null, 25, 0, null, null, null), - ('name', null, 25, 0, null, null, null), - (null, null, null, null, 75, null, null)"""); + assertUpdate(writeSession, "INSERT INTO " + tableName + " SELECT nationkey + 25, reverse(name), regionkey + 5, reverse(comment) FROM tpch.sf1.nation", 25); + String goodStatsAfterSecondInsert = """ + VALUES + ('nationkey', null, 50, 0, null, '0', '49'), + ('regionkey', null, 10, 0, null, '0', '9'), + ('comment', null, 50, 0, null, null, null), + ('name', null, 50, 0, null, null, null), + (null, null, null, null, 75, null, null)"""; + + if (collectOnStatsOnWrites) { + assertQuery("SHOW STATS FOR " + tableName, goodStatsAfterSecondInsert); + } + else { + // without ANALYZE all stats but NDV should be updated + assertQuery( + "SHOW STATS FOR " + tableName, + """ + VALUES + ('nationkey', null, 25, 0, null, '0', '49'), + ('regionkey', null, 5, 0, null, '0', '9'), + ('comment', null, 25, 0, null, null, null), + ('name', null, 25, 0, null, null, null), + (null, null, null, null, 75, null, null)"""); + } // with analyze we should get new NDV assertUpdate("ANALYZE " + tableName); - assertQuery( - "SHOW STATS FOR " + tableName, - """ - VALUES - ('nationkey', null, 50, 0, null, '0', '49'), - ('regionkey', null, 10, 0, null, '0', '9'), - ('comment', null, 50, 0, null, null, null), - ('name', null, 50, 0, null, null, null), - (null, null, null, null, 75, null, null)"""); + assertQuery("SHOW STATS FOR " + tableName, goodStatsAfterSecondInsert); assertUpdate("DROP TABLE " + tableName); } @@ -233,7 +247,9 @@ public void testAnalyzePartitioned() public void testAnalyzeEmpty() { String tableName = "test_analyze_empty"; - assertUpdate("CREATE TABLE " + tableName + " AS SELECT * FROM tpch.sf1.nation WITH NO DATA", 0); + Session noStatsOnWrite = withStatsOnWrite(getSession(), false); + + assertUpdate(noStatsOnWrite, "CREATE TABLE " + tableName + " AS SELECT * FROM tpch.sf1.nation WITH NO DATA", 0); assertQuery( "SHOW STATS FOR " + tableName, @@ -257,7 +273,8 @@ public void testAnalyzeEmpty() (null, null, null, null, 0, null, null)"""); // add some data and reanalyze - assertUpdate("INSERT INTO " + tableName + " SELECT * FROM tpch.sf1.nation", 25); + + assertUpdate(noStatsOnWrite, "INSERT INTO " + tableName + " SELECT * FROM tpch.sf1.nation", 25); assertUpdate("ANALYZE " + tableName); assertQuery( @@ -273,11 +290,53 @@ public void testAnalyzeEmpty() assertUpdate("DROP TABLE " + tableName); } + @Test(dataProvider = "testCollectStatisticsOnWriteDataProvider") + public void testCollectStatisticsOnWriteToEmptyTable(boolean collectOnStatsOnCreateTable, boolean partitioned) + { + String tableName = "test_collect_stats_insert_into_empty_" + collectOnStatsOnCreateTable + partitioned; + + assertUpdate( + withStatsOnWrite(getSession(), collectOnStatsOnCreateTable), + "CREATE TABLE " + tableName + " " + + (partitioned ? "WITH (partitioning=ARRAY['regionkey']) " : "") + + "AS TABLE tpch.sf1.nation WITH NO DATA", + 0); + assertQuery( + "SHOW STATS FOR " + tableName, + """ + VALUES + ('nationkey', 0, 0, 1, null, null, null), + ('regionkey', 0, 0, 1, null, null, null), + ('comment', 0, 0, 1, null, null, null), + ('name', 0, 0, 1, null, null, null), + (null, null, null, null, 0, null, null)"""); + + assertUpdate(withStatsOnWrite(getSession(), true), "INSERT INTO " + tableName + " TABLE tpch.sf1.nation", 25); + assertQuery( + "SHOW STATS FOR " + tableName, + """ + VALUES + ('nationkey', null, 25, 0, null, '0', '24'), + ('regionkey', null, 5, 0, null, '0', '4'), + ('comment', null, 25, 0, null, null, null), + ('name', null, 25, 0, null, null, null), + (null, null, null, null, 25, null, null)"""); + + assertUpdate("DROP TABLE " + tableName); + } + + @DataProvider + public Object[][] testCollectStatisticsOnWriteDataProvider() + { + return cartesianProduct(trueFalse(), trueFalse()); + } + @Test public void testAnalyzeSomeColumns() { String tableName = "test_analyze_some_columns"; - assertUpdate("CREATE TABLE " + tableName + " AS SELECT * FROM tpch.sf1.nation", 25); + Session noStatsOnWrite = withStatsOnWrite(getSession(), false); + assertUpdate(noStatsOnWrite, "CREATE TABLE " + tableName + " AS SELECT * FROM tpch.sf1.nation", 25); // analyze NULL list of columns assertQueryFails("ANALYZE " + tableName + " WITH (columns = NULL)", "\\QInvalid null value for catalog 'iceberg' analyze property 'columns' from [null]"); @@ -309,7 +368,7 @@ public void testAnalyzeSomeColumns() (null, null, null, null, 25, null, null)"""); // insert modified rows - assertUpdate("INSERT INTO " + tableName + " SELECT nationkey + 25, concat(name, '1'), regionkey + 5, concat(comment, '21') FROM tpch.sf1.nation", 25); + assertUpdate(noStatsOnWrite, "INSERT INTO " + tableName + " SELECT nationkey + 25, concat(name, '1'), regionkey + 5, concat(comment, '21') FROM tpch.sf1.nation", 25); // perform one more analyze for nationkey and regionkey assertUpdate("ANALYZE " + tableName + " WITH (columns = ARRAY['nationkey', 'regionkey'])"); @@ -339,7 +398,7 @@ public void testAnalyzeSomeColumns() (null, null, null, null, 50, null, null)"""); // insert modified rows - assertUpdate("INSERT INTO " + tableName + " SELECT nationkey + 50, concat(name, '2'), regionkey + 10, concat(comment, '22') FROM tpch.sf1.nation", 25); + assertUpdate(noStatsOnWrite, "INSERT INTO " + tableName + " SELECT nationkey + 50, concat(name, '2'), regionkey + 10, concat(comment, '22') FROM tpch.sf1.nation", 25); // without ANALYZE all stats but NDV should be updated assertQuery( @@ -427,10 +486,6 @@ public void testDropExtendedStats() ('name', null, 25, 0, null, null, null), (null, null, null, null, 25, null, null)"""; - assertQuery("SHOW STATS FOR " + tableName, baseStats); - - // Update stats to include distinct count - assertUpdate("ANALYZE " + tableName); assertQuery("SHOW STATS FOR " + tableName, extendedStats); // Dropping extended stats clears distinct count and leaves other stats alone @@ -572,9 +627,111 @@ public void testAnalyzeAndDeleteOrphanFiles() assertUpdate("DROP TABLE " + tableName); } + @Test + public void testEmptyNoScalarColumns() + { + // Currently, only scalar columns can be analyzed + String tableName = "empty_table_without_scalar_columns"; + + assertUpdate("CREATE TABLE " + tableName + " (a row(x integer), b row(y varchar))"); + assertQuery( + "SHOW STATS FOR " + tableName, + """ + VALUES + ('a', 0, 0, 1, null, null, null), + ('b', 0, 0, 1, null, null, null), + (null, null, null, null, 0, null, null)"""); + + // On empty table + assertQueryFails("ANALYZE " + tableName + " WITH (columns = ARRAY[])", "Cannot specify empty list of columns for analysis"); + assertQueryFails("ANALYZE " + tableName + " WITH (columns = ARRAY['a'])", "Invalid columns specified for analysis: \\[a]"); + assertQueryFails("ANALYZE " + tableName + " WITH (columns = ARRAY['a.x'])", "Invalid columns specified for analysis: \\[a.x]"); + assertQueryFails("ANALYZE " + tableName + " WITH (columns = ARRAY['b'])", "Invalid columns specified for analysis: \\[b]"); + assertUpdate("ANALYZE " + tableName); + assertQuery( + "SHOW STATS FOR " + tableName, + """ + VALUES + ('a', 0, 0, 1, null, null, null), + ('b', 0, 0, 1, null, null, null), + (null, null, null, null, 0, null, null)"""); + + // write with stats collection + assertUpdate( + withStatsOnWrite(getSession(), true), + "INSERT INTO " + tableName + " VALUES (ROW(52), ROW('hot')), (ROW(53), ROW('dog'))", + 2); + assertQuery( + "SHOW STATS FOR " + tableName, + """ + VALUES + ('a', null, null, 0, null, null, null), + ('b', null, null, 0, null, null, null), + (null, null, null, null, 2, null, null)"""); + + assertUpdate("DROP TABLE " + tableName); + } + + @Test + public void testNoScalarColumns() + { + // Currently, only scalar columns can be analyzed + String tableName = "table_without_scalar_columns"; + + assertUpdate("CREATE TABLE " + tableName + " (a row(x integer), b row(y varchar))"); + assertUpdate( + withStatsOnWrite(getSession(), false), + "INSERT INTO " + tableName + " VALUES (ROW(42), ROW('ala')), (ROW(43), ROW('has a cat'))", + 2); + assertQuery( + "SHOW STATS FOR " + tableName, + """ + VALUES + ('a', null, null, 0, null, null, null), + ('b', null, null, 0, null, null, null), + (null, null, null, null, 2, null, null)"""); + + // On non-empty table + assertQueryFails("ANALYZE " + tableName + " WITH (columns = ARRAY[])", "Cannot specify empty list of columns for analysis"); + assertQueryFails("ANALYZE " + tableName + " WITH (columns = ARRAY['a'])", "Invalid columns specified for analysis: \\[a]"); + assertQueryFails("ANALYZE " + tableName + " WITH (columns = ARRAY['a.x'])", "Invalid columns specified for analysis: \\[a.x]"); + assertQueryFails("ANALYZE " + tableName + " WITH (columns = ARRAY['b'])", "Invalid columns specified for analysis: \\[b]"); + assertUpdate("ANALYZE " + tableName); + assertQuery( + "SHOW STATS FOR " + tableName, + """ + VALUES + ('a', null, null, 0, null, null, null), + ('b', null, null, 0, null, null, null), + (null, null, null, null, 2, null, null)"""); + + // write with stats collection + assertUpdate( + withStatsOnWrite(getSession(), true), + "INSERT INTO " + tableName + " VALUES (ROW(52), ROW('hot')), (ROW(53), ROW('dog'))", + 2); + assertQuery( + "SHOW STATS FOR " + tableName, + """ + VALUES + ('a', null, null, 0, null, null, null), + ('b', null, null, 0, null, null, null), + (null, null, null, null, 4, null, null)"""); + + assertUpdate("DROP TABLE " + tableName); + } + private long getCurrentSnapshotId(String tableName) { return (long) computeActual(format("SELECT snapshot_id FROM \"%s$snapshots\" ORDER BY committed_at DESC FETCH FIRST 1 ROW WITH TIES", tableName)) .getOnlyValue(); } + + private static Session withStatsOnWrite(Session session, boolean enabled) + { + String catalog = session.getCatalog().orElseThrow(); + return Session.builder(session) + .setCatalogSessionProperty(catalog, COLLECT_EXTENDED_STATISTICS_ON_WRITE, Boolean.toString(enabled)) + .build(); + } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestMetadataQueryOptimization.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestMetadataQueryOptimization.java index bba9dd19405f..b74e0cfad23a 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestMetadataQueryOptimization.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestMetadataQueryOptimization.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.trino.Session; +import io.trino.metadata.InternalFunctionBundle; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.iceberg.catalog.file.TestingIcebergFileMetastoreCatalogModule; @@ -68,6 +69,10 @@ protected LocalQueryRunner createLocalQueryRunner() HiveMetastore metastore = createTestingFileHiveMetastore(baseDir); LocalQueryRunner queryRunner = LocalQueryRunner.create(session); + InternalFunctionBundle.InternalFunctionBundleBuilder functions = InternalFunctionBundle.builder(); + new IcebergPlugin().getFunctions().forEach(functions::functions); + queryRunner.addFunctions(functions.build()); + queryRunner.createCatalog( ICEBERG_CATALOG, new TestingIcebergConnectorFactory(Optional.of(new TestingIcebergFileMetastoreCatalogModule(metastore)), Optional.empty(), EMPTY_MODULE), diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestIcebergFileMetastoreTableOperationsInsertFailure.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestIcebergFileMetastoreTableOperationsInsertFailure.java index 94a7e93e8365..5da898650dba 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestIcebergFileMetastoreTableOperationsInsertFailure.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestIcebergFileMetastoreTableOperationsInsertFailure.java @@ -15,6 +15,7 @@ import com.google.common.collect.ImmutableMap; import io.trino.Session; +import io.trino.metadata.InternalFunctionBundle; import io.trino.plugin.hive.NodeVersion; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.HiveMetastore; @@ -23,6 +24,7 @@ import io.trino.plugin.hive.metastore.Table; import io.trino.plugin.hive.metastore.file.FileHiveMetastore; import io.trino.plugin.hive.metastore.file.FileHiveMetastoreConfig; +import io.trino.plugin.iceberg.IcebergPlugin; import io.trino.plugin.iceberg.TestingIcebergConnectorFactory; import io.trino.spi.security.PrincipalType; import io.trino.testing.AbstractTestQueryFramework; @@ -79,6 +81,10 @@ public synchronized void replaceTable(String databaseName, String tableName, Tab }; LocalQueryRunner queryRunner = LocalQueryRunner.create(session); + InternalFunctionBundle.InternalFunctionBundleBuilder functions = InternalFunctionBundle.builder(); + new IcebergPlugin().getFunctions().forEach(functions::functions); + queryRunner.addFunctions(functions.build()); + queryRunner.createCatalog( ICEBERG_CATALOG, new TestingIcebergConnectorFactory(Optional.of(new TestingIcebergFileMetastoreCatalogModule(metastore)), Optional.empty(), EMPTY_MODULE), diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogAccessOperations.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogAccessOperations.java index f240a059fe76..ebcc2cc46ff5 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogAccessOperations.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogAccessOperations.java @@ -52,6 +52,7 @@ import static com.google.common.base.Verify.verifyNotNull; import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static io.trino.plugin.iceberg.IcebergSessionProperties.COLLECT_EXTENDED_STATISTICS_ON_WRITE; import static io.trino.plugin.iceberg.TableType.DATA; import static io.trino.plugin.iceberg.TableType.FILES; import static io.trino.plugin.iceberg.TableType.HISTORY; @@ -158,7 +159,9 @@ public void testCreateTable() public void testCreateTableAsSelect() { try { - assertGlueMetastoreApiInvocations("CREATE TABLE test_ctas AS SELECT 1 AS age", + assertGlueMetastoreApiInvocations( + withStatsOnWrite(getSession(), false), + "CREATE TABLE test_ctas AS SELECT 1 AS age", ImmutableMultiset.builder() .add(GET_DATABASE) .add(GET_DATABASE) @@ -169,6 +172,22 @@ public void testCreateTableAsSelect() finally { getQueryRunner().execute("DROP TABLE IF EXISTS test_ctas"); } + + try { + assertGlueMetastoreApiInvocations( + withStatsOnWrite(getSession(), true), + "CREATE TABLE test_ctas_with_stats AS SELECT 1 AS age", + ImmutableMultiset.builder() + .add(GET_DATABASE) + .add(GET_DATABASE) + .add(CREATE_TABLE) + .addCopies(GET_TABLE, 6) + .add(UPDATE_TABLE) + .build()); + } + finally { + getQueryRunner().execute("DROP TABLE IF EXISTS test_ctas_with_stats"); + } } @Test @@ -475,6 +494,14 @@ private void assertGlueMetastoreApiInvocations(Session session, @Language("SQL") fail("Expected: \n\t\t" + join(",\n\t\t", mismatchReport)); } + private static Session withStatsOnWrite(Session session, boolean enabled) + { + String catalog = session.getCatalog().orElseThrow(); + return Session.builder(session) + .setCatalogSessionProperty(catalog, COLLECT_EXTENDED_STATISTICS_ON_WRITE, Boolean.toString(enabled)) + .build(); + } + @Retention(RUNTIME) @Target({FIELD, PARAMETER, METHOD}) @Qualifier diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCreateTableFailure.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCreateTableFailure.java index e0c8fc6dff82..110e4ea77b66 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCreateTableFailure.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCreateTableFailure.java @@ -23,8 +23,10 @@ import io.trino.filesystem.FileIterator; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.hdfs.HdfsFileSystemFactory; +import io.trino.metadata.InternalFunctionBundle; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.glue.GlueHiveMetastore; +import io.trino.plugin.iceberg.IcebergPlugin; import io.trino.plugin.iceberg.TestingIcebergConnectorFactory; import io.trino.spi.security.PrincipalType; import io.trino.testing.AbstractTestQueryFramework; @@ -95,6 +97,10 @@ protected LocalQueryRunner createQueryRunner() return result; }); + InternalFunctionBundle.InternalFunctionBundleBuilder functions = InternalFunctionBundle.builder(); + new IcebergPlugin().getFunctions().forEach(functions::functions); + queryRunner.addFunctions(functions.build()); + queryRunner.createCatalog( ICEBERG_CATALOG, new TestingIcebergConnectorFactory(Optional.of(new TestingIcebergGlueCatalogModule(awsGlueAsyncAdapterProvider)), Optional.empty(), EMPTY_MODULE), diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueTableOperationsInsertFailure.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueTableOperationsInsertFailure.java index 71af39cb35c7..9f1ec7318db8 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueTableOperationsInsertFailure.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueTableOperationsInsertFailure.java @@ -17,8 +17,10 @@ import com.google.common.collect.ImmutableMap; import io.airlift.log.Logger; import io.trino.Session; +import io.trino.metadata.InternalFunctionBundle; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.glue.GlueHiveMetastore; +import io.trino.plugin.iceberg.IcebergPlugin; import io.trino.plugin.iceberg.TestingIcebergConnectorFactory; import io.trino.spi.security.PrincipalType; import io.trino.testing.AbstractTestQueryFramework; @@ -80,6 +82,10 @@ protected LocalQueryRunner createQueryRunner() return result; }); + InternalFunctionBundle.InternalFunctionBundleBuilder functions = InternalFunctionBundle.builder(); + new IcebergPlugin().getFunctions().forEach(functions::functions); + queryRunner.addFunctions(functions.build()); + queryRunner.createCatalog( ICEBERG_CATALOG, new TestingIcebergConnectorFactory(Optional.of(new TestingIcebergGlueCatalogModule(awsGlueAsyncAdapterProvider)), Optional.empty(), EMPTY_MODULE), diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestIcebergHiveMetastoreTableOperationsReleaseLockFailure.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestIcebergHiveMetastoreTableOperationsReleaseLockFailure.java index ff254f33c4ce..456815db1754 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestIcebergHiveMetastoreTableOperationsReleaseLockFailure.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestIcebergHiveMetastoreTableOperationsReleaseLockFailure.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableMap; import io.trino.Session; import io.trino.hive.thrift.metastore.Table; +import io.trino.metadata.InternalFunctionBundle; import io.trino.plugin.hive.metastore.AcidTransactionOwner; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.HiveMetastore; @@ -24,6 +25,7 @@ import io.trino.plugin.hive.metastore.thrift.ThriftMetastore; import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreConfig; import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreFactory; +import io.trino.plugin.iceberg.IcebergPlugin; import io.trino.plugin.iceberg.TestingIcebergConnectorFactory; import io.trino.spi.security.ConnectorIdentity; import io.trino.spi.security.PrincipalType; @@ -59,6 +61,10 @@ protected LocalQueryRunner createQueryRunner() throws Exception LocalQueryRunner queryRunner = LocalQueryRunner.create(session); + InternalFunctionBundle.InternalFunctionBundleBuilder functions = InternalFunctionBundle.builder(); + new IcebergPlugin().getFunctions().forEach(functions::functions); + queryRunner.addFunctions(functions.build()); + ThriftMetastore thriftMetastore = createMetastoreWithReleaseLockFailure(); HiveMetastore hiveMetastore = new BridgingHiveMetastore(thriftMetastore); TestingIcebergHiveMetastoreCatalogModule testModule = new TestingIcebergHiveMetastoreCatalogModule(hiveMetastore, buildThriftMetastoreFactory(thriftMetastore)); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/optimizer/TestConnectorPushdownRulesWithIceberg.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/optimizer/TestConnectorPushdownRulesWithIceberg.java index 9ca3b957925b..56088a3642dc 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/optimizer/TestConnectorPushdownRulesWithIceberg.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/optimizer/TestConnectorPushdownRulesWithIceberg.java @@ -18,12 +18,14 @@ import com.google.common.collect.ImmutableSet; import io.trino.Session; import io.trino.cost.ScalarStatsCalculator; +import io.trino.metadata.InternalFunctionBundle; import io.trino.metadata.TableHandle; import io.trino.plugin.hive.HiveTransactionHandle; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.iceberg.ColumnIdentity; import io.trino.plugin.iceberg.IcebergColumnHandle; +import io.trino.plugin.iceberg.IcebergPlugin; import io.trino.plugin.iceberg.IcebergTableHandle; import io.trino.plugin.iceberg.TestingIcebergConnectorFactory; import io.trino.plugin.iceberg.catalog.file.TestingIcebergFileMetastoreCatalogModule; @@ -116,6 +118,10 @@ protected Optional createLocalQueryRunner() HiveMetastore metastore = createTestingFileHiveMetastore(baseDir); LocalQueryRunner queryRunner = LocalQueryRunner.create(ICEBERG_SESSION); + InternalFunctionBundle.InternalFunctionBundleBuilder functions = InternalFunctionBundle.builder(); + new IcebergPlugin().getFunctions().forEach(functions::functions); + queryRunner.addFunctions(functions.build()); + queryRunner.createCatalog( TEST_CATALOG_NAME, new TestingIcebergConnectorFactory(Optional.of(new TestingIcebergFileMetastoreCatalogModule(metastore)), Optional.empty(), EMPTY_MODULE), diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveRedirectionToIceberg.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveRedirectionToIceberg.java index 24144f50d5e8..417ea40c43cd 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveRedirectionToIceberg.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveRedirectionToIceberg.java @@ -316,10 +316,10 @@ public void testShowStats() assertThat(onTrino().executeQuery("SHOW STATS FOR " + hiveTableName)) .containsOnly( - row("nationkey", null, null, 0d, null, "0", "24"), - row("name", null, null, 0d, null, null, null), - row("regionkey", null, null, 0d, null, "0", "4"), - row("comment", null, null, 0d, null, null, null), + row("nationkey", null, 25d, 0d, null, "0", "24"), + row("name", null, 25d, 0d, null, null, null), + row("regionkey", null, 5d, 0d, null, "0", "4"), + row("comment", null, 25d, 0d, null, null, null), row(null, null, null, null, 25d, null, null)); onTrino().executeQuery("DROP TABLE " + icebergTableName);