diff --git a/testing/trino-testing-containers/src/main/java/io/trino/testing/containers/BaseTestContainer.java b/testing/trino-testing-containers/src/main/java/io/trino/testing/containers/BaseTestContainer.java
index 41df08e34f5121..d38ed08d434162 100644
--- a/testing/trino-testing-containers/src/main/java/io/trino/testing/containers/BaseTestContainer.java
+++ b/testing/trino-testing-containers/src/main/java/io/trino/testing/containers/BaseTestContainer.java
@@ -156,6 +156,11 @@ public void stop()
container.stop();
}
+ public String getContainerId()
+ {
+ return container.getContainerId();
+ }
+
public String executeInContainerFailOnError(String... commandAndArgs)
{
Container.ExecResult execResult = executeInContainer(commandAndArgs);
diff --git a/testing/trino-tests/pom.xml b/testing/trino-tests/pom.xml
index 866ad75b9cb33a..7f0a2d07f590e2 100644
--- a/testing/trino-tests/pom.xml
+++ b/testing/trino-tests/pom.xml
@@ -158,6 +158,12 @@
test
+
+ io.trino
+ trino-benchmark-queries
+ test
+
+
io.trino
trino-blackhole
@@ -170,6 +176,31 @@
test
+
+ io.trino
+ trino-hdfs
+ test
+
+
+
+ io.trino
+ trino-hive
+ test
+
+
+
+ io.trino
+ trino-hive
+ test-jar
+ test
+
+
+
+ io.trino
+ trino-iceberg
+ test
+
+
io.trino
trino-main
@@ -196,6 +227,12 @@
test
+
+ io.trino
+ trino-metastore
+ test
+
+
io.trino
trino-plugin-toolkit
@@ -264,6 +301,12 @@
test
+
+ io.trino.tpcds
+ tpcds
+ test
+
+
io.trino.tpch
tpch
@@ -276,6 +319,12 @@
test
+
+ org.apache.iceberg
+ iceberg-core
+ test
+
+
org.apache.maven
maven-model
@@ -300,6 +349,12 @@
test
+
+ org.junit.jupiter
+ junit-jupiter-params
+ test
+
+
org.openjdk.jmh
jmh-core
diff --git a/testing/trino-tests/src/test/java/io/trino/sql/planner/BaseCostBasedPlanTest.java b/testing/trino-tests/src/test/java/io/trino/sql/planner/BaseCostBasedPlanTest.java
new file mode 100644
index 00000000000000..8755e75a47240b
--- /dev/null
+++ b/testing/trino-tests/src/test/java/io/trino/sql/planner/BaseCostBasedPlanTest.java
@@ -0,0 +1,370 @@
+/*
+ * 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.sql.planner;
+
+import com.google.common.base.VerifyException;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.io.Resources;
+import com.google.errorprone.annotations.FormatMethod;
+import io.airlift.log.Logger;
+import io.trino.Session;
+import io.trino.spi.catalog.CatalogName;
+import io.trino.spi.connector.CatalogSchemaTableName;
+import io.trino.spi.connector.ConnectorFactory;
+import io.trino.sql.DynamicFilters;
+import io.trino.sql.ir.Expression;
+import io.trino.sql.ir.Reference;
+import io.trino.sql.planner.OptimizerConfig.JoinDistributionType;
+import io.trino.sql.planner.OptimizerConfig.JoinReorderingStrategy;
+import io.trino.sql.planner.assertions.BasePlanTest;
+import io.trino.sql.planner.plan.AggregationNode;
+import io.trino.sql.planner.plan.ExchangeNode;
+import io.trino.sql.planner.plan.FilterNode;
+import io.trino.sql.planner.plan.JoinNode;
+import io.trino.sql.planner.plan.SemiJoinNode;
+import io.trino.sql.planner.plan.TableScanNode;
+import io.trino.sql.planner.plan.ValuesNode;
+import io.trino.testing.PlanTester;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.IntStream;
+
+import static com.google.common.base.Preconditions.checkState;
+import static com.google.common.base.Verify.verify;
+import static com.google.common.collect.ImmutableList.toImmutableList;
+import static com.google.common.io.Files.createParentDirs;
+import static com.google.common.io.Files.write;
+import static com.google.common.io.Resources.getResource;
+import static io.trino.Session.SessionBuilder;
+import static io.trino.SystemSessionProperties.JOIN_DISTRIBUTION_TYPE;
+import static io.trino.SystemSessionProperties.JOIN_REORDERING_STRATEGY;
+import static io.trino.execution.querystats.PlanOptimizersStatsCollector.createPlanOptimizersStatsCollector;
+import static io.trino.execution.warnings.WarningCollector.NOOP;
+import static io.trino.sql.DynamicFilters.extractDynamicFilters;
+import static io.trino.sql.planner.LogicalPlanner.Stage.OPTIMIZED_AND_VALIDATED;
+import static io.trino.sql.planner.plan.JoinNode.DistributionType.REPLICATED;
+import static io.trino.sql.planner.plan.JoinType.INNER;
+import static io.trino.testing.TestingSession.testSessionBuilder;
+import static java.lang.String.format;
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static java.nio.file.Files.isDirectory;
+import static java.util.Locale.ENGLISH;
+import static java.util.Objects.requireNonNull;
+import static java.util.stream.Collectors.joining;
+import static org.assertj.core.api.Assertions.assertThat;
+
+public abstract class BaseCostBasedPlanTest
+ extends BasePlanTest
+{
+ private static final Logger log = Logger.get(BaseCostBasedPlanTest.class);
+
+ public static final List TPCH_SQL_FILES = IntStream.rangeClosed(1, 22)
+ .mapToObj(i -> format("q%02d", i))
+ .map(queryId -> format("/sql/trino/tpch/%s.sql", queryId))
+ .collect(toImmutableList());
+
+ public static final List TPCDS_SQL_FILES = IntStream.range(1, 100)
+ .mapToObj(i -> format("q%02d", i))
+ .map(queryId -> format("/sql/trino/tpcds/%s.sql", queryId))
+ .collect(toImmutableList());
+
+ protected static final String CATALOG_NAME = "local";
+
+ protected final String schemaName;
+ private final Optional fileFormatName;
+ private final boolean partitioned;
+
+ public BaseCostBasedPlanTest(String schemaName, Optional fileFormatName, boolean partitioned)
+ {
+ this.schemaName = requireNonNull(schemaName, "schemaName is null");
+ this.fileFormatName = requireNonNull(fileFormatName, "fileFormatName is null");
+ this.partitioned = partitioned;
+ }
+
+ @Override
+ protected PlanTester createPlanTester()
+ {
+ SessionBuilder sessionBuilder = testSessionBuilder()
+ .setCatalog(CATALOG_NAME)
+ .setSchema(schemaName)
+ // Reducing ARM and x86 floating point arithmetic differences, mostly visible at PlanNodeStatsEstimateMath::estimateCorrelatedConjunctionRowCount
+ .setSystemProperty("filter_conjunction_independence_factor", "0.750000001")
+ .setSystemProperty("task_concurrency", "1") // these tests don't handle exchanges from local parallel
+ .setSystemProperty(JOIN_REORDERING_STRATEGY, JoinReorderingStrategy.AUTOMATIC.name())
+ .setSystemProperty(JOIN_DISTRIBUTION_TYPE, JoinDistributionType.AUTOMATIC.name());
+ PlanTester planTester = PlanTester.create(sessionBuilder.build(), 8);
+ planTester.createCatalog(
+ CATALOG_NAME,
+ createConnectorFactory(),
+ ImmutableMap.of());
+ return planTester;
+ }
+
+ protected abstract ConnectorFactory createConnectorFactory();
+
+ @BeforeAll
+ public abstract void prepareTables()
+ throws Exception;
+
+ protected abstract List getQueryResourcePaths();
+
+ @ParameterizedTest
+ @MethodSource("getQueryResourcePaths")
+ public void test(String queryResourcePath)
+ {
+ assertThat(generateQueryPlan(readQuery(queryResourcePath))).isEqualTo(read(getQueryPlanResourcePath(queryResourcePath)));
+ }
+
+ protected String getQueryPlanResourcePath(String queryResourcePath)
+ {
+ Path queryPath = Paths.get(queryResourcePath);
+ String connectorName = getPlanTester().getCatalogManager().getCatalog(new CatalogName(CATALOG_NAME)).orElseThrow().getConnectorName().toString();
+ Path directory = queryPath.getParent();
+ directory = directory.resolve(connectorName);
+ if (fileFormatName.isPresent()) {
+ directory = directory.resolve(fileFormatName.get());
+ }
+ directory = directory.resolve(partitioned ? "partitioned" : "unpartitioned");
+ String planResourceName = queryPath.getFileName().toString().replaceAll("\\.sql$", ".plan.txt");
+ return directory.resolve(planResourceName).toString();
+ }
+
+ protected void generate()
+ {
+ initPlanTest();
+ try {
+ prepareTables();
+ getQueryResourcePaths().stream()
+ .parallel()
+ .forEach(queryResourcePath -> {
+ try {
+ Path queryPlanWritePath = Paths.get(
+ getSourcePath().toString(),
+ "src/test/resources",
+ getQueryPlanResourcePath(queryResourcePath));
+ createParentDirs(queryPlanWritePath.toFile());
+ write(generateQueryPlan(readQuery(queryResourcePath)).getBytes(UTF_8), queryPlanWritePath.toFile());
+ log.info("Generated expected plan for query: %s", queryResourcePath);
+ }
+ catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ });
+ }
+ catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new RuntimeException("Interrupted", e);
+ }
+ catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ finally {
+ destroyPlanTest();
+ }
+ }
+
+ public static String readQuery(String resource)
+ {
+ return read(resource).replaceAll("\\s+;\\s+$", "")
+ .replace("${database}.${schema}.", "")
+ .replace("\"${database}\".\"${schema}\".\"${prefix}", "\"")
+ .replace("${scale}", "1");
+ }
+
+ private static String read(String resource)
+ {
+ try {
+ return Resources.toString(getResource(BaseCostBasedPlanTest.class, resource), UTF_8);
+ }
+ catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ }
+
+ private String generateQueryPlan(String query)
+ {
+ try {
+ return getPlanTester().inTransaction(transactionSession -> {
+ PlanTester planTester = getPlanTester();
+ Plan plan = planTester.createPlan(transactionSession, query, planTester.getPlanOptimizers(false), OPTIMIZED_AND_VALIDATED, NOOP, createPlanOptimizersStatsCollector());
+ JoinOrderPrinter joinOrderPrinter = new JoinOrderPrinter(transactionSession);
+ plan.getRoot().accept(joinOrderPrinter, 0);
+ return joinOrderPrinter.result();
+ });
+ }
+ catch (RuntimeException e) {
+ throw new AssertionError("Planning failed for SQL: " + query, e);
+ }
+ }
+
+ protected Path getSourcePath()
+ {
+ Path workingDir = Paths.get(System.getProperty("user.dir"));
+ verify(isDirectory(workingDir), "Working directory is not a directory");
+ if (isDirectory(workingDir.resolve(".git"))) {
+ // Top-level of the repo
+ return workingDir.resolve("testing/trino-tests");
+ }
+ if (workingDir.getFileName().toString().equals("trino-tests")) {
+ return workingDir;
+ }
+ throw new IllegalStateException("This class must be executed from trino-tests or Trino source directory");
+ }
+
+ private class JoinOrderPrinter
+ extends SimplePlanVisitor
+ {
+ private final Session session;
+ private final StringBuilder result = new StringBuilder();
+
+ public JoinOrderPrinter(Session session)
+ {
+ this.session = requireNonNull(session, "session is null");
+ }
+
+ public String result()
+ {
+ return result.toString();
+ }
+
+ @Override
+ public Void visitJoin(JoinNode node, Integer indent)
+ {
+ JoinNode.DistributionType distributionType = node.getDistributionType()
+ .orElseThrow(() -> new VerifyException("Expected distribution type to be set"));
+ if (node.isCrossJoin()) {
+ checkState(node.getType() == INNER && distributionType == REPLICATED, "Expected CROSS JOIN to be INNER REPLICATED");
+ if (node.isMaySkipOutputDuplicates()) {
+ output(indent, "cross join (can skip output duplicates):");
+ }
+ else {
+ output(indent, "cross join:");
+ }
+ }
+ else {
+ if (node.isMaySkipOutputDuplicates()) {
+ output(indent, "join (%s, %s, can skip output duplicates):", node.getType(), distributionType);
+ }
+ else {
+ output(indent, "join (%s, %s):", node.getType(), distributionType);
+ }
+ }
+
+ return visitPlan(node, indent + 1);
+ }
+
+ @Override
+ public Void visitExchange(ExchangeNode node, Integer indent)
+ {
+ Partitioning partitioning = node.getPartitioningScheme().getPartitioning();
+ output(
+ indent,
+ "%s exchange (%s, %s, %s)",
+ node.getScope().name().toLowerCase(ENGLISH),
+ node.getType(),
+ partitioning.getHandle(),
+ partitioning.getArguments().stream()
+ .map(BaseCostBasedPlanTest::argumentBindingToString)
+ .sorted() // Currently, order of hash columns is not deterministic
+ .collect(joining(", ", "[", "]")));
+
+ return visitPlan(node, indent + 1);
+ }
+
+ @Override
+ public Void visitAggregation(AggregationNode node, Integer indent)
+ {
+ output(
+ indent,
+ "%s aggregation over (%s)",
+ node.getStep().name().toLowerCase(ENGLISH),
+ node.getGroupingKeys().stream()
+ .map(Symbol::name)
+ .sorted()
+ .collect(joining(", ")));
+
+ return visitPlan(node, indent + 1);
+ }
+
+ @Override
+ public Void visitFilter(FilterNode node, Integer indent)
+ {
+ DynamicFilters.ExtractResult filters = extractDynamicFilters(node.getPredicate());
+ String inputs = filters.getDynamicConjuncts().stream()
+ .map(descriptor -> descriptor.getInput().toString())
+ .sorted()
+ .collect(joining(", "));
+
+ if (!inputs.isEmpty()) {
+ output(indent, "dynamic filter (%s)", String.join(", ", inputs));
+ indent = indent + 1;
+ }
+ return visitPlan(node, indent);
+ }
+
+ @Override
+ public Void visitTableScan(TableScanNode node, Integer indent)
+ {
+ CatalogSchemaTableName tableName = getPlanTester().getPlannerContext().getMetadata().getTableName(session, node.getTable());
+ output(indent, "scan %s", tableName.getSchemaTableName().getTableName());
+
+ return null;
+ }
+
+ @Override
+ public Void visitSemiJoin(SemiJoinNode node, Integer indent)
+ {
+ output(indent, "semijoin (%s):", node.getDistributionType().get());
+
+ return visitPlan(node, indent + 1);
+ }
+
+ @Override
+ public Void visitValues(ValuesNode node, Integer indent)
+ {
+ output(indent, "values (%s rows)", node.getRowCount());
+
+ return null;
+ }
+
+ @FormatMethod
+ private void output(int indent, String message, Object... args)
+ {
+ String formattedMessage = format(message, args);
+ result.append(format("%s%s\n", " ".repeat(indent), formattedMessage));
+ }
+ }
+
+ private static String argumentBindingToString(Partitioning.ArgumentBinding argument)
+ {
+ if (argument.getConstant() != null) {
+ return argument.getConstant().toString();
+ }
+ Expression expression = argument.getExpression();
+ requireNonNull(expression, "expression is null");
+ if (expression instanceof Reference symbolReference) {
+ return symbolReference.name();
+ }
+ return expression.toString();
+ }
+}
diff --git a/testing/trino-tests/src/test/java/io/trino/sql/planner/BaseIcebergCostBasedPlanTest.java b/testing/trino-tests/src/test/java/io/trino/sql/planner/BaseIcebergCostBasedPlanTest.java
new file mode 100644
index 00000000000000..a4a116bb7670cc
--- /dev/null
+++ b/testing/trino-tests/src/test/java/io/trino/sql/planner/BaseIcebergCostBasedPlanTest.java
@@ -0,0 +1,224 @@
+/*
+ * 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.sql.planner;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.errorprone.annotations.concurrent.GuardedBy;
+import io.airlift.log.Logger;
+import io.trino.metastore.Database;
+import io.trino.metastore.HiveMetastore;
+import io.trino.metastore.Table;
+import io.trino.plugin.hive.metastore.HiveMetastoreFactory;
+import io.trino.plugin.iceberg.IcebergConnector;
+import io.trino.plugin.iceberg.IcebergConnectorFactory;
+import io.trino.spi.connector.Connector;
+import io.trino.spi.connector.ConnectorContext;
+import io.trino.spi.connector.ConnectorFactory;
+import io.trino.testing.containers.Minio;
+import io.trino.testing.containers.junit.ReportLeakedContainers;
+import io.trino.testing.minio.MinioClient;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.file.Path;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+import static com.google.common.base.Verify.verify;
+import static com.google.common.collect.MoreCollectors.onlyElement;
+import static com.google.common.io.MoreFiles.deleteRecursively;
+import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE;
+import static io.trino.metastore.PrincipalPrivileges.NO_PRIVILEGES;
+import static io.trino.plugin.hive.TableType.EXTERNAL_TABLE;
+import static io.trino.plugin.iceberg.CatalogType.TESTING_FILE_METASTORE;
+import static io.trino.plugin.iceberg.IcebergConfig.EXTENDED_STATISTICS_CONFIG;
+import static io.trino.plugin.iceberg.IcebergUtil.METADATA_FILE_EXTENSION;
+import static io.trino.plugin.iceberg.catalog.AbstractIcebergTableOperations.ICEBERG_METASTORE_STORAGE_FORMAT;
+import static io.trino.testing.containers.Minio.MINIO_ACCESS_KEY;
+import static io.trino.testing.containers.Minio.MINIO_REGION;
+import static io.trino.testing.containers.Minio.MINIO_SECRET_KEY;
+import static java.nio.file.Files.createTempDirectory;
+import static java.util.Locale.ENGLISH;
+import static org.apache.iceberg.BaseMetastoreTableOperations.ICEBERG_TABLE_TYPE_VALUE;
+import static org.apache.iceberg.BaseMetastoreTableOperations.METADATA_LOCATION_PROP;
+import static org.apache.iceberg.BaseMetastoreTableOperations.TABLE_TYPE_PROP;
+
+public abstract class BaseIcebergCostBasedPlanTest
+ extends BaseCostBasedPlanTest
+{
+ private static final Logger log = Logger.get(BaseIcebergCostBasedPlanTest.class);
+
+ // Iceberg metadata files are linked using absolute paths, so the bucket name must match where the metadata was exported from.
+ // See more at https://github.com/apache/iceberg/issues/1617
+ private static final String BUCKET_NAME = "starburst-benchmarks-data";
+
+ // The container needs to be shared, since bucket name cannot be reused between tests.
+ // The bucket name is used as a key in TrinoFileSystemCache which is managed in static manner.
+ @GuardedBy("sharedMinioLock")
+ private static Minio sharedMinio;
+ @GuardedBy("sharedMinioLock")
+ private static boolean sharedMinioClosed;
+ private static final Object sharedMinioLock = new Object();
+
+ protected Minio minio;
+ private Path temporaryMetastoreDirectory;
+ private HiveMetastore hiveMetastore;
+ private Map connectorConfiguration;
+
+ protected BaseIcebergCostBasedPlanTest(String schemaName, String fileFormatName, boolean partitioned)
+ {
+ super(schemaName, Optional.of(fileFormatName), partitioned);
+ }
+
+ @Override
+ protected ConnectorFactory createConnectorFactory()
+ {
+ synchronized (sharedMinioLock) {
+ if (sharedMinio == null) {
+ checkState(!sharedMinioClosed, "sharedMinio already closed");
+ Minio minio = Minio.builder().build();
+ minio.start();
+ minio.createBucket(BUCKET_NAME);
+ sharedMinio = minio;
+ Runtime.getRuntime().addShutdownHook(new Thread(BaseIcebergCostBasedPlanTest::disposeSharedResources));
+ // Disable ReportLeakedContainers for this container, as it is intentional that it stays after tests finish
+ ReportLeakedContainers.ignoreContainerId(sharedMinio.getContainerId());
+ }
+ minio = sharedMinio;
+ }
+
+ try {
+ temporaryMetastoreDirectory = createTempDirectory("file-metastore");
+ }
+ catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+
+ connectorConfiguration = ImmutableMap.builder()
+ .put("iceberg.catalog.type", TESTING_FILE_METASTORE.name())
+ .put("hive.metastore.catalog.dir", temporaryMetastoreDirectory.toString())
+ .put("fs.native-s3.enabled", "true")
+ .put("fs.hadoop.enabled", "true")
+ .put("s3.aws-access-key", MINIO_ACCESS_KEY)
+ .put("s3.aws-secret-key", MINIO_SECRET_KEY)
+ .put("s3.region", MINIO_REGION)
+ .put("s3.endpoint", minio.getMinioAddress())
+ .put("s3.path-style-access", "true")
+ .put(EXTENDED_STATISTICS_CONFIG, "true")
+ .put("bootstrap.quiet", "true")
+ .buildOrThrow();
+
+ return new IcebergConnectorFactory()
+ {
+ @Override
+ public Connector create(String catalogName, Map config, ConnectorContext context)
+ {
+ checkArgument(config.isEmpty(), "Unexpected configuration %s", config);
+ Connector connector = super.create(catalogName, connectorConfiguration, context);
+ hiveMetastore = ((IcebergConnector) connector).getInjector()
+ .getInstance(HiveMetastoreFactory.class)
+ .createMetastore(Optional.empty());
+ return connector;
+ }
+ };
+ }
+
+ @Override
+ @BeforeAll
+ public void prepareTables()
+ {
+ String schema = getPlanTester().getDefaultSession().getSchema().orElseThrow();
+ hiveMetastore.createDatabase(
+ Database.builder()
+ .setDatabaseName(schema)
+ .setOwnerName(Optional.empty())
+ .setOwnerType(Optional.empty())
+ .build());
+ doPrepareTables();
+ }
+
+ protected abstract void doPrepareTables();
+
+ // Iceberg metadata files are linked using absolute paths, so the path within the bucket name must match where the metadata was exported from.
+ protected void populateTableFromResource(String tableName, String resourcePath, String targetPath)
+ {
+ String schema = getPlanTester().getDefaultSession().getSchema().orElseThrow();
+
+ log.info("Copying resources for %s unpartitioned table from %s to %s in the container", tableName, resourcePath, targetPath);
+ minio.copyResources(resourcePath, BUCKET_NAME, targetPath);
+
+ String tableLocation = "s3://%s/%s".formatted(BUCKET_NAME, targetPath);
+ String metadataLocation;
+ try (MinioClient minioClient = minio.createMinioClient()) {
+ String metadataPath = minioClient.listObjects(BUCKET_NAME, targetPath + "/metadata/").stream()
+ .filter(path -> path.endsWith(METADATA_FILE_EXTENSION))
+ .collect(onlyElement());
+ metadataLocation = "s3://%s/%s".formatted(BUCKET_NAME, metadataPath);
+ }
+
+ log.info("Registering table %s using metadata location %s", tableName, metadataLocation);
+ hiveMetastore.createTable(
+ Table.builder()
+ .setDatabaseName(schema)
+ .setTableName(tableName)
+ .setOwner(Optional.empty())
+ .setTableType(EXTERNAL_TABLE.name())
+ .setDataColumns(List.of())
+ .withStorage(storage -> storage.setLocation(tableLocation))
+ .withStorage(storage -> storage.setStorageFormat(ICEBERG_METASTORE_STORAGE_FORMAT))
+ // This is a must-have property for the EXTERNAL_TABLE table type
+ .setParameter("EXTERNAL", "TRUE")
+ .setParameter(TABLE_TYPE_PROP, ICEBERG_TABLE_TYPE_VALUE.toUpperCase(ENGLISH))
+ .setParameter(METADATA_LOCATION_PROP, metadataLocation)
+ .build(),
+ NO_PRIVILEGES);
+ }
+
+ @AfterAll
+ public void cleanUp()
+ throws Exception
+ {
+ if (minio != null) {
+ // Don't stop container, as it's shared
+ synchronized (sharedMinioLock) {
+ verify(minio == sharedMinio);
+ }
+ minio = null;
+ }
+
+ if (temporaryMetastoreDirectory != null) {
+ deleteRecursively(temporaryMetastoreDirectory, ALLOW_INSECURE);
+ }
+
+ hiveMetastore = null;
+ connectorConfiguration = null;
+ }
+
+ private static void disposeSharedResources()
+ {
+ synchronized (sharedMinioLock) {
+ sharedMinioClosed = true;
+ if (sharedMinio != null) {
+ sharedMinio.stop();
+ sharedMinio = null;
+ }
+ }
+ }
+}
diff --git a/testing/trino-tests/src/test/java/io/trino/sql/planner/TestIcebergPartitionedTpcdsCostBasedPlan.java b/testing/trino-tests/src/test/java/io/trino/sql/planner/TestIcebergPartitionedTpcdsCostBasedPlan.java
new file mode 100644
index 00000000000000..189cc61db79f2f
--- /dev/null
+++ b/testing/trino-tests/src/test/java/io/trino/sql/planner/TestIcebergPartitionedTpcdsCostBasedPlan.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.sql.planner;
+
+import java.util.List;
+
+/**
+ * This class tests cost-based optimization rules. It contains unmodified TPC-DS queries.
+ * This class is using Iceberg connector un-partitioned TPC-DS tables.
+ */
+public class TestIcebergPartitionedTpcdsCostBasedPlan
+ extends BaseIcebergCostBasedPlanTest
+{
+ protected TestIcebergPartitionedTpcdsCostBasedPlan()
+ {
+ super("tpcds_sf1000_parquet_part", "parquet", true);
+ }
+
+ @Override
+ protected void doPrepareTables()
+ {
+ io.trino.tpcds.Table.getBaseTables().forEach(table -> {
+ if (table == io.trino.tpcds.Table.DBGEN_VERSION) {
+ return;
+ }
+ populateTableFromResource(
+ table.getName(),
+ "iceberg/tpcds/sf1000/parquet/partitioned/" + table.getName(),
+ "iceberg-tpcds-sf1000-parquet-part/" + table.getName());
+ });
+ }
+
+ @Override
+ protected List getQueryResourcePaths()
+ {
+ return TPCDS_SQL_FILES;
+ }
+
+ public static void main(String[] args)
+ {
+ new TestIcebergPartitionedTpcdsCostBasedPlan().generate();
+ }
+}
diff --git a/testing/trino-tests/src/test/java/io/trino/sql/planner/TestIcebergTpchCostBasedPlan.java b/testing/trino-tests/src/test/java/io/trino/sql/planner/TestIcebergTpchCostBasedPlan.java
new file mode 100644
index 00000000000000..c2aa2273bee4ec
--- /dev/null
+++ b/testing/trino-tests/src/test/java/io/trino/sql/planner/TestIcebergTpchCostBasedPlan.java
@@ -0,0 +1,54 @@
+/*
+ * 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.sql.planner;
+
+import io.trino.tpch.TpchTable;
+
+import java.util.List;
+
+/**
+ * This class tests cost-based optimization rules. It contains unmodified TPC-H queries.
+ * This class is using Iceberg connector unpartitioned TPC-H tables.
+ */
+public class TestIcebergTpchCostBasedPlan
+ extends BaseIcebergCostBasedPlanTest
+{
+ public TestIcebergTpchCostBasedPlan()
+ {
+ super("tpch_sf1000_parquet", "parquet", false);
+ }
+
+ @Override
+ protected void doPrepareTables()
+ {
+ TpchTable.getTables().forEach(table -> {
+ populateTableFromResource(
+ table.getTableName(),
+ "iceberg/tpch/sf1000/parquet/unpartitioned/" + table.getTableName(),
+ "iceberg-tpch-sf1000-parquet/" + table.getTableName());
+ });
+ }
+
+ @Override
+ protected List getQueryResourcePaths()
+ {
+ return TPCH_SQL_FILES;
+ }
+
+ public static void main(String[] args)
+ {
+ new TestIcebergTpchCostBasedPlan().generate();
+ }
+}
diff --git a/testing/trino-tests/src/test/java/io/trino/sql/planner/UpdateExpectedPlans.java b/testing/trino-tests/src/test/java/io/trino/sql/planner/UpdateExpectedPlans.java
new file mode 100644
index 00000000000000..d3220624400746
--- /dev/null
+++ b/testing/trino-tests/src/test/java/io/trino/sql/planner/UpdateExpectedPlans.java
@@ -0,0 +1,56 @@
+/*
+ * 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.sql.planner;
+
+import com.google.common.collect.ImmutableList;
+
+import java.lang.reflect.Method;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.Future;
+
+import static io.airlift.concurrent.MoreFutures.getDone;
+
+public final class UpdateExpectedPlans
+{
+ private UpdateExpectedPlans() {}
+
+ public static void main(String[] args)
+ throws Exception
+ {
+ String[] noArgs = new String[0];
+
+ List> futures = ForkJoinPool.commonPool().invokeAll(
+ ImmutableList.>builder()
+ // in alphabetical order
+ .add(runMain(TestIcebergPartitionedTpcdsCostBasedPlan.class, noArgs))
+ .add(runMain(TestIcebergTpchCostBasedPlan.class, noArgs))
+ .build());
+
+ for (Future future : futures) {
+ getDone(future);
+ }
+ }
+
+ private static Callable runMain(Class> clazz, String[] args)
+ throws NoSuchMethodException
+ {
+ Method main = clazz.getMethod("main", String[].class);
+ return () -> {
+ main.invoke(null, new Object[] {args});
+ return null;
+ };
+ }
+}
diff --git a/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/call_center/metadata/00001-e89d97b2-9b2d-47ff-b99d-6651df79a90f.metadata.json b/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/call_center/metadata/00001-e89d97b2-9b2d-47ff-b99d-6651df79a90f.metadata.json
new file mode 100644
index 00000000000000..3feb2cee125880
--- /dev/null
+++ b/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/call_center/metadata/00001-e89d97b2-9b2d-47ff-b99d-6651df79a90f.metadata.json
@@ -0,0 +1,397 @@
+{
+ "format-version" : 1,
+ "table-uuid" : "96c9269f-6a33-409e-9dab-4441acacaed8",
+ "location" : "s3://starburst-benchmarks-data/iceberg-tpcds-sf1000-parquet-part/call_center",
+ "last-updated-ms" : 1663712305837,
+ "last-column-id" : 30,
+ "schema" : {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "cc_call_center_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 2,
+ "name" : "cc_call_center_id",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 3,
+ "name" : "cc_rec_start_date",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 4,
+ "name" : "cc_rec_end_date",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 5,
+ "name" : "cc_closed_date_sk",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 6,
+ "name" : "cc_open_date_sk",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 7,
+ "name" : "cc_name",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 8,
+ "name" : "cc_class",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "cc_employees",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 10,
+ "name" : "cc_sq_ft",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 11,
+ "name" : "cc_hours",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 12,
+ "name" : "cc_manager",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 13,
+ "name" : "cc_mkt_id",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 14,
+ "name" : "cc_mkt_class",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 15,
+ "name" : "cc_mkt_desc",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 16,
+ "name" : "cc_market_manager",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 17,
+ "name" : "cc_division",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 18,
+ "name" : "cc_division_name",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 19,
+ "name" : "cc_company_name",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 20,
+ "name" : "cc_street_number",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 21,
+ "name" : "cc_street_name",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 22,
+ "name" : "cc_street_type",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 23,
+ "name" : "cc_suite_number",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 24,
+ "name" : "cc_city",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 25,
+ "name" : "cc_county",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 26,
+ "name" : "cc_state",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 27,
+ "name" : "cc_zip",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 28,
+ "name" : "cc_country",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 29,
+ "name" : "cc_gmt_offset",
+ "required" : false,
+ "type" : "decimal(5, 2)"
+ }, {
+ "id" : 30,
+ "name" : "cc_tax_percentage",
+ "required" : false,
+ "type" : "decimal(5, 2)"
+ } ]
+ },
+ "current-schema-id" : 0,
+ "schemas" : [ {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "cc_call_center_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 2,
+ "name" : "cc_call_center_id",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 3,
+ "name" : "cc_rec_start_date",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 4,
+ "name" : "cc_rec_end_date",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 5,
+ "name" : "cc_closed_date_sk",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 6,
+ "name" : "cc_open_date_sk",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 7,
+ "name" : "cc_name",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 8,
+ "name" : "cc_class",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "cc_employees",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 10,
+ "name" : "cc_sq_ft",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 11,
+ "name" : "cc_hours",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 12,
+ "name" : "cc_manager",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 13,
+ "name" : "cc_mkt_id",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 14,
+ "name" : "cc_mkt_class",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 15,
+ "name" : "cc_mkt_desc",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 16,
+ "name" : "cc_market_manager",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 17,
+ "name" : "cc_division",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 18,
+ "name" : "cc_division_name",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 19,
+ "name" : "cc_company_name",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 20,
+ "name" : "cc_street_number",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 21,
+ "name" : "cc_street_name",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 22,
+ "name" : "cc_street_type",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 23,
+ "name" : "cc_suite_number",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 24,
+ "name" : "cc_city",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 25,
+ "name" : "cc_county",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 26,
+ "name" : "cc_state",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 27,
+ "name" : "cc_zip",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 28,
+ "name" : "cc_country",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 29,
+ "name" : "cc_gmt_offset",
+ "required" : false,
+ "type" : "decimal(5, 2)"
+ }, {
+ "id" : 30,
+ "name" : "cc_tax_percentage",
+ "required" : false,
+ "type" : "decimal(5, 2)"
+ } ]
+ } ],
+ "partition-spec" : [ ],
+ "default-spec-id" : 0,
+ "partition-specs" : [ {
+ "spec-id" : 0,
+ "fields" : [ ]
+ } ],
+ "last-partition-id" : 999,
+ "default-sort-order-id" : 0,
+ "sort-orders" : [ {
+ "order-id" : 0,
+ "fields" : [ ]
+ } ],
+ "properties" : {
+ "trino.stats.ndv.22.ndv" : "11",
+ "trino.stats.ndv.7.ndv" : "21",
+ "trino.stats.ndv.2.ndv" : "21",
+ "trino.stats.ndv.27.ndv" : "19",
+ "trino.stats.ndv.19.ndv" : "6",
+ "trino.stats.ndv.30.ndv" : "12",
+ "trino.stats.ndv.18.ndv" : "6",
+ "trino.stats.ndv.14.ndv" : "36",
+ "trino.stats.ndv.23.ndv" : "18",
+ "trino.stats.ndv.8.ndv" : "3",
+ "trino.stats.ndv.21.ndv" : "21",
+ "trino.stats.ndv.3.ndv" : "4",
+ "write.format.default" : "PARQUET",
+ "trino.stats.ndv.17.ndv" : "6",
+ "trino.stats.ndv.13.ndv" : "6",
+ "trino.stats.ndv.26.ndv" : "14",
+ "trino.stats.ndv.16.ndv" : "35",
+ "trino.stats.ndv.4.ndv" : "3",
+ "trino.stats.ndv.5.ndv" : "0",
+ "trino.stats.ndv.29.ndv" : "4",
+ "trino.stats.ndv.9.ndv" : "30",
+ "trino.stats.ndv.12.ndv" : "28",
+ "trino.stats.ndv.25.ndv" : "16",
+ "trino.stats.ndv.10.ndv" : "31",
+ "trino.stats.ndv.1.ndv" : "42",
+ "trino.stats.ndv.28.ndv" : "1",
+ "trino.stats.ndv.15.ndv" : "33",
+ "trino.stats.ndv.6.ndv" : "21",
+ "trino.stats.ndv.20.ndv" : "21",
+ "trino.stats.ndv.24.ndv" : "17",
+ "trino.stats.ndv.11.ndv" : "3"
+ },
+ "current-snapshot-id" : 5654429153846711549,
+ "refs" : {
+ "main" : {
+ "snapshot-id" : 5654429153846711549,
+ "type" : "branch"
+ }
+ },
+ "snapshots" : [ {
+ "snapshot-id" : 5654429153846711549,
+ "timestamp-ms" : 1648081195882,
+ "summary" : {
+ "operation" : "append",
+ "added-data-files" : "1",
+ "added-records" : "42",
+ "added-files-size" : "10246",
+ "changed-partition-count" : "1",
+ "total-records" : "42",
+ "total-files-size" : "10246",
+ "total-data-files" : "1",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "s3://starburst-benchmarks-data/iceberg-tpcds-sf1000-parquet-part/call_center/metadata/snap-5654429153846711549-1-e21023b4-6337-466c-8a49-ca737e4de1e6.avro",
+ "schema-id" : 0
+ } ],
+ "snapshot-log" : [ {
+ "timestamp-ms" : 1648081195882,
+ "snapshot-id" : 5654429153846711549
+ } ],
+ "metadata-log" : [ {
+ "timestamp-ms" : 1648081195882,
+ "metadata-file" : "s3://starburst-benchmarks-data/iceberg-tpcds-sf1000-parquet-part/call_center/metadata/00000-f7996e1b-6f13-4cc6-92f6-e9a0de2d1cab.metadata.json"
+ } ]
+}
\ No newline at end of file
diff --git a/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/call_center/metadata/e21023b4-6337-466c-8a49-ca737e4de1e6-m0.avro b/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/call_center/metadata/e21023b4-6337-466c-8a49-ca737e4de1e6-m0.avro
new file mode 100644
index 00000000000000..6db9560f5dc6c7
Binary files /dev/null and b/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/call_center/metadata/e21023b4-6337-466c-8a49-ca737e4de1e6-m0.avro differ
diff --git a/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/call_center/metadata/snap-5654429153846711549-1-e21023b4-6337-466c-8a49-ca737e4de1e6.avro b/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/call_center/metadata/snap-5654429153846711549-1-e21023b4-6337-466c-8a49-ca737e4de1e6.avro
new file mode 100644
index 00000000000000..ecc121960b7151
Binary files /dev/null and b/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/call_center/metadata/snap-5654429153846711549-1-e21023b4-6337-466c-8a49-ca737e4de1e6.avro differ
diff --git a/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/catalog_page/metadata/00001-f71cfe6e-55e3-41b5-880a-5147a972c288.metadata.json b/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/catalog_page/metadata/00001-f71cfe6e-55e3-41b5-880a-5147a972c288.metadata.json
new file mode 100644
index 00000000000000..28d8aed2b25188
--- /dev/null
+++ b/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/catalog_page/metadata/00001-f71cfe6e-55e3-41b5-880a-5147a972c288.metadata.json
@@ -0,0 +1,166 @@
+{
+ "format-version" : 1,
+ "table-uuid" : "6ed5e9b9-f04a-43f2-b8ce-6039e48389e0",
+ "location" : "s3://starburst-benchmarks-data/iceberg-tpcds-sf1000-parquet-part/catalog_page",
+ "last-updated-ms" : 1663712309433,
+ "last-column-id" : 9,
+ "schema" : {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "cp_catalog_page_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 2,
+ "name" : "cp_catalog_page_id",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 3,
+ "name" : "cp_start_date_sk",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 4,
+ "name" : "cp_end_date_sk",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 5,
+ "name" : "cp_department",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 6,
+ "name" : "cp_catalog_number",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 7,
+ "name" : "cp_catalog_page_number",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 8,
+ "name" : "cp_description",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "cp_type",
+ "required" : false,
+ "type" : "string"
+ } ]
+ },
+ "current-schema-id" : 0,
+ "schemas" : [ {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "cp_catalog_page_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 2,
+ "name" : "cp_catalog_page_id",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 3,
+ "name" : "cp_start_date_sk",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 4,
+ "name" : "cp_end_date_sk",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 5,
+ "name" : "cp_department",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 6,
+ "name" : "cp_catalog_number",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 7,
+ "name" : "cp_catalog_page_number",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 8,
+ "name" : "cp_description",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "cp_type",
+ "required" : false,
+ "type" : "string"
+ } ]
+ } ],
+ "partition-spec" : [ ],
+ "default-spec-id" : 0,
+ "partition-specs" : [ {
+ "spec-id" : 0,
+ "fields" : [ ]
+ } ],
+ "last-partition-id" : 999,
+ "default-sort-order-id" : 0,
+ "sort-orders" : [ {
+ "order-id" : 0,
+ "fields" : [ ]
+ } ],
+ "properties" : {
+ "trino.stats.ndv.7.ndv" : "273",
+ "trino.stats.ndv.2.ndv" : "29136",
+ "trino.stats.ndv.8.ndv" : "29704",
+ "trino.stats.ndv.1.ndv" : "29928",
+ "trino.stats.ndv.3.ndv" : "91",
+ "trino.stats.ndv.6.ndv" : "109",
+ "write.format.default" : "PARQUET",
+ "trino.stats.ndv.4.ndv" : "97",
+ "trino.stats.ndv.5.ndv" : "1",
+ "trino.stats.ndv.9.ndv" : "3"
+ },
+ "current-snapshot-id" : 8481719299089367901,
+ "refs" : {
+ "main" : {
+ "snapshot-id" : 8481719299089367901,
+ "type" : "branch"
+ }
+ },
+ "snapshots" : [ {
+ "snapshot-id" : 8481719299089367901,
+ "timestamp-ms" : 1648081199134,
+ "summary" : {
+ "operation" : "append",
+ "added-data-files" : "6",
+ "added-records" : "30000",
+ "added-files-size" : "960474",
+ "changed-partition-count" : "1",
+ "total-records" : "30000",
+ "total-files-size" : "960474",
+ "total-data-files" : "6",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "s3://starburst-benchmarks-data/iceberg-tpcds-sf1000-parquet-part/catalog_page/metadata/snap-8481719299089367901-1-c80f6b66-1991-4295-b758-8a103f131278.avro",
+ "schema-id" : 0
+ } ],
+ "snapshot-log" : [ {
+ "timestamp-ms" : 1648081199134,
+ "snapshot-id" : 8481719299089367901
+ } ],
+ "metadata-log" : [ {
+ "timestamp-ms" : 1648081199134,
+ "metadata-file" : "s3://starburst-benchmarks-data/iceberg-tpcds-sf1000-parquet-part/catalog_page/metadata/00000-332e624c-b685-40f5-8d5f-57d4ef5f8412.metadata.json"
+ } ]
+}
\ No newline at end of file
diff --git a/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/catalog_page/metadata/c80f6b66-1991-4295-b758-8a103f131278-m0.avro b/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/catalog_page/metadata/c80f6b66-1991-4295-b758-8a103f131278-m0.avro
new file mode 100644
index 00000000000000..0156285795ca1c
Binary files /dev/null and b/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/catalog_page/metadata/c80f6b66-1991-4295-b758-8a103f131278-m0.avro differ
diff --git a/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/catalog_page/metadata/snap-8481719299089367901-1-c80f6b66-1991-4295-b758-8a103f131278.avro b/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/catalog_page/metadata/snap-8481719299089367901-1-c80f6b66-1991-4295-b758-8a103f131278.avro
new file mode 100644
index 00000000000000..23d1998d4e71b3
Binary files /dev/null and b/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/catalog_page/metadata/snap-8481719299089367901-1-c80f6b66-1991-4295-b758-8a103f131278.avro differ
diff --git a/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/catalog_returns/metadata/00001-535dd9c7-f58d-4caf-a091-9d9f69ccf38a.metadata.json b/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/catalog_returns/metadata/00001-535dd9c7-f58d-4caf-a091-9d9f69ccf38a.metadata.json
new file mode 100644
index 00000000000000..c2824b5e9c505b
--- /dev/null
+++ b/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/catalog_returns/metadata/00001-535dd9c7-f58d-4caf-a091-9d9f69ccf38a.metadata.json
@@ -0,0 +1,374 @@
+{
+ "format-version" : 1,
+ "table-uuid" : "a0848f2b-0f57-4a01-af1a-6c2538bb75bb",
+ "location" : "s3://starburst-benchmarks-data/iceberg-tpcds-sf1000-parquet-part/catalog_returns",
+ "last-updated-ms" : 1663712316253,
+ "last-column-id" : 27,
+ "schema" : {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "cr_returned_time_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 2,
+ "name" : "cr_item_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 3,
+ "name" : "cr_refunded_customer_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "cr_refunded_cdemo_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 5,
+ "name" : "cr_refunded_hdemo_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 6,
+ "name" : "cr_refunded_addr_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 7,
+ "name" : "cr_returning_customer_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 8,
+ "name" : "cr_returning_cdemo_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 9,
+ "name" : "cr_returning_hdemo_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 10,
+ "name" : "cr_returning_addr_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 11,
+ "name" : "cr_call_center_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 12,
+ "name" : "cr_catalog_page_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 13,
+ "name" : "cr_ship_mode_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 14,
+ "name" : "cr_warehouse_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 15,
+ "name" : "cr_reason_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 16,
+ "name" : "cr_order_number",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 17,
+ "name" : "cr_return_quantity",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 18,
+ "name" : "cr_return_amount",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 19,
+ "name" : "cr_return_tax",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 20,
+ "name" : "cr_return_amt_inc_tax",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 21,
+ "name" : "cr_fee",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 22,
+ "name" : "cr_return_ship_cost",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 23,
+ "name" : "cr_refunded_cash",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 24,
+ "name" : "cr_reversed_charge",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 25,
+ "name" : "cr_store_credit",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 26,
+ "name" : "cr_net_loss",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 27,
+ "name" : "cr_returned_date_sk",
+ "required" : false,
+ "type" : "long"
+ } ]
+ },
+ "current-schema-id" : 0,
+ "schemas" : [ {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "cr_returned_time_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 2,
+ "name" : "cr_item_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 3,
+ "name" : "cr_refunded_customer_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "cr_refunded_cdemo_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 5,
+ "name" : "cr_refunded_hdemo_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 6,
+ "name" : "cr_refunded_addr_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 7,
+ "name" : "cr_returning_customer_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 8,
+ "name" : "cr_returning_cdemo_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 9,
+ "name" : "cr_returning_hdemo_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 10,
+ "name" : "cr_returning_addr_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 11,
+ "name" : "cr_call_center_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 12,
+ "name" : "cr_catalog_page_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 13,
+ "name" : "cr_ship_mode_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 14,
+ "name" : "cr_warehouse_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 15,
+ "name" : "cr_reason_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 16,
+ "name" : "cr_order_number",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 17,
+ "name" : "cr_return_quantity",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 18,
+ "name" : "cr_return_amount",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 19,
+ "name" : "cr_return_tax",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 20,
+ "name" : "cr_return_amt_inc_tax",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 21,
+ "name" : "cr_fee",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 22,
+ "name" : "cr_return_ship_cost",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 23,
+ "name" : "cr_refunded_cash",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 24,
+ "name" : "cr_reversed_charge",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 25,
+ "name" : "cr_store_credit",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 26,
+ "name" : "cr_net_loss",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 27,
+ "name" : "cr_returned_date_sk",
+ "required" : false,
+ "type" : "long"
+ } ]
+ } ],
+ "partition-spec" : [ {
+ "name" : "cr_returned_date_sk",
+ "transform" : "identity",
+ "source-id" : 27,
+ "field-id" : 1000
+ } ],
+ "default-spec-id" : 0,
+ "partition-specs" : [ {
+ "spec-id" : 0,
+ "fields" : [ {
+ "name" : "cr_returned_date_sk",
+ "transform" : "identity",
+ "source-id" : 27,
+ "field-id" : 1000
+ } ]
+ } ],
+ "last-partition-id" : 1000,
+ "default-sort-order-id" : 0,
+ "sort-orders" : [ {
+ "order-id" : 0,
+ "fields" : [ ]
+ } ],
+ "properties" : {
+ "trino.stats.ndv.22.ndv" : "475083",
+ "trino.stats.ndv.2.ndv" : "297612",
+ "trino.stats.ndv.7.ndv" : "12236563",
+ "trino.stats.ndv.27.ndv" : "2141",
+ "trino.stats.ndv.19.ndv" : "152629",
+ "trino.stats.ndv.18.ndv" : "891994",
+ "trino.stats.ndv.14.ndv" : "20",
+ "trino.stats.ndv.23.ndv" : "1084039",
+ "trino.stats.ndv.21.ndv" : "10141",
+ "trino.stats.ndv.8.ndv" : "1890006",
+ "trino.stats.ndv.3.ndv" : "12236563",
+ "write.format.default" : "PARQUET",
+ "trino.stats.ndv.17.ndv" : "100",
+ "trino.stats.ndv.13.ndv" : "20",
+ "trino.stats.ndv.26.ndv" : "906299",
+ "trino.stats.ndv.16.ndv" : "95523325",
+ "trino.stats.ndv.4.ndv" : "1890006",
+ "trino.stats.ndv.5.ndv" : "7082",
+ "trino.stats.ndv.9.ndv" : "7082",
+ "trino.stats.ndv.12.ndv" : "16739",
+ "trino.stats.ndv.25.ndv" : "795604",
+ "trino.stats.ndv.10.ndv" : "5947530",
+ "trino.stats.ndv.1.ndv" : "89157",
+ "trino.stats.ndv.15.ndv" : "65",
+ "trino.stats.ndv.6.ndv" : "5947530",
+ "trino.stats.ndv.20.ndv" : "1528121",
+ "trino.stats.ndv.24.ndv" : "808456",
+ "trino.stats.ndv.11.ndv" : "42"
+ },
+ "current-snapshot-id" : 3711210276322022202,
+ "refs" : {
+ "main" : {
+ "snapshot-id" : 3711210276322022202,
+ "type" : "branch"
+ }
+ },
+ "snapshots" : [ {
+ "snapshot-id" : 3711210276322022202,
+ "timestamp-ms" : 1648081623191,
+ "summary" : {
+ "operation" : "append",
+ "added-data-files" : "2104",
+ "added-records" : "143996756",
+ "added-files-size" : "9045793978",
+ "changed-partition-count" : "2104",
+ "total-records" : "143996756",
+ "total-files-size" : "9045793978",
+ "total-data-files" : "2104",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "s3://starburst-benchmarks-data/iceberg-tpcds-sf1000-parquet-part/catalog_returns/metadata/snap-3711210276322022202-1-0e607888-8425-44d1-a805-3fa09850a308.avro",
+ "schema-id" : 0
+ } ],
+ "snapshot-log" : [ {
+ "timestamp-ms" : 1648081623191,
+ "snapshot-id" : 3711210276322022202
+ } ],
+ "metadata-log" : [ {
+ "timestamp-ms" : 1648081623191,
+ "metadata-file" : "s3://starburst-benchmarks-data/iceberg-tpcds-sf1000-parquet-part/catalog_returns/metadata/00000-5a704af0-8c39-412f-9e3f-b7db771ab8ba.metadata.json"
+ } ]
+}
\ No newline at end of file
diff --git a/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/catalog_returns/metadata/0e607888-8425-44d1-a805-3fa09850a308-m0.avro b/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/catalog_returns/metadata/0e607888-8425-44d1-a805-3fa09850a308-m0.avro
new file mode 100644
index 00000000000000..954589a478a25a
Binary files /dev/null and b/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/catalog_returns/metadata/0e607888-8425-44d1-a805-3fa09850a308-m0.avro differ
diff --git a/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/catalog_returns/metadata/snap-3711210276322022202-1-0e607888-8425-44d1-a805-3fa09850a308.avro b/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/catalog_returns/metadata/snap-3711210276322022202-1-0e607888-8425-44d1-a805-3fa09850a308.avro
new file mode 100644
index 00000000000000..7861dff6163a9a
Binary files /dev/null and b/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/catalog_returns/metadata/snap-3711210276322022202-1-0e607888-8425-44d1-a805-3fa09850a308.avro differ
diff --git a/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/catalog_sales/metadata/00001-614bb4de-488f-47fa-b807-878548c039ed.metadata.json b/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/catalog_sales/metadata/00001-614bb4de-488f-47fa-b807-878548c039ed.metadata.json
new file mode 100644
index 00000000000000..b817f1bc3b338b
--- /dev/null
+++ b/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/catalog_sales/metadata/00001-614bb4de-488f-47fa-b807-878548c039ed.metadata.json
@@ -0,0 +1,451 @@
+{
+ "format-version" : 1,
+ "table-uuid" : "f375e6b2-287a-476b-bce0-dd4518738360",
+ "location" : "s3://starburst-benchmarks-data/iceberg-tpcds-sf1000-parquet-part/catalog_sales",
+ "last-updated-ms" : 1663712624382,
+ "last-column-id" : 34,
+ "schema" : {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "cs_sold_time_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 2,
+ "name" : "cs_ship_date_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 3,
+ "name" : "cs_bill_customer_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "cs_bill_cdemo_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 5,
+ "name" : "cs_bill_hdemo_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 6,
+ "name" : "cs_bill_addr_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 7,
+ "name" : "cs_ship_customer_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 8,
+ "name" : "cs_ship_cdemo_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 9,
+ "name" : "cs_ship_hdemo_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 10,
+ "name" : "cs_ship_addr_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 11,
+ "name" : "cs_call_center_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 12,
+ "name" : "cs_catalog_page_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 13,
+ "name" : "cs_ship_mode_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 14,
+ "name" : "cs_warehouse_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 15,
+ "name" : "cs_item_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 16,
+ "name" : "cs_promo_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 17,
+ "name" : "cs_order_number",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 18,
+ "name" : "cs_quantity",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 19,
+ "name" : "cs_wholesale_cost",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 20,
+ "name" : "cs_list_price",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 21,
+ "name" : "cs_sales_price",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 22,
+ "name" : "cs_ext_discount_amt",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 23,
+ "name" : "cs_ext_sales_price",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 24,
+ "name" : "cs_ext_wholesale_cost",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 25,
+ "name" : "cs_ext_list_price",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 26,
+ "name" : "cs_ext_tax",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 27,
+ "name" : "cs_coupon_amt",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 28,
+ "name" : "cs_ext_ship_cost",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 29,
+ "name" : "cs_net_paid",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 30,
+ "name" : "cs_net_paid_inc_tax",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 31,
+ "name" : "cs_net_paid_inc_ship",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 32,
+ "name" : "cs_net_paid_inc_ship_tax",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 33,
+ "name" : "cs_net_profit",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 34,
+ "name" : "cs_sold_date_sk",
+ "required" : false,
+ "type" : "long"
+ } ]
+ },
+ "current-schema-id" : 0,
+ "schemas" : [ {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "cs_sold_time_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 2,
+ "name" : "cs_ship_date_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 3,
+ "name" : "cs_bill_customer_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "cs_bill_cdemo_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 5,
+ "name" : "cs_bill_hdemo_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 6,
+ "name" : "cs_bill_addr_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 7,
+ "name" : "cs_ship_customer_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 8,
+ "name" : "cs_ship_cdemo_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 9,
+ "name" : "cs_ship_hdemo_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 10,
+ "name" : "cs_ship_addr_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 11,
+ "name" : "cs_call_center_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 12,
+ "name" : "cs_catalog_page_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 13,
+ "name" : "cs_ship_mode_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 14,
+ "name" : "cs_warehouse_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 15,
+ "name" : "cs_item_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 16,
+ "name" : "cs_promo_sk",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 17,
+ "name" : "cs_order_number",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 18,
+ "name" : "cs_quantity",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 19,
+ "name" : "cs_wholesale_cost",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 20,
+ "name" : "cs_list_price",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 21,
+ "name" : "cs_sales_price",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 22,
+ "name" : "cs_ext_discount_amt",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 23,
+ "name" : "cs_ext_sales_price",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 24,
+ "name" : "cs_ext_wholesale_cost",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 25,
+ "name" : "cs_ext_list_price",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 26,
+ "name" : "cs_ext_tax",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 27,
+ "name" : "cs_coupon_amt",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 28,
+ "name" : "cs_ext_ship_cost",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 29,
+ "name" : "cs_net_paid",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 30,
+ "name" : "cs_net_paid_inc_tax",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 31,
+ "name" : "cs_net_paid_inc_ship",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 32,
+ "name" : "cs_net_paid_inc_ship_tax",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 33,
+ "name" : "cs_net_profit",
+ "required" : false,
+ "type" : "decimal(7, 2)"
+ }, {
+ "id" : 34,
+ "name" : "cs_sold_date_sk",
+ "required" : false,
+ "type" : "long"
+ } ]
+ } ],
+ "partition-spec" : [ {
+ "name" : "cs_sold_date_sk",
+ "transform" : "identity",
+ "source-id" : 34,
+ "field-id" : 1000
+ } ],
+ "default-spec-id" : 0,
+ "partition-specs" : [ {
+ "spec-id" : 0,
+ "fields" : [ {
+ "name" : "cs_sold_date_sk",
+ "transform" : "identity",
+ "source-id" : 34,
+ "field-id" : 1000
+ } ]
+ } ],
+ "last-partition-id" : 1000,
+ "default-sort-order-id" : 0,
+ "sort-orders" : [ {
+ "order-id" : 0,
+ "fields" : [ ]
+ } ],
+ "properties" : {
+ "trino.stats.ndv.22.ndv" : "1070707",
+ "trino.stats.ndv.2.ndv" : "1950",
+ "trino.stats.ndv.7.ndv" : "12236563",
+ "trino.stats.ndv.27.ndv" : "1568468",
+ "trino.stats.ndv.30.ndv" : "2432400",
+ "trino.stats.ndv.19.ndv" : "10091",
+ "trino.stats.ndv.18.ndv" : "100",
+ "trino.stats.ndv.14.ndv" : "20",
+ "trino.stats.ndv.23.ndv" : "1067497",
+ "trino.stats.ndv.34.ndv" : "1828",
+ "trino.stats.ndv.21.ndv" : "29691",
+ "trino.stats.ndv.8.ndv" : "1890006",
+ "trino.stats.ndv.3.ndv" : "12236563",
+ "write.format.default" : "PARQUET",
+ "trino.stats.ndv.17.ndv" : "165467466",
+ "trino.stats.ndv.31.ndv" : "2468889",
+ "trino.stats.ndv.13.ndv" : "20",
+ "trino.stats.ndv.26.ndv" : "216236",
+ "trino.stats.ndv.16.ndv" : "1483",
+ "trino.stats.ndv.33.ndv" : "2073314",
+ "trino.stats.ndv.4.ndv" : "1890006",
+ "trino.stats.ndv.5.ndv" : "7082",
+ "trino.stats.ndv.29.ndv" : "1810472",
+ "trino.stats.ndv.9.ndv" : "7082",
+ "trino.stats.ndv.12.ndv" : "16739",
+ "trino.stats.ndv.25.ndv" : "1115360",
+ "trino.stats.ndv.10.ndv" : "5947530",
+ "trino.stats.ndv.1.ndv" : "89157",
+ "trino.stats.ndv.28.ndv" : "553991",
+ "trino.stats.ndv.15.ndv" : "297612",
+ "trino.stats.ndv.6.ndv" : "5947530",
+ "trino.stats.ndv.32.ndv" : "3332491",
+ "trino.stats.ndv.20.ndv" : "29733",
+ "trino.stats.ndv.24.ndv" : "388752",
+ "trino.stats.ndv.11.ndv" : "42"
+ },
+ "current-snapshot-id" : 1524015146186969396,
+ "refs" : {
+ "main" : {
+ "snapshot-id" : 1524015146186969396,
+ "type" : "branch"
+ }
+ },
+ "snapshots" : [ {
+ "snapshot-id" : 1524015146186969396,
+ "timestamp-ms" : 1648084884149,
+ "summary" : {
+ "operation" : "append",
+ "added-data-files" : "1837",
+ "added-records" : "1439980416",
+ "added-files-size" : "81851841736",
+ "changed-partition-count" : "1837",
+ "total-records" : "1439980416",
+ "total-files-size" : "81851841736",
+ "total-data-files" : "1837",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "s3://starburst-benchmarks-data/iceberg-tpcds-sf1000-parquet-part/catalog_sales/metadata/snap-1524015146186969396-1-bffa6a85-351c-4221-9149-e24f845e0aab.avro",
+ "schema-id" : 0
+ } ],
+ "snapshot-log" : [ {
+ "timestamp-ms" : 1648084884149,
+ "snapshot-id" : 1524015146186969396
+ } ],
+ "metadata-log" : [ {
+ "timestamp-ms" : 1648084884149,
+ "metadata-file" : "s3://starburst-benchmarks-data/iceberg-tpcds-sf1000-parquet-part/catalog_sales/metadata/00000-2dbda7b7-b41d-435e-adf4-c7180a1c42a6.metadata.json"
+ } ]
+}
\ No newline at end of file
diff --git a/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/catalog_sales/metadata/bffa6a85-351c-4221-9149-e24f845e0aab-m0.avro b/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/catalog_sales/metadata/bffa6a85-351c-4221-9149-e24f845e0aab-m0.avro
new file mode 100644
index 00000000000000..9ddc7a5040610f
--- /dev/null
+++ b/testing/trino-tests/src/test/resources/iceberg/tpcds/sf1000/parquet/partitioned/catalog_sales/metadata/bffa6a85-351c-4221-9149-e24f845e0aab-m0.avro
@@ -0,0 +1,3023 @@
+Objschema†&{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"cs_sold_time_sk","required":false,"type":"long"},{"id":2,"name":"cs_ship_date_sk","required":false,"type":"long"},{"id":3,"name":"cs_bill_customer_sk","required":false,"type":"long"},{"id":4,"name":"cs_bill_cdemo_sk","required":false,"type":"long"},{"id":5,"name":"cs_bill_hdemo_sk","required":false,"type":"long"},{"id":6,"name":"cs_bill_addr_sk","required":false,"type":"long"},{"id":7,"name":"cs_ship_customer_sk","required":false,"type":"long"},{"id":8,"name":"cs_ship_cdemo_sk","required":false,"type":"long"},{"id":9,"name":"cs_ship_hdemo_sk","required":false,"type":"long"},{"id":10,"name":"cs_ship_addr_sk","required":false,"type":"long"},{"id":11,"name":"cs_call_center_sk","required":false,"type":"long"},{"id":12,"name":"cs_catalog_page_sk","required":false,"type":"long"},{"id":13,"name":"cs_ship_mode_sk","required":false,"type":"long"},{"id":14,"name":"cs_warehouse_sk","required":false,"type":"long"},{"id":15,"name":"cs_item_sk","required":false,"type":"long"},{"id":16,"name":"cs_promo_sk","required":false,"type":"long"},{"id":17,"name":"cs_order_number","required":false,"type":"long"},{"id":18,"name":"cs_quantity","required":false,"type":"int"},{"id":19,"name":"cs_wholesale_cost","required":false,"type":"decimal(7, 2)"},{"id":20,"name":"cs_list_price","required":false,"type":"decimal(7, 2)"},{"id":21,"name":"cs_sales_price","required":false,"type":"decimal(7, 2)"},{"id":22,"name":"cs_ext_discount_amt","required":false,"type":"decimal(7, 2)"},{"id":23,"name":"cs_ext_sales_price","required":false,"type":"decimal(7, 2)"},{"id":24,"name":"cs_ext_wholesale_cost","required":false,"type":"decimal(7, 2)"},{"id":25,"name":"cs_ext_list_price","required":false,"type":"decimal(7, 2)"},{"id":26,"name":"cs_ext_tax","required":false,"type":"decimal(7, 2)"},{"id":27,"name":"cs_coupon_amt","required":false,"type":"decimal(7, 2)"},{"id":28,"name":"cs_ext_ship_cost","required":false,"type":"decimal(7, 2)"},{"id":29,"name":"cs_net_paid","required":false,"type":"decimal(7, 2)"},{"id":30,"name":"cs_net_paid_inc_tax","required":false,"type":"decimal(7, 2)"},{"id":31,"name":"cs_net_paid_inc_ship","required":false,"type":"decimal(7, 2)"},{"id":32,"name":"cs_net_paid_inc_ship_tax","required":false,"type":"decimal(7, 2)"},{"id":33,"name":"cs_net_profit","required":false,"type":"decimal(7, 2)"},{"id":34,"name":"cs_sold_date_sk","required":false,"type":"long"}]}avro.schema¶/{"type":"record","name":"manifest_entry","fields":[{"name":"status","type":"int","field-id":0},{"name":"snapshot_id","type":["null","long"],"default":null,"field-id":1},{"name":"data_file","type":{"type":"record","name":"r2","fields":[{"name":"file_path","type":"string","doc":"Location URI with FS scheme","field-id":100},{"name":"file_format","type":"string","doc":"File format name: avro, orc, or parquet","field-id":101},{"name":"partition","type":{"type":"record","name":"r102","fields":[{"name":"cs_sold_date_sk","type":["null","long"],"default":null,"field-id":1000}]},"field-id":102},{"name":"record_count","type":"long","doc":"Number of records in the file","field-id":103},{"name":"file_size_in_bytes","type":"long","doc":"Total file size in bytes","field-id":104},{"name":"block_size_in_bytes","type":"long","field-id":105},{"name":"column_sizes","type":["null",{"type":"array","items":{"type":"record","name":"k117_v118","fields":[{"name":"key","type":"int","field-id":117},{"name":"value","type":"long","field-id":118}]},"logicalType":"map"}],"doc":"Map of column id to total size on disk","default":null,"field-id":108},{"name":"value_counts","type":["null",{"type":"array","items":{"type":"record","name":"k119_v120","fields":[{"name":"key","type":"int","field-id":119},{"name":"value","type":"long","field-id":120}]},"logicalType":"map"}],"doc":"Map of column id to total count, including null and NaN","default":null,"field-id":109},{"name":"null_value_counts","type":["null",{"type":"array","items":{"type":"record","name":"k121_v122","fields":[{"name":"key","type":"int","field-id":121},{"name":"value","type":"long","field-id":122}]},"logicalType":"map"}],"doc":"Map of column id to null value count","default":null,"field-id":110},{"name":"nan_value_counts","type":["null",{"type":"array","items":{"type":"record","name":"k138_v139","fields":[{"name":"key","type":"int","field-id":138},{"name":"value","type":"long","field-id":139}]},"logicalType":"map"}],"doc":"Map of column id to number of NaN values in the column","default":null,"field-id":137},{"name":"lower_bounds","type":["null",{"type":"array","items":{"type":"record","name":"k126_v127","fields":[{"name":"key","type":"int","field-id":126},{"name":"value","type":"bytes","field-id":127}]},"logicalType":"map"}],"doc":"Map of column id to lower bound","default":null,"field-id":125},{"name":"upper_bounds","type":["null",{"type":"array","items":{"type":"record","name":"k129_v130","fields":[{"name":"key","type":"int","field-id":129},{"name":"value","type":"bytes","field-id":130}]},"logicalType":"map"}],"doc":"Map of column id to upper bound","default":null,"field-id":128},{"name":"key_metadata","type":["null","bytes"],"doc":"Encryption key metadata blob","default":null,"field-id":131},{"name":"split_offsets","type":["null",{"type":"array","items":"long","element-id":133}],"doc":"Splittable offsets","default":null,"field-id":132},{"name":"sort_order_id","type":["null","int"],"doc":"Sort order ID","default":null,"field-id":140}]},"field-id":2}]}avro.codecdeflateformat-version1"partition-spec-id0iceberg.schemað%{"type":"struct","schema-id":0,"fields":[{"id":0,"name":"status","required":true,"type":"int"},{"id":1,"name":"snapshot_id","required":false,"type":"long"},{"id":2,"name":"data_file","required":true,"type":{"type":"struct","fields":[{"id":100,"name":"file_path","required":true,"type":"string","doc":"Location URI with FS scheme"},{"id":101,"name":"file_format","required":true,"type":"string","doc":"File format name: avro, orc, or parquet"},{"id":102,"name":"partition","required":true,"type":{"type":"struct","fields":[{"id":1000,"name":"cs_sold_date_sk","required":false,"type":"long"}]}},{"id":103,"name":"record_count","required":true,"type":"long","doc":"Number of records in the file"},{"id":104,"name":"file_size_in_bytes","required":true,"type":"long","doc":"Total file size in bytes"},{"id":105,"name":"block_size_in_bytes","required":true,"type":"long"},{"id":108,"name":"column_sizes","required":false,"type":{"type":"map","key-id":117,"key":"int","value-id":118,"value":"long","value-required":true},"doc":"Map of column id to total size on disk"},{"id":109,"name":"value_counts","required":false,"type":{"type":"map","key-id":119,"key":"int","value-id":120,"value":"long","value-required":true},"doc":"Map of column id to total count, including null and NaN"},{"id":110,"name":"null_value_counts","required":false,"type":{"type":"map","key-id":121,"key":"int","value-id":122,"value":"long","value-required":true},"doc":"Map of column id to null value count"},{"id":137,"name":"nan_value_counts","required":false,"type":{"type":"map","key-id":138,"key":"int","value-id":139,"value":"long","value-required":true},"doc":"Map of column id to number of NaN values in the column"},{"id":125,"name":"lower_bounds","required":false,"type":{"type":"map","key-id":126,"key":"int","value-id":127,"value":"binary","value-required":true},"doc":"Map of column id to lower bound"},{"id":128,"name":"upper_bounds","required":false,"type":{"type":"map","key-id":129,"key":"int","value-id":130,"value":"binary","value-required":true},"doc":"Map of column id to upper bound"},{"id":131,"name":"key_metadata","required":false,"type":"binary","doc":"Encryption key metadata blob"},{"id":132,"name":"split_offsets","required":false,"type":{"type":"list","element-id":133,"element":"long","element-required":true},"doc":"Splittable offsets"},{"id":140,"name":"sort_order_id","required":false,"type":"int","doc":"Sort order ID"}]}}]}partition-spec¤[{"name":"cs_sold_date_sk","transform":"identity","source-id":34,"field-id":1000}] í¸£W_´A¨xBmÃtŒ§Ä½\[õ½ÿߤñs³,ç˜eY–òƒˆá$M²“p8ÒCŒww;»éîþ¹»ïÝ€Àœ:ñO»©w:DDJ±"bÅŠˆˆ)EJ+"E¤ˆˆˆˆ+"ÖZEŠˆô÷ú09ò“çïò}ôîñ|¯·'{xιËó¾ÞŸQ*wôõ|4þT•ã¸òúXß~pýÖ”ëR·]wýVWjúÕi—ÿ9åº+¯wS¶¦üàOié©é×ýѵõš´àõ®ë3<ǹ®I¹îÚmé[É¿oýA>vUæ}ÊUé×ÿ`ñ?”výï¯Ï¼*ø{ü/é¿¿þÊD/çø¸x9¯—‹õò¿ç8o