From 6881afbc2a967ed71ba5853a3d8385524d657ccc Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 5 Jun 2024 13:43:17 -0700 Subject: [PATCH] Initial commit of Iceberg integration. (#5277) * Initial commit of Iceberg integration. * Tests simplified and passing. * Exposing 'iceberg-aws' in gradle. * Addressing PR comments. --- buildSrc/src/main/groovy/Classpaths.groovy | 45 +- extensions/iceberg/build.gradle | 39 ++ extensions/iceberg/gradle.properties | 1 + extensions/iceberg/s3/build.gradle | 41 ++ extensions/iceberg/s3/gradle.properties | 4 + .../iceberg/util/IcebergToolsS3.java | 65 ++ .../iceberg/util/IcebergLocalStackTest.java | 31 + .../iceberg/util/IcebergMinIOTest.java | 35 ++ .../iceberg/util/IcebergToolsTest.java | 594 ++++++++++++++++++ ...e7d-4830-aedb-14dab6fabdd6-0-00001.parquet | 3 + ...f10-4e08-89ae-f1b1b578ce63-0-00001.parquet | 3 + ...256-4a04-9248-d7c69bec4881-0-00001.parquet | 3 + ...3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet | 3 + ...-a414-468b-b7b2-78558b2e8c1f.metadata.json | 91 +++ ...-9c8a-4fa9-911d-e356b2b04061.metadata.json | 118 ++++ ...-a010-4afe-8461-f5261787aae9.metadata.json | 145 +++++ ...-48c8-4f6d-bd6b-9b452de74dc3.metadata.json | 172 +++++ ...f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro | Bin 0 -> 6976 bytes ...be90a2-7294-4ebb-8eee-20d512fff8b0-m0.avro | Bin 0 -> 6978 bytes ...bfad1c-123c-452c-814d-298a1483a99f-m0.avro | Bin 0 -> 6978 bytes ...1cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro | Bin 0 -> 6978 bytes ...-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro | Bin 0 -> 4399 bytes ...-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro | Bin 0 -> 4224 bytes ...-77bfad1c-123c-452c-814d-298a1483a99f.avro | Bin 0 -> 4353 bytes ...-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro | Bin 0 -> 4302 bytes ...66f-4474-a2a5-ad5e580e8f54-0-00001.parquet | 3 + ...66f-4474-a2a5-ad5e580e8f54-0-00001.parquet | 3 + ...-14fb-4a05-b8af-2c9f5986dfe6.metadata.json | 111 ++++ ...305b6c-008f-407a-8e3b-ea1187f6421e-m0.avro | Bin 0 -> 7627 bytes ...-4c305b6c-008f-407a-8e3b-ea1187f6421e.avro | Bin 0 -> 4251 bytes ...7ce-4968-b83d-30b13e1e7151-0-00001.parquet | 3 + ...-ae4e-42dd-ab04-884f6fd52372.metadata.json | 91 +++ ...95af41-65d8-41f0-8674-13624d6129c6-m0.avro | Bin 0 -> 6982 bytes ...-3395af41-65d8-41f0-8674-13624d6129c6.avro | Bin 0 -> 4227 bytes ...528ac1-dea2-472b-b72b-48c4fdccc8bd.parquet | 3 + ...-4659-434a-b422-227b8b15dfb9.metadata.json | 100 +++ ...-5cf6-4266-9f0c-5ae723cd77ba.metadata.json | 130 ++++ ...8d67e5-5760-40c1-a460-c8c4ee5b8629-m0.avro | Bin 0 -> 7582 bytes ...-388d67e5-5760-40c1-a460-c8c4ee5b8629.avro | Bin 0 -> 4226 bytes .../iceberg/layout/IcebergBaseLayout.java | 145 +++++ .../iceberg/layout/IcebergFlatLayout.java | 46 ++ .../IcebergKeyValuePartitionedLayout.java | 104 +++ .../location/IcebergTableLocationFactory.java | 33 + .../location/IcebergTableLocationKey.java | 18 + .../IcebergTableParquetLocationKey.java | 51 ++ .../iceberg/util/IcebergCatalogAdapter.java | 462 ++++++++++++++ .../iceberg/util/IcebergInstructions.java | 56 ++ .../deephaven/iceberg/util/IcebergTools.java | 19 + .../TestCatalog/IcebergTestCatalog.java | 106 ++++ .../TestCatalog/IcebergTestFileIO.java | 49 ++ .../iceberg/TestCatalog/IcebergTestTable.java | 239 +++++++ extensions/parquet/table/build.gradle | 4 +- .../parquet/table/ParquetInstructions.java | 12 +- extensions/s3/build.gradle | 9 +- py/embedded-server/java-runtime/build.gradle | 1 + server/build.gradle | 1 + server/jetty-app-custom/build.gradle | 1 + server/jetty-app/build.gradle | 1 + server/netty-app/build.gradle | 1 + settings.gradle | 6 + 60 files changed, 3184 insertions(+), 17 deletions(-) create mode 100644 extensions/iceberg/build.gradle create mode 100644 extensions/iceberg/gradle.properties create mode 100644 extensions/iceberg/s3/build.gradle create mode 100644 extensions/iceberg/s3/gradle.properties create mode 100644 extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java create mode 100644 extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergLocalStackTest.java create mode 100644 extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergMinIOTest.java create mode 100644 extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/02f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/51be90a2-7294-4ebb-8eee-20d512fff8b0-m0.avro create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/77bfad1c-123c-452c-814d-298a1483a99f-m0.avro create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/a61cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-1792185872197984875-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_partitioned/data/year=2022/month=11/00000-8-981b110e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_partitioned/data/year=2022/month=12/00001-9-981b110e-f66f-4474-a2a5-ad5e580e8f54-0-00001.parquet create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_partitioned/metadata/00000-9635bc47-14fb-4a05-b8af-2c9f5986dfe6.metadata.json create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_partitioned/metadata/4c305b6c-008f-407a-8e3b-ea1187f6421e-m0.avro create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_partitioned/metadata/snap-5230665555732911945-1-4c305b6c-008f-407a-8e3b-ea1187f6421e.avro create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_single/data/00000-2-1b3d5352-57ce-4968-b83d-30b13e1e7151-0-00001.parquet create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_single/metadata/00000-3a7beb71-ae4e-42dd-ab04-884f6fd52372.metadata.json create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_single/metadata/3395af41-65d8-41f0-8674-13624d6129c6-m0.avro create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_single/metadata/snap-5481576066981634597-1-3395af41-65d8-41f0-8674-13624d6129c6.avro create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/ac528ac1-dea2-472b-b72b-48c4fdccc8bd.parquet create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/metadata/00000-bbb283f2-4659-434a-b422-227b8b15dfb9.metadata.json create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/metadata/00001-15b8eb1c-5cf6-4266-9f0c-5ae723cd77ba.metadata.json create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/metadata/388d67e5-5760-40c1-a460-c8c4ee5b8629-m0.avro create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/metadata/snap-6186754175552482648-1-388d67e5-5760-40c1-a460-c8c4ee5b8629.avro create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocationKey.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java create mode 100644 extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestCatalog.java create mode 100644 extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestFileIO.java create mode 100644 extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestTable.java diff --git a/buildSrc/src/main/groovy/Classpaths.groovy b/buildSrc/src/main/groovy/Classpaths.groovy index a09da2269ad..caf4f5769d1 100644 --- a/buildSrc/src/main/groovy/Classpaths.groovy +++ b/buildSrc/src/main/groovy/Classpaths.groovy @@ -123,6 +123,18 @@ class Classpaths { static final String GUAVA_NAME = 'guava' static final String GUAVA_VERSION = '33.2.0-jre' + static final String HADOOP_GROUP = 'org.apache.hadoop' + static final String HADOOP_VERSION = '3.4.0' + + static final String ICEBERG_GROUP = 'org.apache.iceberg' + static final String ICEBERG_VERSION = '1.5.0' + + static final String AWSSDK_GROUP = 'software.amazon.awssdk' + static final String AWSSDK_VERSION = '2.23.19' + + static final String TESTCONTAINER_GROUP = 'org.testcontainers' + static final String TESTCONTAINER_VERSION = '1.19.4' + static boolean addDependency(Configuration conf, String group, String name, String version, Action configure = Actions.doNothing()) { if (!conf.dependencies.find { it.name == name && it.group == group}) { DefaultExternalModuleDependency dep = dependency group, name, version @@ -295,7 +307,7 @@ class Classpaths { /** configName controls only the Configuration's classpath, all transitive dependencies are runtimeOnly */ static void inheritParquetHadoopConfiguration(Project p, String configName = JavaPlugin.IMPLEMENTATION_CONFIGURATION_NAME) { Configuration config = p.configurations.getByName(configName) - addDependency(config, 'org.apache.hadoop', 'hadoop-common', '3.4.0') { + addDependency(config, HADOOP_GROUP, 'hadoop-common', HADOOP_VERSION) { it.setTransitive(false) // Do not take any extra dependencies of this project transitively. We just want a few classes for // configuration and compression codecs. For any additional required dependencies, add them separately, as @@ -314,4 +326,35 @@ class Classpaths { it.because('hadoop-common required dependency for Configuration') } } + + static void inheritIcebergHadoop(Project p, String configName = JavaPlugin.IMPLEMENTATION_CONFIGURATION_NAME) { + Configuration config = p.configurations.getByName(configName) + addDependency(config, HADOOP_GROUP, 'hadoop-common', HADOOP_VERSION) + addDependency(config, HADOOP_GROUP, 'hadoop-hdfs-client', HADOOP_VERSION) + } + + + static void inheritIcebergCore(Project p, String configName = JavaPlugin.IMPLEMENTATION_CONFIGURATION_NAME) { + Configuration config = p.configurations.getByName(configName) + addDependency(config, p.getDependencies().platform(ICEBERG_GROUP + ":iceberg-bom:" + ICEBERG_VERSION)) + + addDependency(config, ICEBERG_GROUP, 'iceberg-core', ICEBERG_VERSION) + addDependency(config, ICEBERG_GROUP, 'iceberg-bundled-guava', ICEBERG_VERSION) + } + + static void inheritAWSSDK(Project p, String configName = JavaPlugin.IMPLEMENTATION_CONFIGURATION_NAME) { + Configuration config = p.configurations.getByName(configName) + addDependency(config, p.getDependencies().platform(AWSSDK_GROUP + ":bom:" + AWSSDK_VERSION)) + + addDependency(config, AWSSDK_GROUP, 's3', AWSSDK_VERSION) + addDependency(config, AWSSDK_GROUP, 'aws-crt-client', AWSSDK_VERSION) + } + + static void inheritTestContainers(Project p, String configName = JavaPlugin.TEST_IMPLEMENTATION_CONFIGURATION_NAME) { + Configuration config = p.configurations.getByName(configName) + addDependency(config, TESTCONTAINER_GROUP, 'testcontainers', TESTCONTAINER_VERSION) + addDependency(config, TESTCONTAINER_GROUP, 'junit-jupiter', TESTCONTAINER_VERSION) + addDependency(config, TESTCONTAINER_GROUP, 'localstack', TESTCONTAINER_VERSION) + addDependency(config, TESTCONTAINER_GROUP, 'minio', TESTCONTAINER_VERSION) + } } diff --git a/extensions/iceberg/build.gradle b/extensions/iceberg/build.gradle new file mode 100644 index 00000000000..2eba0158fb1 --- /dev/null +++ b/extensions/iceberg/build.gradle @@ -0,0 +1,39 @@ +plugins { + id 'java-library' + id 'io.deephaven.project.register' +} + +description 'Iceberg: Support to read iceberg catalogs.' + +dependencies { + api project(':engine-api') + api project(':engine-table') + + implementation project(':engine-base') + implementation project(':log-factory') + implementation project(':Configuration') + + Classpaths.inheritAutoService(project) + Classpaths.inheritImmutables(project) + + Classpaths.inheritParquetHadoop(project) + + implementation project(':extensions-parquet-base') + implementation project(':extensions-parquet-table') + + Classpaths.inheritIcebergCore(project) + Classpaths.inheritIcebergHadoop(project) + + Classpaths.inheritJUnitPlatform(project) + Classpaths.inheritAssertJ(project) + + testImplementation 'org.junit.jupiter:junit-jupiter' + testRuntimeOnly 'org.junit.jupiter:junit-jupiter-engine' + testRuntimeOnly 'org.junit.platform:junit-platform-launcher' + + Classpaths.inheritTestContainers(project) + + testRuntimeOnly project(':test-configs') + testRuntimeOnly project(':log-to-slf4j') + Classpaths.inheritSlf4j(project, 'slf4j-simple', 'testRuntimeOnly') +} diff --git a/extensions/iceberg/gradle.properties b/extensions/iceberg/gradle.properties new file mode 100644 index 00000000000..c186bbfdde1 --- /dev/null +++ b/extensions/iceberg/gradle.properties @@ -0,0 +1 @@ +io.deephaven.project.ProjectType=JAVA_PUBLIC diff --git a/extensions/iceberg/s3/build.gradle b/extensions/iceberg/s3/build.gradle new file mode 100644 index 00000000000..c07df457cf7 --- /dev/null +++ b/extensions/iceberg/s3/build.gradle @@ -0,0 +1,41 @@ +plugins { + id 'java-library' + id 'io.deephaven.project.register' +} + +description 'Iceberg: Support to read iceberg catalogs.' + +dependencies { + implementation project(':extensions-iceberg') + + // Bring in the AWS / S3 extensions + Classpaths.inheritIcebergCore(project) + + implementation project(':extensions-s3') + implementation "org.apache.iceberg:iceberg-aws" + runtimeOnly "org.apache.iceberg:iceberg-aws-bundle" + Classpaths.inheritAWSSDK(project) + + Classpaths.inheritTestContainers(project) + + testImplementation TestTools.projectDependency(project, 'extensions-s3') + testImplementation TestTools.projectDependency(project, 'extensions-iceberg') + + testRuntimeOnly project(':test-configs') + testRuntimeOnly project(':log-to-slf4j') + Classpaths.inheritSlf4j(project, 'slf4j-simple', 'testRuntimeOnly') +} + +test { + useJUnitPlatform { + excludeTags("testcontainers") + } +} + +tasks.register('testOutOfBand', Test) { + useJUnitPlatform { + includeTags("testcontainers") + } + systemProperty 'testcontainers.localstack.image', project.property('testcontainers.localstack.image') + systemProperty 'testcontainers.minio.image', project.property('testcontainers.minio.image') +} diff --git a/extensions/iceberg/s3/gradle.properties b/extensions/iceberg/s3/gradle.properties new file mode 100644 index 00000000000..cfd384f094a --- /dev/null +++ b/extensions/iceberg/s3/gradle.properties @@ -0,0 +1,4 @@ +io.deephaven.project.ProjectType=JAVA_PUBLIC + +testcontainers.localstack.image=localstack/localstack:3.1.0 +testcontainers.minio.image=minio/minio:RELEASE.2024-02-04T22-36-13Z diff --git a/extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java b/extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java new file mode 100644 index 00000000000..6f7845c43eb --- /dev/null +++ b/extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java @@ -0,0 +1,65 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.util; + +import com.google.common.base.Strings; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.aws.AwsClientProperties; +import org.apache.iceberg.aws.s3.S3FileIOProperties; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.rest.RESTCatalog; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.HashMap; +import java.util.Map; + +/** + * Tools for accessing tables in the Iceberg table format. + */ +@SuppressWarnings("unused") +public class IcebergToolsS3 extends IcebergTools { + private static final String S3_FILE_IO_CLASS = "org.apache.iceberg.aws.s3.S3FileIO"; + + public static IcebergCatalogAdapter createS3Rest( + @Nullable final String name, + @NotNull final String catalogURI, + @NotNull final String warehouseLocation, + @Nullable final String region, + @Nullable final String accessKeyId, + @Nullable final String secretAccessKey, + @Nullable final String endpointOverride) { + + // Set up the properties map for the Iceberg catalog + final Map properties = new HashMap<>(); + + final RESTCatalog catalog = new RESTCatalog(); + + properties.put(CatalogProperties.CATALOG_IMPL, catalog.getClass().getName()); + properties.put(CatalogProperties.URI, catalogURI); + properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation); + + // Configure the properties map from the Iceberg instructions. + if (!Strings.isNullOrEmpty(accessKeyId) && !Strings.isNullOrEmpty(secretAccessKey)) { + properties.put(S3FileIOProperties.ACCESS_KEY_ID, accessKeyId); + properties.put(S3FileIOProperties.SECRET_ACCESS_KEY, secretAccessKey); + } + if (!Strings.isNullOrEmpty(region)) { + properties.put(AwsClientProperties.CLIENT_REGION, region); + } + if (!Strings.isNullOrEmpty(endpointOverride)) { + properties.put(S3FileIOProperties.ENDPOINT, endpointOverride); + } + + // TODO: create a FileIO interface wrapping the Deephaven S3SeekableByteChannel/Provider + final FileIO fileIO = CatalogUtil.loadFileIO(S3_FILE_IO_CLASS, properties, null); + + final String catalogName = name != null ? name : "IcebergCatalog-" + catalogURI; + catalog.initialize(catalogName, properties); + + return new IcebergCatalogAdapter(catalog, fileIO); + } + +} diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergLocalStackTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergLocalStackTest.java new file mode 100644 index 00000000000..578e358985e --- /dev/null +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergLocalStackTest.java @@ -0,0 +1,31 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.util; + + +import io.deephaven.extensions.s3.S3Instructions.Builder; +import io.deephaven.extensions.s3.testlib.SingletonContainers; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import software.amazon.awssdk.services.s3.S3AsyncClient; + +@Tag("testcontainers") +public class IcebergLocalStackTest extends IcebergToolsTest { + + @BeforeAll + static void initContainer() { + // ensure container is started so container startup time isn't associated with a specific test + SingletonContainers.LocalStack.init(); + } + + @Override + public Builder s3Instructions(Builder builder) { + return SingletonContainers.LocalStack.s3Instructions(builder); + } + + @Override + public S3AsyncClient s3AsyncClient() { + return SingletonContainers.LocalStack.s3AsyncClient(); + } +} diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergMinIOTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergMinIOTest.java new file mode 100644 index 00000000000..804d2d01746 --- /dev/null +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergMinIOTest.java @@ -0,0 +1,35 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.util; + + +import io.deephaven.extensions.s3.S3Instructions.Builder; +import io.deephaven.extensions.s3.testlib.SingletonContainers; +import io.deephaven.stats.util.OSUtil; +import org.junit.jupiter.api.Assumptions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import software.amazon.awssdk.services.s3.S3AsyncClient; + +@Tag("testcontainers") +public class IcebergMinIOTest extends IcebergToolsTest { + + @BeforeAll + static void initContainer() { + // TODO(deephaven-core#5116): MinIO testcontainers does not work on OS X + Assumptions.assumeFalse(OSUtil.runningMacOS(), "OSUtil.runningMacOS()"); + // ensure container is started so container startup time isn't associated with a specific test + SingletonContainers.MinIO.init(); + } + + @Override + public Builder s3Instructions(Builder builder) { + return SingletonContainers.MinIO.s3Instructions(builder); + } + + @Override + public S3AsyncClient s3AsyncClient() { + return SingletonContainers.MinIO.s3AsyncClient(); + } +} diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java new file mode 100644 index 00000000000..0fd3b3fcf7e --- /dev/null +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java @@ -0,0 +1,594 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.util; + +import gnu.trove.list.array.TLongArrayList; +import io.deephaven.base.verify.Assert; +import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.table.impl.locations.TableDataException; +import io.deephaven.engine.util.TableTools; +import io.deephaven.extensions.s3.S3Instructions; +import io.deephaven.iceberg.TestCatalog.IcebergTestCatalog; +import io.deephaven.iceberg.TestCatalog.IcebergTestFileIO; +import io.deephaven.time.DateTimeUtils; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.io.FileIO; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import software.amazon.awssdk.core.async.AsyncRequestBody; +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.model.*; + +import java.io.File; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + +public abstract class IcebergToolsTest { + IcebergInstructions instructions; + + public abstract S3AsyncClient s3AsyncClient(); + + public abstract S3Instructions.Builder s3Instructions(S3Instructions.Builder builder); + + private S3AsyncClient asyncClient; + private String bucket; + + private final List keys = new ArrayList<>(); + + private String warehousePath; + private Catalog resourceCatalog; + private FileIO resourceFileIO; + + @BeforeEach + void setUp() throws ExecutionException, InterruptedException { + bucket = "warehouse"; + asyncClient = s3AsyncClient(); + asyncClient.createBucket(CreateBucketRequest.builder().bucket(bucket).build()).get(); + + warehousePath = IcebergToolsTest.class.getResource("/warehouse").getPath(); + resourceFileIO = new IcebergTestFileIO("s3://warehouse", warehousePath); + + // Create the test catalog for the tests + resourceCatalog = IcebergTestCatalog.create(warehousePath, resourceFileIO); + + final S3Instructions s3Instructions = s3Instructions(S3Instructions.builder()).build(); + + instructions = IcebergInstructions.builder() + .dataInstructions(s3Instructions) + .build(); + } + + private void uploadParquetFiles(final File root, final String prefixToRemove) + throws ExecutionException, InterruptedException, TimeoutException { + for (final File file : root.listFiles()) { + if (file.isDirectory()) { + uploadParquetFiles(file, prefixToRemove); + } else if (file.getName().endsWith(".parquet")) { + final String key = file.getPath().substring(prefixToRemove.length() + 1); + + keys.add(key); + final CompletableFuture future = asyncClient.putObject( + PutObjectRequest.builder().bucket(bucket).key(key).build(), + AsyncRequestBody.fromFile(file)); + + final PutObjectResponse response = future.get(10, TimeUnit.SECONDS); + if (!response.sdkHttpResponse().isSuccessful()) { + Assert.statementNeverExecuted("Failed to upload file: " + file.getPath()); + } + } + } + } + + @AfterEach + public void tearDown() throws ExecutionException, InterruptedException { + for (String key : keys) { + asyncClient.deleteObject(DeleteObjectRequest.builder().bucket(bucket).key(key).build()).get(); + } + keys.clear(); + asyncClient.deleteBucket(DeleteBucketRequest.builder().bucket(bucket).build()).get(); + asyncClient.close(); + } + + @Test + public void testListNamespaces() { + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Collection namespaces = adapter.listNamespaces(); + final Collection namespaceNames = + namespaces.stream().map(Namespace::toString).collect(Collectors.toList()); + + Assert.eq(namespaceNames.size(), "namespaceNames.size()", 2, "2 namespace in the catalog"); + Assert.eqTrue(namespaceNames.contains("sales"), "namespaceNames.contains(sales)"); + Assert.eqTrue(namespaceNames.contains("sample"), "namespaceNames.contains(sample)"); + + final Table table = adapter.listNamespacesAsTable(); + Assert.eq(table.size(), "table.size()", 2, "2 namespace in the catalog"); + Assert.eqTrue(table.getColumnSource("namespace").getType().equals(String.class), "namespace column type"); + Assert.eqTrue(table.getColumnSource("namespace_object").getType().equals(Namespace.class), + "namespace_object column type"); + } + + @Test + public void testListTables() { + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Namespace ns = Namespace.of("sales"); + + final Collection tables = adapter.listTables(ns); + Assert.eq(tables.size(), "tables.size()", 3, "3 tables in the namespace"); + Assert.eqTrue(tables.contains(TableIdentifier.of(ns, "sales_multi")), "tables.contains(sales_multi)"); + Assert.eqTrue(tables.contains(TableIdentifier.of(ns, "sales_partitioned")), + "tables.contains(sales_partitioned)"); + Assert.eqTrue(tables.contains(TableIdentifier.of(ns, "sales_single")), "tables.contains(sales_single)"); + + final Table table = adapter.listTablesAsTable(ns); + Assert.eq(table.size(), "table.size()", 3, "3 tables in the namespace"); + Assert.eqTrue(table.getColumnSource("namespace").getType().equals(String.class), "namespace column type"); + Assert.eqTrue(table.getColumnSource("table_name").getType().equals(String.class), "table_name column type"); + Assert.eqTrue(table.getColumnSource("table_identifier_object").getType().equals(TableIdentifier.class), + "table_identifier_object column type"); + } + + @Test + public void testListSnapshots() { + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final TLongArrayList snapshotIds = new TLongArrayList(); + final TableIdentifier tableIdentifier = TableIdentifier.of("sales", "sales_multi"); + adapter.listSnapshots(tableIdentifier) + .forEach(snapshot -> snapshotIds.add(snapshot.snapshotId())); + + Assert.eq(snapshotIds.size(), "snapshots.size()", 4, "4 snapshots for sales/sales_multi"); + + Assert.eqTrue(snapshotIds.contains(2001582482032951248L), "snapshots.contains(2001582482032951248)"); + Assert.eqTrue(snapshotIds.contains(8325605756612719366L), "snapshots.contains(8325605756612719366L)"); + Assert.eqTrue(snapshotIds.contains(3247344357341484163L), "snapshots.contains(3247344357341484163L)"); + Assert.eqTrue(snapshotIds.contains(1792185872197984875L), "snapshots.contains(1792185872197984875L)"); + + final Table table = adapter.listSnapshotsAsTable(tableIdentifier); + Assert.eq(table.size(), "table.size()", 4, "4 snapshots for sales/sales_multi"); + Assert.eqTrue(table.getColumnSource("id").getType().equals(long.class), "id column type"); + Assert.eqTrue(table.getColumnSource("timestamp_ms").getType().equals(long.class), "timestamp_ms column type"); + Assert.eqTrue(table.getColumnSource("operation").getType().equals(String.class), "operation column type"); + Assert.eqTrue(table.getColumnSource("summary").getType().equals(Map.class), "summary column type"); + Assert.eqTrue(table.getColumnSource("snapshot_object").getType().equals(Snapshot.class), + "snapshot_object column type"); + } + + @Test + public void testOpenTableA() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_partitioned").getPath()), + warehousePath); + + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); + + // Verify we retrieved all the rows. + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + } + + @Test + public void testOpenTableB() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_multi").getPath()), + warehousePath); + + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); + + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + } + + @Test + public void testOpenTableC() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_single").getPath()), + warehousePath); + + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_single"); + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); + + // Verify we retrieved all the rows. + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + } + + @Test + public void testOpenTableS3Only() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_partitioned").getPath()), + warehousePath); + + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); + + // Verify we retrieved all the rows. + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + } + + @Test + public void testOpenTableDefinition() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_partitioned").getPath()), + warehousePath); + + final TableDefinition tableDef = TableDefinition.of( + ColumnDefinition.ofInt("year").withPartitioning(), + ColumnDefinition.ofInt("month").withPartitioning(), + ColumnDefinition.ofString("Region"), + ColumnDefinition.ofString("Item_Type"), + ColumnDefinition.ofInt("Units_Sold"), + ColumnDefinition.ofDouble("Unit_Price"), + ColumnDefinition.fromGenericType("Order_Date", Instant.class)); + + final IcebergInstructions localInstructions = IcebergInstructions.builder() + .tableDefinition(tableDef) + .dataInstructions(instructions.dataInstructions().get()) + .build(); + + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); + + // Verify we retrieved all the rows. + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + } + + @Test + public void testOpenTablePartitionTypeException() { + final TableDefinition tableDef = TableDefinition.of( + ColumnDefinition.ofLong("year").withPartitioning(), + ColumnDefinition.ofInt("month").withPartitioning(), + ColumnDefinition.ofLong("Region"), + ColumnDefinition.ofString("Item_Type"), + ColumnDefinition.ofDouble("Units_Sold"), + ColumnDefinition.ofLong("Unit_Price"), + ColumnDefinition.fromGenericType("Order_Date", Instant.class)); + + final IcebergInstructions localInstructions = IcebergInstructions.builder() + .tableDefinition(tableDef) + .dataInstructions(instructions.dataInstructions().get()) + .build(); + + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); + try { + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); + TableTools.showWithRowSet(table, 100, DateTimeUtils.timeZone(), System.out); + Assert.statementNeverExecuted("Expected an exception for missing columns"); + } catch (final TableDefinition.IncompatibleTableDefinitionException e) { + Assert.eqTrue(e.getMessage().startsWith("Table definition incompatibilities"), "Exception message"); + } + } + + @Test + public void testOpenTableDefinitionRename() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_partitioned").getPath()), + warehousePath); + + final TableDefinition tableDef = TableDefinition.of( + ColumnDefinition.ofInt("__year").withPartitioning(), + ColumnDefinition.ofInt("__month").withPartitioning(), + ColumnDefinition.ofString("RegionName"), + ColumnDefinition.ofString("ItemType"), + ColumnDefinition.ofInt("UnitsSold"), + ColumnDefinition.ofDouble("UnitPrice"), + ColumnDefinition.fromGenericType("OrderDate", Instant.class)); + + final IcebergInstructions localInstructions = IcebergInstructions.builder() + .tableDefinition(tableDef) + .dataInstructions(instructions.dataInstructions().get()) + .putColumnRenames("Region", "RegionName") + .putColumnRenames("Item_Type", "ItemType") + .putColumnRenames("Units_Sold", "UnitsSold") + .putColumnRenames("Unit_Price", "UnitPrice") + .putColumnRenames("Order_Date", "OrderDate") + .putColumnRenames("year", "__year") + .putColumnRenames("month", "__month") + .build(); + + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); + + // Verify we retrieved all the rows. + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + } + + @Test + public void testSkippedPartitioningColumn() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_partitioned").getPath()), + warehousePath); + + final TableDefinition tableDef = TableDefinition.of( + ColumnDefinition.ofInt("year").withPartitioning(), + // Omitting month partitioning column + ColumnDefinition.ofString("Region"), + ColumnDefinition.ofString("Item_Type"), + ColumnDefinition.ofInt("Units_Sold"), + ColumnDefinition.ofDouble("Unit_Price"), + ColumnDefinition.fromGenericType("Order_Date", Instant.class)); + + final IcebergInstructions localInstructions = IcebergInstructions.builder() + .tableDefinition(tableDef) + .dataInstructions(instructions.dataInstructions().get()) + .build(); + + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); + + // Verify we retrieved all the rows. + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + } + + @Test + public void testReorderedPartitioningColumn() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_partitioned").getPath()), + warehousePath); + + final TableDefinition tableDef = TableDefinition.of( + ColumnDefinition.ofInt("month").withPartitioning(), + ColumnDefinition.ofInt("year").withPartitioning(), + ColumnDefinition.ofString("Region"), + ColumnDefinition.ofString("Item_Type"), + ColumnDefinition.ofInt("Units_Sold"), + ColumnDefinition.ofDouble("Unit_Price"), + ColumnDefinition.fromGenericType("Order_Date", Instant.class)); + + final IcebergInstructions localInstructions = IcebergInstructions.builder() + .tableDefinition(tableDef) + .dataInstructions(instructions.dataInstructions().get()) + .build(); + + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); + + // Verify we retrieved all the rows. + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + } + + @Test + public void testZeroPartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_partitioned").getPath()), + warehousePath); + + final TableDefinition tableDef = TableDefinition.of( + ColumnDefinition.ofString("Region"), + ColumnDefinition.ofString("Item_Type"), + ColumnDefinition.ofInt("Units_Sold"), + ColumnDefinition.ofDouble("Unit_Price"), + ColumnDefinition.fromGenericType("Order_Date", Instant.class)); + + final IcebergInstructions localInstructions = IcebergInstructions.builder() + .tableDefinition(tableDef) + .dataInstructions(instructions.dataInstructions().get()) + .build(); + + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); + + // Verify we retrieved all the rows. + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + } + + @Test + public void testIncorrectPartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { + final TableDefinition tableDef = TableDefinition.of( + ColumnDefinition.ofInt("month").withPartitioning(), + ColumnDefinition.ofInt("year").withPartitioning(), + ColumnDefinition.ofString("Region").withPartitioning(), + ColumnDefinition.ofString("Item_Type"), + ColumnDefinition.ofInt("Units_Sold"), + ColumnDefinition.ofDouble("Unit_Price"), + ColumnDefinition.fromGenericType("Order_Date", Instant.class)); + + final IcebergInstructions localInstructions = IcebergInstructions.builder() + .tableDefinition(tableDef) + .dataInstructions(instructions.dataInstructions().get()) + .build(); + + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); + + try { + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); + Assert.statementNeverExecuted("Expected an exception for missing columns"); + } catch (final TableDataException e) { + Assert.eqTrue(e.getMessage().startsWith("The following columns are not partitioned"), "Exception message"); + } + } + + @Test + public void testMissingPartitioningColumns() { + final TableDefinition tableDef = TableDefinition.of( + ColumnDefinition.ofInt("__year").withPartitioning(), // Incorrect name + ColumnDefinition.ofInt("__month").withPartitioning(), // Incorrect name + ColumnDefinition.ofLong("Region"), + ColumnDefinition.ofString("Item_Type"), + ColumnDefinition.ofDouble("Units_Sold"), + ColumnDefinition.ofLong("Unit_Price"), + ColumnDefinition.fromGenericType("Order_Date", Instant.class)); + + final IcebergInstructions localInstructions = IcebergInstructions.builder() + .tableDefinition(tableDef) + .dataInstructions(instructions.dataInstructions().get()) + .build(); + + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); + try { + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); + Assert.statementNeverExecuted("Expected an exception for missing columns"); + } catch (final TableDefinition.IncompatibleTableDefinitionException e) { + Assert.eqTrue(e.getMessage().startsWith("Table definition incompatibilities"), "Exception message"); + } + } + + @Test + public void testOpenTableColumnRename() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_partitioned").getPath()), + warehousePath); + + final IcebergInstructions localInstructions = IcebergInstructions.builder() + .dataInstructions(instructions.dataInstructions().get()) + .putColumnRenames("RegionName", "Region") + .putColumnRenames("ItemType", "Item_Type") + .build(); + + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); + + // Verify we retrieved all the rows. + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + } + + @Test + public void testOpenTableColumnRenamePartitioningColumns() + throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_partitioned").getPath()), + warehousePath); + + final IcebergInstructions localInstructions = IcebergInstructions.builder() + .dataInstructions(instructions.dataInstructions().get()) + .putColumnRenames("VendorID", "vendor_id") + .putColumnRenames("month", "__month") + .putColumnRenames("year", "__year") + .build(); + + final IcebergCatalogAdapter adapter = + IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); + + // Verify we retrieved all the rows. + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + } + + @Test + public void testOpenTableSnapshot() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_multi").getPath()), + warehousePath); + + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); + final List snapshots = adapter.listSnapshots(tableId); + + // Verify we retrieved all the rows. + final io.deephaven.engine.table.Table table0 = + adapter.readTable(tableId, snapshots.get(0).snapshotId(), instructions); + Assert.eq(table0.size(), "table0.size()", 18266, "18266 rows in the table"); + + final io.deephaven.engine.table.Table table1 = + adapter.readTable(tableId, snapshots.get(1).snapshotId(), instructions); + Assert.eq(table1.size(), "table1.size()", 54373, "54373 rows in the table"); + + final io.deephaven.engine.table.Table table2 = + adapter.readTable(tableId, snapshots.get(2).snapshotId(), instructions); + Assert.eq(table2.size(), "table2.size()", 72603, "72603 rows in the table"); + + final io.deephaven.engine.table.Table table3 = + adapter.readTable(tableId, snapshots.get(3).snapshotId(), instructions); + Assert.eq(table3.size(), "table3.size()", 100_000, "100_000 rows in the table"); + } + + @Test + public void testOpenTableSnapshotByID() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sales/sales_multi").getPath()), + warehousePath); + + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); + final List snapshots = adapter.listSnapshots(tableId); + + // Verify we retrieved all the rows. + final io.deephaven.engine.table.Table table0 = adapter.readTable(tableId, snapshots.get(0), instructions); + Assert.eq(table0.size(), "table0.size()", 18266, "18266 rows in the table"); + + final io.deephaven.engine.table.Table table1 = adapter.readTable(tableId, snapshots.get(1), instructions); + Assert.eq(table1.size(), "table1.size()", 54373, "54373 rows in the table"); + + final io.deephaven.engine.table.Table table2 = adapter.readTable(tableId, snapshots.get(2), instructions); + Assert.eq(table2.size(), "table2.size()", 72603, "72603 rows in the table"); + + final io.deephaven.engine.table.Table table3 = adapter.readTable(tableId, snapshots.get(3), instructions); + Assert.eq(table3.size(), "table3.size()", 100_000, "100_000 rows in the table"); + } + + @Test + public void testOpenAllTypesTable() throws ExecutionException, InterruptedException, TimeoutException { + uploadParquetFiles(new File(IcebergToolsTest.class.getResource("/warehouse/sample/all_types").getPath()), + warehousePath); + + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Namespace ns = Namespace.of("sample"); + final TableIdentifier tableId = TableIdentifier.of(ns, "all_types"); + final List snapshots = adapter.listSnapshots(tableId); + + // Verify we retrieved all the rows. + final io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); + Assert.eq(table.size(), "table.size()", 10, "10 rows in the table"); + } +} diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet new file mode 100644 index 00000000000..04d259decea --- /dev/null +++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:9749ae2922aa9d21b7e779142d6c2476d0444c2c24f7e93397e6750147180724 +size 176970 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet new file mode 100644 index 00000000000..206e1f84a6c --- /dev/null +++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:d5007c70d0e8be33ef012c48d7f067b47812b22747c145cfa4bab4bef944fb0f +size 331675 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet new file mode 100644 index 00000000000..6d552bf0e9a --- /dev/null +++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:c02909448106787f93e8a2550e9db360aedd9ab25c925e8a536b18096aa8ed91 +size 176482 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet new file mode 100644 index 00000000000..540e5bc3392 --- /dev/null +++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:c872a3defa86826496fbc76fa4235c858d45edb5441e196c2c812f4c8a1166e7 +size 256855 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json new file mode 100644 index 00000000000..53c1afcba7b --- /dev/null +++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json @@ -0,0 +1,91 @@ +{ + "format-version" : 2, + "table-uuid" : "49ca5b8c-6402-40f9-96b0-4c09f9f1b512", + "location" : "s3://warehouse/sales/sales_multi", + "last-sequence-number" : 1, + "last-updated-ms" : 1716234545155, + "last-column-id" : 5, + "current-schema-id" : 0, + "schemas" : [ { + "type" : "struct", + "schema-id" : 0, + "fields" : [ { + "id" : 1, + "name" : "Region", + "required" : false, + "type" : "string" + }, { + "id" : 2, + "name" : "Item_Type", + "required" : false, + "type" : "string" + }, { + "id" : 3, + "name" : "Units_Sold", + "required" : false, + "type" : "int" + }, { + "id" : 4, + "name" : "Unit_Price", + "required" : false, + "type" : "double" + }, { + "id" : 5, + "name" : "Order_Date", + "required" : false, + "type" : "timestamptz" + } ] + } ], + "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" : { + "owner" : "root", + "created-at" : "2024-05-20T19:49:04.669537174Z", + "write.format.default" : "parquet", + "write.parquet.compression-codec" : "zstd" + }, + "current-snapshot-id" : 2001582482032951248, + "refs" : { + "main" : { + "snapshot-id" : 2001582482032951248, + "type" : "branch" + } + }, + "snapshots" : [ { + "sequence-number" : 1, + "snapshot-id" : 2001582482032951248, + "timestamp-ms" : 1716234545155, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1716234462547", + "added-data-files" : "1", + "added-records" : "18266", + "added-files-size" : "176970", + "changed-partition-count" : "1", + "total-records" : "18266", + "total-files-size" : "176970", + "total-data-files" : "1", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro", + "schema-id" : 0 + } ], + "statistics" : [ ], + "partition-statistics" : [ ], + "snapshot-log" : [ { + "timestamp-ms" : 1716234545155, + "snapshot-id" : 2001582482032951248 + } ], + "metadata-log" : [ ] +} \ No newline at end of file diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json new file mode 100644 index 00000000000..5aa1996ce92 --- /dev/null +++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json @@ -0,0 +1,118 @@ +{ + "format-version" : 2, + "table-uuid" : "49ca5b8c-6402-40f9-96b0-4c09f9f1b512", + "location" : "s3://warehouse/sales/sales_multi", + "last-sequence-number" : 2, + "last-updated-ms" : 1716234545529, + "last-column-id" : 5, + "current-schema-id" : 0, + "schemas" : [ { + "type" : "struct", + "schema-id" : 0, + "fields" : [ { + "id" : 1, + "name" : "Region", + "required" : false, + "type" : "string" + }, { + "id" : 2, + "name" : "Item_Type", + "required" : false, + "type" : "string" + }, { + "id" : 3, + "name" : "Units_Sold", + "required" : false, + "type" : "int" + }, { + "id" : 4, + "name" : "Unit_Price", + "required" : false, + "type" : "double" + }, { + "id" : 5, + "name" : "Order_Date", + "required" : false, + "type" : "timestamptz" + } ] + } ], + "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" : { + "owner" : "root", + "created-at" : "2024-05-20T19:49:04.669537174Z", + "write.format.default" : "parquet", + "write.parquet.compression-codec" : "zstd" + }, + "current-snapshot-id" : 8325605756612719366, + "refs" : { + "main" : { + "snapshot-id" : 8325605756612719366, + "type" : "branch" + } + }, + "snapshots" : [ { + "sequence-number" : 1, + "snapshot-id" : 2001582482032951248, + "timestamp-ms" : 1716234545155, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1716234462547", + "added-data-files" : "1", + "added-records" : "18266", + "added-files-size" : "176970", + "changed-partition-count" : "1", + "total-records" : "18266", + "total-files-size" : "176970", + "total-data-files" : "1", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro", + "schema-id" : 0 + }, { + "sequence-number" : 2, + "snapshot-id" : 8325605756612719366, + "parent-snapshot-id" : 2001582482032951248, + "timestamp-ms" : 1716234545529, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1716234462547", + "added-data-files" : "1", + "added-records" : "36107", + "added-files-size" : "331675", + "changed-partition-count" : "1", + "total-records" : "54373", + "total-files-size" : "508645", + "total-data-files" : "2", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro", + "schema-id" : 0 + } ], + "statistics" : [ ], + "partition-statistics" : [ ], + "snapshot-log" : [ { + "timestamp-ms" : 1716234545155, + "snapshot-id" : 2001582482032951248 + }, { + "timestamp-ms" : 1716234545529, + "snapshot-id" : 8325605756612719366 + } ], + "metadata-log" : [ { + "timestamp-ms" : 1716234545155, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json" + } ] +} \ No newline at end of file diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json new file mode 100644 index 00000000000..1be00194c4e --- /dev/null +++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json @@ -0,0 +1,145 @@ +{ + "format-version" : 2, + "table-uuid" : "49ca5b8c-6402-40f9-96b0-4c09f9f1b512", + "location" : "s3://warehouse/sales/sales_multi", + "last-sequence-number" : 3, + "last-updated-ms" : 1716234545865, + "last-column-id" : 5, + "current-schema-id" : 0, + "schemas" : [ { + "type" : "struct", + "schema-id" : 0, + "fields" : [ { + "id" : 1, + "name" : "Region", + "required" : false, + "type" : "string" + }, { + "id" : 2, + "name" : "Item_Type", + "required" : false, + "type" : "string" + }, { + "id" : 3, + "name" : "Units_Sold", + "required" : false, + "type" : "int" + }, { + "id" : 4, + "name" : "Unit_Price", + "required" : false, + "type" : "double" + }, { + "id" : 5, + "name" : "Order_Date", + "required" : false, + "type" : "timestamptz" + } ] + } ], + "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" : { + "owner" : "root", + "created-at" : "2024-05-20T19:49:04.669537174Z", + "write.format.default" : "parquet", + "write.parquet.compression-codec" : "zstd" + }, + "current-snapshot-id" : 3247344357341484163, + "refs" : { + "main" : { + "snapshot-id" : 3247344357341484163, + "type" : "branch" + } + }, + "snapshots" : [ { + "sequence-number" : 1, + "snapshot-id" : 2001582482032951248, + "timestamp-ms" : 1716234545155, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1716234462547", + "added-data-files" : "1", + "added-records" : "18266", + "added-files-size" : "176970", + "changed-partition-count" : "1", + "total-records" : "18266", + "total-files-size" : "176970", + "total-data-files" : "1", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro", + "schema-id" : 0 + }, { + "sequence-number" : 2, + "snapshot-id" : 8325605756612719366, + "parent-snapshot-id" : 2001582482032951248, + "timestamp-ms" : 1716234545529, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1716234462547", + "added-data-files" : "1", + "added-records" : "36107", + "added-files-size" : "331675", + "changed-partition-count" : "1", + "total-records" : "54373", + "total-files-size" : "508645", + "total-data-files" : "2", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro", + "schema-id" : 0 + }, { + "sequence-number" : 3, + "snapshot-id" : 3247344357341484163, + "parent-snapshot-id" : 8325605756612719366, + "timestamp-ms" : 1716234545865, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1716234462547", + "added-data-files" : "1", + "added-records" : "18230", + "added-files-size" : "176482", + "changed-partition-count" : "1", + "total-records" : "72603", + "total-files-size" : "685127", + "total-data-files" : "3", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro", + "schema-id" : 0 + } ], + "statistics" : [ ], + "partition-statistics" : [ ], + "snapshot-log" : [ { + "timestamp-ms" : 1716234545155, + "snapshot-id" : 2001582482032951248 + }, { + "timestamp-ms" : 1716234545529, + "snapshot-id" : 8325605756612719366 + }, { + "timestamp-ms" : 1716234545865, + "snapshot-id" : 3247344357341484163 + } ], + "metadata-log" : [ { + "timestamp-ms" : 1716234545155, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json" + }, { + "timestamp-ms" : 1716234545529, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json" + } ] +} \ No newline at end of file diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json new file mode 100644 index 00000000000..16cca0f3f8d --- /dev/null +++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json @@ -0,0 +1,172 @@ +{ + "format-version" : 2, + "table-uuid" : "49ca5b8c-6402-40f9-96b0-4c09f9f1b512", + "location" : "s3://warehouse/sales/sales_multi", + "last-sequence-number" : 4, + "last-updated-ms" : 1716234546189, + "last-column-id" : 5, + "current-schema-id" : 0, + "schemas" : [ { + "type" : "struct", + "schema-id" : 0, + "fields" : [ { + "id" : 1, + "name" : "Region", + "required" : false, + "type" : "string" + }, { + "id" : 2, + "name" : "Item_Type", + "required" : false, + "type" : "string" + }, { + "id" : 3, + "name" : "Units_Sold", + "required" : false, + "type" : "int" + }, { + "id" : 4, + "name" : "Unit_Price", + "required" : false, + "type" : "double" + }, { + "id" : 5, + "name" : "Order_Date", + "required" : false, + "type" : "timestamptz" + } ] + } ], + "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" : { + "owner" : "root", + "created-at" : "2024-05-20T19:49:04.669537174Z", + "write.format.default" : "parquet", + "write.parquet.compression-codec" : "zstd" + }, + "current-snapshot-id" : 1792185872197984875, + "refs" : { + "main" : { + "snapshot-id" : 1792185872197984875, + "type" : "branch" + } + }, + "snapshots" : [ { + "sequence-number" : 1, + "snapshot-id" : 2001582482032951248, + "timestamp-ms" : 1716234545155, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1716234462547", + "added-data-files" : "1", + "added-records" : "18266", + "added-files-size" : "176970", + "changed-partition-count" : "1", + "total-records" : "18266", + "total-files-size" : "176970", + "total-data-files" : "1", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro", + "schema-id" : 0 + }, { + "sequence-number" : 2, + "snapshot-id" : 8325605756612719366, + "parent-snapshot-id" : 2001582482032951248, + "timestamp-ms" : 1716234545529, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1716234462547", + "added-data-files" : "1", + "added-records" : "36107", + "added-files-size" : "331675", + "changed-partition-count" : "1", + "total-records" : "54373", + "total-files-size" : "508645", + "total-data-files" : "2", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro", + "schema-id" : 0 + }, { + "sequence-number" : 3, + "snapshot-id" : 3247344357341484163, + "parent-snapshot-id" : 8325605756612719366, + "timestamp-ms" : 1716234545865, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1716234462547", + "added-data-files" : "1", + "added-records" : "18230", + "added-files-size" : "176482", + "changed-partition-count" : "1", + "total-records" : "72603", + "total-files-size" : "685127", + "total-data-files" : "3", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro", + "schema-id" : 0 + }, { + "sequence-number" : 4, + "snapshot-id" : 1792185872197984875, + "parent-snapshot-id" : 3247344357341484163, + "timestamp-ms" : 1716234546189, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1716234462547", + "added-data-files" : "1", + "added-records" : "27397", + "added-files-size" : "256855", + "changed-partition-count" : "1", + "total-records" : "100000", + "total-files-size" : "941982", + "total-data-files" : "4", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-1792185872197984875-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro", + "schema-id" : 0 + } ], + "statistics" : [ ], + "partition-statistics" : [ ], + "snapshot-log" : [ { + "timestamp-ms" : 1716234545155, + "snapshot-id" : 2001582482032951248 + }, { + "timestamp-ms" : 1716234545529, + "snapshot-id" : 8325605756612719366 + }, { + "timestamp-ms" : 1716234545865, + "snapshot-id" : 3247344357341484163 + }, { + "timestamp-ms" : 1716234546189, + "snapshot-id" : 1792185872197984875 + } ], + "metadata-log" : [ { + "timestamp-ms" : 1716234545155, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json" + }, { + "timestamp-ms" : 1716234545529, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json" + }, { + "timestamp-ms" : 1716234545865, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json" + } ] +} \ No newline at end of file diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/02f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/02f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro new file mode 100644 index 0000000000000000000000000000000000000000..d871fbb9fd098d297903c3e1dcd7f46abe821c6e GIT binary patch literal 6976 zcmb_gYlsz99N)|}iIBzTu5NW0bV0nEcV^aoT#&?6ShaO=6_%D|I(P2e-D&5}TxaI4 zuCkOG#uiv8$|_tockK@-d(sX?94g; z$M66A&5q;?4>dgI>BB|_U#UMJ`D0l_N=TmX=5$|*Nx_>|GbJVBF)3{tR?3qS0|zAX zM2<-|&XDsvjUm&q;k;|?$(gPJFVooajF=$Mw1=eeSa4HeH@kf!qwOMEtG=wV%e!sU z_q0C8N=4i@ZJ+6FE4Zrlx~3kbIOXJ$mceeXW4Cv>DZ|w^VL$4&Z)Ob7$C<3ZUm71A zUy4Uv=cxeDW9t~yuAw_Fu|E%G2HR#D?rDbYyJPTN06l*Mg!(xTUge`jJj|a9B1K>h z&)PWa4LiOD0Shk-NOsP$;9tvum<-0Gl##|c%a;=5xhP2%B|&HnTQ@X2mq{9K4boIW zTH3S>&6}~vHbG~N3aswfKETfIO98SeM~57LCV1%~Cyo4J0})LL z6kku&#*n-|>v*OQRe(~4W%z~{L&|#a4_l@`7CsBL(lE)YXgviai~V7)9jcHN4aA2U zu-(xyRj_+!H`;6Z!)Qw%qWMHTrrrc{<8c8MDmU%887%l!BzX(Cf`T)Mh7zr`|K`D`PLm3(y}N4;}0cg&=|{@^e|shy@;El*FEqLI5wp zo~c7#*3gT*R|M~a>e6&4Cx(^geWw5=zk49-IT3(=Nh=N1v$Mig+J>C8$;P1sR?5%o6`BxRjoO)i_A zU64pOQ)}sFir6TMim?nxYHjU|`dbZ1VzxsU(K3dQ$(B%(Nm5`On>U+n-5tx)y$SB2 z@E%H9PBJnss@RRZEUa;wlTLevUw!t56A~7}4D6Nptxx5{u#brKF^~+Xu~2s9cBG1{ zO}JaaW~Bo{@dIhOJN4kzb(0A;k*m7$?6#pE>s>-jWh8q|9CcG2>SYc{AM`!pofa zmJ+9`1aSIGSwW3bIC&Qt7dJq3z7$j%J>eUI%8lamtGuw;r(+?qa-4^gLX#(A73?UY zPf^5-Yv5l?s0K#Ks?)KhN{Yty@i4LMhw3V?Eh0>_K+m{MJcEmfS$VU<{u%J!XrjiI z@PrDoyojPzqpBih<61gd{TGonF2A6Z8N`GR#Uz+T!8njR=s)m*A!yBX)aGA-j$ra8 zLL`E>Tm`8$AY1`JRZ=VNLl^Ub%H<@0E2z!~5JXn{GCn9!ShM3pH0NPX$-I)w(VJ;{ zra;wJ@(c=Wrk<6M5$1*pX_22VWq8Lly-*>MDtMtn=w|N416cy|OW_|;iwMCqeNx&h z`lJAMp-)NE#`H5PebCC4Jg{=a{hgc7%f4t;=aQ=`8KrW`s6xrqZoar#GxJqGgdiWp zCAxz75;GuZF_GIeSHovd@#0)T59^MW>5=b@g$pW-jr;>>G5Jp^;R!CVAfjb9Yi=IT z!J;La3o9SQverQ&a_b8BYc8+AxacjL+1bWhS@A;61s52apCLi43KwfwmEqUSb^*Ex z&m9dC1t@9uB6R{#6aSyhS0n{llD z!|#4ryCb&so6py__I$s#=j-m)BX9h1`Q5Hnr|z$wST%8f9(rr!?U6H+lV>dZ*gGR< z)AN5mzr6qUbNwynrp`6BG_{;MJ+Sip>83l&Zy(yVVrt3CrAtnpJNbKk=e+tOSB`AI zdF^86&3TKKU4J@#^6_7-HGjK%+CK0;zSQzn-QtJ0UaXT>G4 zsj+c#va@l6)O_^QQxma`Jrmt09<}e?TWvL?7qjz+Kl|&+`&Tx8ad6w!t9QS>d(-Lv z<_wjK8mFH#r2w(!rz&DUSO*uUx8&g*^K79M};iPWuz-bc*(UvBKXG3Ull|81YT Pclnu)w^U!KuC-_=G?LA@|%FYk@nh|ln@=M*)M7kK|oo1Wt{NrBJn$jG^TW^gZ$ zpNIl)ss((0D;m}<6V5wmS5bEmyv(bHiv&iXZVvO~f`3yAZ}xboplrih>%J_9m$#d` z=PCo1k&C&ln_j55z3i&8#nH7m#W|~(HBfjv72e+L?K+h2mRpqhFf?kULhoH2OLfL=NRLcO93uS(Iv9+uAek-{(s zXHC_1M=Vc)fCU!@d9!F3@ULM(OojwLhw^IC@c0aV&PozlNf26OYDh7Qg)DL!kR~(I z^16W(cgiB$8Id{FQxz;XG_5Acjs#~afz>S21K7j+ypL?o(jdpz`Y&D1%5&Zb;;^O+ zm+Z!B1y1a?Em!xT3b-6HkcV7>le)n_)zH1M;8~!Rgh`ZH>j@yX>WxtCP=&Z?AU?!^ zUQ1Jng6&&-xZS!p!fhDfNIqeYi8sF7WRgLJ$jw_$L1p}kNZtUhaQ+z%=S_ygxhim$ zqv1cCtvV3<@+1?JY;wQ}k9Z$iUmJTdS%&`5c;MicKnRC1<-DS8Ai?)g<+7@aavZ>m zu@BX8u8p*a_Y&j1UtNl371^+oybrYYlXS*W^$)w8?kmB_MMPI-u)YT=x~XKxAmX9# zeqjX)wppHPkaXc(IF7{!$JwB<$)H(Av0y6rIF%ZdZHTPuI4bTCI`mNJ%Hi{9OW7%k zoyw>vrUP&M7GCNQI}w&qqm{r-2F+2`DAMW*ykwjZ)6nQ}Um4al)gWy>!xvN=`d@%X zpK4>QeGfT3$9Wd~FhT?zTtB+I6D+FTl#=PFO@4r<2dXZBN=h~YRaew3ifbw37^tM= z6rdvcEI=k{MllCHg2RoQQ_URLulC1d#~O)mDI}=V2%scl0Lw}|oJR4vCb1pmfw?A0 z4FO1yBElgD{jN|u(?qtDnyNYNV999!OYUkQ+vJKOh|4y)t3j~vsK@mRdBqXb!uIG_ zaBQ;4Z1XUz-LNxd0drU(gN8GOo+fqRJnL)#I;o>}p;qb4QfEWh5q~-2Z!Absim|3> z+tUjY?q+f$-ArN|MWh(3fFw88&WOMDfW&4ybP=V1JQZ&V6`92Q#)b1{y{S24Hrbot z9v9r>vWAt7jf>27BiDvCPOnMQr~1e*(BXh$1$2y(U!c#F`(kf#iI-Dmw~Uz4tL40z> zI%@hP0d|2;S>060XH@c_jVgIy<*@rZHJ_G!WL2k<>na(kcFD+0$uw?0yIF_k%XA1q zKCnx44fFZX0Kdh=Zqrl^ojuXTxrUywI~ux+zcZFEs4zB451_@AKBWXFsKBy_R@tnn zc{B$jOEeW$J&0AUgG8j(mG9S7UYT+6TXtw?J8fmrg_;U3Gqf~A{8*JQ*03tWuQl8S z$R<2{aA;v^>za#~xu)0uxO?=^{$mrreDuRRw-R^I%=cz%*$dY{AH99$oBRJRSdvH_ zOtvf?ym{-$ik9Q~-;T7L{C)9?8y~i;y>$1diFXh6-hb<>L+i$8HD69$O`T858{0RZ zyVfW5_fMpHq)pfQ+Rye*q&CXEslJZn*<`AHB6WGr>5glwHVRJ*eQm!!V$Yv9zqsJ$ zjxWD3KYs49#aAEAAAWxH+{%a3Y4gmkLRz@kv}o3*Pn*Q0&2E#pxcRmHOI}R;m`Eo+ zOC*jSO($0Ktp~q8c0gFuf1u~>dFH))D~;ASSHEJ<9l7|=6aQUU^XR3m=>$*Ues+*&uOlQbGEln z80{Zjf`=VzwGZgn+7N0-Q!IzrUw|@)O+5qmWX<%P5qQplUN{0my}S#r3eh4S7S8#R zA}|MMO>Db^mM25Nf(yF@GjAC1uVFz<`eTBsWpLi`gcNzsOG@&RAhepPXtJ5lr8TDp zX%Z(bqZ^v+PFrLXCsM^8mWkZZw2B}*BAl5FtYVoSz#iTgd}LKifgIoHzjTq6LEfN- zh^7=utRre;D7ntIT-}2zK&obFp614oxDNcohVG37&jPJ9Oi78io&sWHZ;)w+DkMb% z@u3EES_-BLwr^`kdv$LRZRtfcpNPlQ8((fB!J$IsW-KR%IloFIZvj`3e+H4fNg5Wy6AdE3xpzK0m4v8$;F z;3e3H>X2(|N{RO(=e=KDvSQ`=u+qE_w06;SCQoJ_U`-uNxN&>^xKkx`+Qz)cR#VQl1Cbp>8>P9$5Q(UHC~pexv*Z9OIAunqk$ zKw~Sm3D&-cNLP_(!H*zB!6EgdyIHWPa#KpQmfGY8cxIq#1E|DAEl{Te`S;#$s{ zylu}cNTi!dwRAI)ZxkiPSOz4iwsuDStp+4M+o6ldInBdlOPG{Nf^S?nZ#J8XGh)-d z3GSib9!eWlIx;R2-;G=w);QV9WL(XwK6`@+2@7Ek_R7N6C$VAJOT>B@NCwncAiHuq zlK9mo*ezkRpMpjHMq6x9Gc#y&u&^pt4wfw4g?UVe7HZ!_v_#m(r?G1}fEFc5<#-1% zA`DP3oP{M52o>mvRMF(;7#pHw0xY7mO0iT;sR!!};AD;dDjOfx92XYyI#CGs#W>j) zAr5tkMxcJs+Xf1Z?AEGqeX5T90v!%0RzSxX`33sSxG(XRlz1^^c8iD^yIRbf>D~}r zX2f?gajHrHqra3D)F_3Kcb;)E19;~rgG!^veM69#QH*|-7dG>BEI>AyVBw_D6o^;_ zJ4)zN6ft8O*w+%OA*5v0>DW>wMPvF{n8NIb>ME}-B1|(t&zMatgG-26d9#B4>GR)c zqQ;c4gz~ezgrZfWsv>1$T3W;UFClA8eo-lh5ED8S6JQqk;~?2W|A7w-eru+qw(tsc z_>(skAmP7dDoCvX;qn2hl3Fnz+Cm?wTt)(zg6ez#L1eWrkC`ge6q~V+A#js-FZt_Y*=|So}GW^-z#;e<|yf*ufH7n`{X~*t=I&6 zQX`Hve|7uLn(kP~_czzXyM9^IHPIe_=kSBeAGa+(_HgB)<%b^5L#MMBvguP}$B$%A zesV%pht6fk2KSuF>T3GK6X{b%=Gc+rMjGSu+0zTQ<<7r;ct_uk<2$b{{k>sP!=KOm zm8t*v{g2Lc?(ST0^o=E-eZFkx^&1b~{UCem#TVwCZs-}Cd+^r-M`s_re4yd^iScXW zH^#4ye>*;Y{rc5ePyb=Ze&|#eIvt%C`t0w1?8f)@%z60s#`V&<|9-l&xTAh|-I#Fc zzWEKFd-1jTtGC{MvTNeySjR-K-MQ*kdg0@*Z0(M>ch>DY{NBd(%g!!7+k5ZT1w9v) U-@bVE{@vM^Uiws-H3^~r0UDGY{{R30 literal 0 HcmV?d00001 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/a61cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/a61cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro new file mode 100644 index 0000000000000000000000000000000000000000..462894acbdef4351207c59d84392e303f1c0d9eb GIT binary patch literal 6978 zcmb_gTWl0n7={W+kf2dOG<`TsXcgJQ&J2{cNh68Sh{4iO8iCSG&(5B0r#m~d%*;Xx zX$WeJJZPE*YlWmKL@q`j026O-Dj`VV1)?S1*_m^8yE|;RO*TzB zbIyPH{_8he69cm!S?KEhB!fSjdqDCAvqWlS2sErdX(;C3RKlB^J(AJ3Q>_(WR>R9( zmf^Wtr)?%9ZX1>t>RnrKRqJpJJxX!X&LvC|-d-2p-s&WYqiw`q)NRkm5ZA+*thZkp z>KR&!_d51!AJCC?A=D1hZHL;QhcbgLBL(*~VtLLWJQqOE9|56W&V^U`Xi*RI=ln=f zn1izx&bs}!r$NAi3*C~HGfnu{v>_%vF)2w>IA?lN3wt$Di|Gy2wr;ub&{Q zsRhMbsoEHlTeG%ncu)l>NlfAqH-?l}@DH1YHyAt%w6ZYCs%Sj}B#XU%t{tk777fIQ z8L-9HF;mdBZ8O?uc>SoY6R~`v9y4!zx$(Gw3X_|%oeUQIDw5m=t|0#mqIuJTXs*VP z?dbFm%3=p%UzlWaTucr);W6(+>&s&=#|zN!844Wi2!tStDe`hzlf--vF-l;UBoV+% zu@BWDH%s&)?-jv&zq&Nt&WT}Vc^_zPXX#9%>K}HI;VYrY6^Wjl!Nz`~8J3n9gouZ} z`-K%K*lv5+Wa)xjI8Ma}$HkzD$)MY2E@Nr*IF}lfZHO#(985ce0X-DDa`?QsrKDwf zy|!1Dn*wkA7M|}AX+mX`X(ez|KyxoPbG*6&F9j#$CTMiDuk;x@Hd$M5kuo?7{Vzb{ zMVzHr`yL`Ai98#A6d?uI}OKk$M)aEL(O`TK(QQ4+8R|ys!^|W3guMk5mY>$2g zM<$ypHV@O@2Rl;&FozW~XgJg8X-Xr_v-MR#r!oq zW@i>8+RfBzx|t$2ilSmH0g_r>J7fM<0+N{R&_%Qi@i5&ICS{W38yC)-O_uHqX4&2Z z_fT*TB}_XJ85dRTMs60?IL%I_T;f%py}^Wpg)jqqWq#{Z`7rFHVm%Bb18OXgUA`Tu z;%XD@may4R!6JX7Ei|Z+8?-4{SamxCOP1lnJZ3`+ZQe+=MA#;#v1>bk79&Zecn38i z3{WSWg(VXR73hdmvDVKqK1AsRSj=dZZYPPp2&^-KlhyjGY{OpSxUi7dC{nmDHqd<$ z;!qQ>1?mUAtuN2WF0TsLr^?7L(BXh$1$2y)U!c#N`%-Udi5F64w}6=OtA)Io?G3?Y zPJAa5r-}q{`pZ~Bl~Oo)7a12fKy-dGs4RNIHw2X%#pzdRVKYz10%YYl4<~~rPsGaD zQAD4jh#A+wzm_l!AtkF!M_YvyjqBrK3bP-otF*R=FwFox<2LaOE+S^7%?kRb&wrzd z8dt&-%Fps5idKoLvXqT$X$tGVh^%q>1*IH9OzBWef?4E`19?6B2R<xypl`N zn<;vxK-HG>3<_+zo=qYnp&KfsMSi}N;2l%+LWM*s2Ymt0ZFDCJ8=6-uUh^To|NG+*UI z2=YN(qRW^sg$DR7CUTqRYWVCaUYyJ53A>|dxb!`^JKBb?-dythy%ja%bM(ZnKM&sf@#_2Ey?IQH z$7^w{@uO?suGt!U;p(L|4eejAX}_|$;kDu0pPp>4d*|-T!*z%6&O;OF57GnU1IZ&} z=JE9S!1#%D0-F=Z#*U7>o9-VO8B30ijvX0EoJvnTv@LVyrQsdhcO2dI#p5@YEnfEf zljq6m>qkC3v1Rv`J7YUX&z<|e`|`!xhu*ri;jw2Q=veslS2Y*R4OeDg*u7!lqp4op ztKaCok?8GB-0YooINP+eum1e}=DPZ>XFmDqm(yqW#ApBa`SOV;t{?pM)`GeRS0Czl z+B|8Vx@u2pm1uK#V`Q>hh8>waF@bZQAX_xh_RPn{lJQ9rx&(#7Qyi>@uW_TKF? YbI)|G{P&+P-9HZPI{uA3YZ5~L11%RLd;kCd literal 0 HcmV?d00001 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-1792185872197984875-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-1792185872197984875-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro new file mode 100644 index 0000000000000000000000000000000000000000..c6e3e85459ba2cbb6885352e7aaef3756e017fef GIT binary patch literal 4399 zcmbVPZERCj7*;?|1|0C zxzF>S=ly!RE!ul?`3l&lDm9vrKuNf-YJk)GQ;2KeR3s>BjPr0ZOq$=5P!{7z(<5_UlMYXi43!m1M3Z4kODMW4A?}N1PMBHI6HvGB zFb|8@m&IaMiWU(RBVVRkO#KLd$&i<*ur-D;3|Q8#8z&I8PfZ zFzyDcX6B1GTWD2DXT|E$*_OaOmP-MiEqbCrK~SPVUR*X=jsOK=9Rc#9viTyNZ;l7M zOS?JQkR%R|olMt4RfYZdXWZ;bJnP8wxx-NnctcK2yJn&}2cRfxxJgZ52REuH5`r>* zU=PJk)xgs#4xrp`kFSK;l(=~$36MBJAC!!_L#LjpKuTsoyYIv~oI4AVLOBO#48@@# z>0E;ulY3Bvm=Gk~{%RzOJBkicM^(s|aYY3XU)$eYQn)$k%)P+R3p-)ej4FYvDt z1%!7*PiD=Jts#|+x3eLbd(Jih)@8h|?Vef2*+w9<=2g5GL=~@(1`LHriI=eU&HAxf z_-q!k@bmDvhE-kzKq+bo1!*j1_2(gBw|8zUKnVCCA)qFp2-s3g0VN9%GAHNM4&)(q z2^KEF9#Gag!kXI{6vJGkQ6QvVA0f?U5sIM<7hxGFpMMUY+brbaV+$qQJWz0&H?lNK z#wSaZ^(T~qCdt{KB{MktG$#R1JIPLnYdz=D$5nh;)`A4j(_|BUN`@!;Fm`}4(DXX; zluY2rlMCNV#{ii#rqQ6f#G#~-aY^Q}giRzqHfOhx2aE?HlbiV`W>3+2Qn}O-*!F|6 zyJodx4GKAfTy}&i8p=Uy84=3rx-)Skg=lja63U8~RpuY6+k50C6=l4e%|&ee9WBO$ zP%^ro+4Y|Im5i_&29VFLt~*GduIk`4by&Yu0-O-`ef5+}P za%r&f_`vQb-kdnye&CwacA)*t%6(nqEl*wn{MlCY`p_$1F6?UCF?{g-vR|Ki$G2zr z@}r}ZQ|aE{ggXPbEL-~ck=Df{>$Y$1PBj^S+;jKl7uqjfUGwm<6Mg?&|NCdn_^5L7 zlisW9W%+@NhtD)VSk^IBb@1yYBZn3p4qtEjxct7+hc;a}`)AZ>UR`aJO?~jpU3;?x F`VSgC?eqWu literal 0 HcmV?d00001 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro new file mode 100644 index 0000000000000000000000000000000000000000..28d439215a04848e617058919ca10091fdd5fbd9 GIT binary patch literal 4224 zcmbVPO^X~w7*6&e3bPR=vP1~6Y$OMnk?z@@?Cd}eVie=3^oEHNc7fV-9j)8e^`k}^+atk& zitdF9_KlEX5AgsIbw41;4bc9W5t@ga0UHYN5J$}A@mO?7Yzvr0;BLS~)F%9hu3tkj zB@SXC8bvOfjJKu0+F`&*@_v>D73{V2M~cEeVgrKm)OL}6)xiumka5)^9w|j`Y$_C# zydRUmCRPyp`y`Ta7gAzor%-wdjRqtrkfIT$2s;kpH&2S1X31zPT)qqWS%_~Z96^(y z{BTYSl~8XwL3Kr;NzaukySg zC}cE7`7tF&`i25Y?zogerzZn46ZQ4%Y@UEhRJC4W^7(p5r?jU9vFB;9cCx9(QH00vGYq>A zyG}fxT#iv&@8msQ*ok}(U^k#Jq#QWM-48>L-~b+^OYue&!?x-p9E{WTHQ+-D!{d2K z#EA~E7f&voEu{k`Spr?Wla_Pdo=1v#eP} z0iP}ckB?dNkVF)OY^8N6-@eUs>8}_UU7LVL9Aq;8-3y|VL7iip>s)F8lFR4?biyk3bjP2;`#+P(-x&60*fnZrMZA* zkbD~xd_}WR!KXG#**r*aSvMh?OXCX>RsD&ikVVS%b7_WLPp1+Xa+1=56y&)gKZWKi zSql-oA&V{YvoyTOk75TTgDh{Jo}~%$^i0Fo;x`+vpWKy4seq(8}YU5F`77~}&3p_^}^bCUmufGHh$9;)n z8=mH_?aa%6yjfrW{o^lcKm7V3diMKZ<(IR)A8&fz*WXxQ-TeH^&#YVbA6WNKPj1y- zd_=eDTdgP5+^(DT?fR4RKi5voz1@fL>OIqByZ0XMvXjQ@`M7BwAMfq$G*3RQ9W%Q6 ncm1bN{_I81UpPHo`{%cR@BH=P+eeRgYPH_um1izLRSMug$_SL1 literal 0 HcmV?d00001 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro new file mode 100644 index 0000000000000000000000000000000000000000..8774b62b1e731a45959b78d041f4d8503027c302 GIT binary patch literal 4353 zcmbVPTWB0r7;X|!u@YLW@xc&J25qfwWM}VLY-lm5Bm^zl1S%!2Co^ZWqq8&Hnb~aD zG*~Rb3zllYdJ94yN<@lQr6Eug>Vpp=RKa@5gH+Q{DaA;Oih_z~Zs(lY-PwyT3p;z} z`~LIazHc9!c&eqn15Rn`gSwDKIe4UNnlnmy#3eZm392S?0ZxHA^Lq{|QU>WpIwQ#l zKh8)<7IiLpV7lT&9vWGE!U56^O;RTDcTp8E&af&#LsAtmHZll`l92^_Q@~;;s6CR% zC8NRMOrR25M#?0ME~^aosA@nNm;nM^dIkYW0TU$y>2A0VFwX$zp=L;CJZU|TgfwO* zflG=(qE6u-!C*fq=tu;H3W}O!l*b!lUVi*MW_QPipadVRa84m ze3aqJNjN_Z@eQj*+$6v~?9$>&us1EEiZN7HC}~}Rc|EHdo`QHU=A1CAVq~FV-=PLe zFkHc6c8Z=B)Pi57_LzAAxR6>2xW||r1tSb3Iwa`^?sRKFYN8G}Cu;z+$+ufUtdHAo z^-QgJD-*4$#fn%xdA3tv9;=oCo^5)Pfx4h11HQOyvD^gIg>@6)i^|rEOuo4u>?u9K zDFs>P@Y->v7HS$S;h*ua=kTs0&&!vi77Zrcp7!iSa}PjC(eaR)z%K5Ds>%o|_<_9? zN3{apRv7@5lD)pN=1}6{k!3*Q1XEBhlrNoLrUNON1?{~P=W>qMBZW#8P7!5bK{mLg z8IyNWB$yCnJpNjG0#6iOq;6I@dlwV_eeUW7)?2`O3uIIcKzwa$Hq8Y26SD$Q$mjnD zvGrE-#QeLZ2+vx1U>5Rh;gM6aAaP?x*%y-K9U#;krr=4_RapWS@=p8BKST$ zo@M2004PO>P>{i5q~C{xRqyInfDrI;LO@MG1F+4Q0!kJoWNyu=9q=LbBrMg0J)o>H z!dl%JG{W4VQ6QwjFd?mG5gMUPHDDPiU!;uBYZiR?*hZ-|4-}l%jfiH`_(Vime?lo} zkzD<3n!(l6ssucpB)cFUdG^VVr}-+Z1qq&~#U}Y_8lL2bu>+KWme);B(*$mMs^JIe z8X$YdG#WH787S*yU6Oq)V-tywm1zrkz<3a{xtV`r_7?3Wol6~oZ9gcxXIHz{pq?|R zW=Cj76IEy}BSKj{cP5Uc9&HsvLRqm`IsZ_--ouwvQm{6gLTvpVZN!97GOABrztJ?A zzwz)`3yN#j3wsVb=o$tCzkUyHIMbJ~wjmJ?MPtEeJQ|Dfp*Y_gj>R^u3~d>G&-mc# z>}`iPFB~0Sy6Jq|YhNx8we8uq`=`%8TF)t&pD)On_m9u*-S|6j>*KAf#}+%fXZP=a zVXXh!8y9}tIdJC29V6doU)}N5z*hO|`9Ch7o<7z$cVm9xsu+5DPChjM)}^lRK0Y{o z|Fw(S`iWC#SJvKbz3*bn@QW?;p{=iUK5#j5``Era+xp&q^vc;S|NOg_`+ez?OX=gG z?!FV_p%42u1FIlQ4PzW(cS`?(+1 zpMUA0(eCZS$y;_lcm3NVJ3iYzD7<`cY4p!$`7gRw7SAnwv$i_*vxr3`9C*j B+3)}W literal 0 HcmV?d00001 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro new file mode 100644 index 0000000000000000000000000000000000000000..b5659a5c9b9c1352e4e14c33433464d2c3786a47 GIT binary patch literal 4302 zcmbVPTWB0r7>=Qh8Zc0!q1e(M2GO=AWoELsECk!6sRQT9xoh4M};)L7MI8(zs;7D*axCmQhA7E|v`wk;i2N znYt_G7gt+O)F7^q6Csd`9m858-*sCfI7e&^V#Bt;@uwC*-M|%axBz^1hB{&BQa&xq zjqz4&6In|lx@im8$7~EuKm!D>@e%@t1xhOzxlyPtmO*tw@+z;D&`+ z)CKY*SU3bc7wG`opzavBIbKzO)ocqR|NSrvBG_#3qYLaZz!e0d)aHRatwRjQfikTl z6SYLHWCaQrz33rJLq*H0mXO2aj-{j!o&v!sP_G~>MhfcC1yI)!y}4i1AWKSG;HnxV zKRxjsvFoHsKy)~wC6y3w+C-HRsQgfht_5pu#m2FM#4i?}pjE*Yh=V(HV38+VSZJrX zMa}k-Ds_1sHbB)k?sL#SS}e-3LN2B2*SvJ5gbw6nK6Ly}Qp9(`_G$kFEJ`a26Z{08*s@AA0GvW) z6l|~r=}#aL)w{hFU<68*5pWaG18hI0fRm*eSyXdw2NFnQ2`ltr4>)Uvv9>n`y)gG^ z6d0*I$w=E-gkC5MJy-_LmuljRn}q~Eu~Ayh0|)1IW1`tNJ`+*VpKuCZq)0#eW{C8( zErG%(DJV!xo)hvDYrYn1!Gc$Ku~~llhG+Q^>;Pxr<&DzQH$jx1cK9-118mPwqrvl1 zhNjEbCELd)F_Gk0leVx2f(K!nTlgmyY|$y!x!e&1_Jgy>c6DS8x;cY(c7$g%*M`h1OCPk(fF`}aYBp#*Ob%Cl7y|+B-b)@}1v)2O~f4vCgjJ_0=b~eK>dbqhG)E z;GYBg?ZzvGq0y@sp84_FlNT?Xe`DkJt<*a=PJN{=zWv$orE8O3Lu-4E-*YK@W$v|i)tTq&^-re%ZG6zkz5eR#t@~#dHm|K|k7u5q|D*K6fy&+k pkL^DA)0rKk`>r?c?D*mH`I|St|NG+RulvruH~sC=e}}W6qOc?7^ZTF%WkqW9eZ|W zGc%j8X-p##85tpJ+LYX0(mteZB-9A0rWKV)BsUncRM3K;>Wh0*8lY6|Ls9&I_|KV{ zbIy);*Xy;VC{kw6`7hsp{Z0l_V{5-~r>kZ49GY%^O!SI*T}+Cu=M*$gjEVj)aU(4z z6EQJk=w{j#laD?o;xAH6v``M8Kd6rywgu-M{cyo>boeoYOjnO_0u5_aoQU~1WqPyE z({t(&)>`*vgv$m%~ZiE*e6|I74!aq$L z!ha|xru7Uen4Xx#-+4(AFA1_pw=`X~3b~YyVe-}2FHPa3WeihS-4%;$=R~HFhg2+= znpPELM}{+tfz=?A06V=e`pBkj4a#Yc|D!9|8Ntiy0@joi61%b5m>})W+pgh3!3k;I z)IHse3G#0651EEn48Mh1Nth&sx1Io!MoNaD85Km;SkhJ(L|ZMVKFXWiZpR(B82)t=#GG+D|m&xsmFY*k&r^J zo)#d`I18xbf}7X1NE~D?4t|}hnqA-%j8tH#wVxCg&M*J4D;T~K9K(p{#kFJ{(^bP# zQ$@%<8U??^Lj^;&hfI>Vf(ys7`0zNNK73kgwpqwoDn8Dnh9ys9j2s8yUS+`GfKf!h zN1NNYBz3AsB&jR(#-Ce3uNv2}j3D`=O$}~xXdXdkfz^5FCFg|HMZ#&?qlSjeA%BAA zP+pwC$wcaDKgwgQeGdgAEqFHk!3Ys>a3ve=I9OD@Jtn%MHu(Ww9jLkiDml>zRNYat z7_LFaaZt&L6+lJuIR=@enT0g;6akMA0aV=oE7vmxdZ1}X9Lj59n}l9LT8pc8^VtGD-eHc zf+X+ctSRL4s|ynDW=bR7Oy(O!q!=rJq%_vfh`;rK#AiEn5jCfK2yY3CGD-A}qw{93 zr8&hs+2G)w5Z)6~rk$!87lrReZXOm&)y`yG-K#%)!wCtiX%05rVEiK|*v6-^Yde4z zVTP3$P`F)^g zWRdQRal9`=9GVg>z;~gyjRxHCQkNu0e%mzxxF(rY}Q2=^(uO3cQg$b zzp$1s#4t9356~uqr@_nAkD>JV4R!{A0 zXRR!{z%#*Rh6Xdlk5&01534!+X`#CZ*}Ts*!T*;9YjgPU>>$g<%MhMOX<$k4a1Q}- z?SiA}WG;cfY~_cAf`5|%=l}iH&mNIKk@o-g>g=Zu2uJ<<<;%%FXQ%dtA|Jya#v@r8=?7r*I^>3Yd{PZhbpLz1(U;Xoo@7}sP_un6H z-m;}g4f(PtTs7`{uPhKbX6*?tx=J`p@oj(&g#qMWOpY D0AU9c literal 0 HcmV?d00001 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_partitioned/metadata/snap-5230665555732911945-1-4c305b6c-008f-407a-8e3b-ea1187f6421e.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_partitioned/metadata/snap-5230665555732911945-1-4c305b6c-008f-407a-8e3b-ea1187f6421e.avro new file mode 100644 index 0000000000000000000000000000000000000000..17af22ffd9f53d228dab8461c2eef00489774fe8 GIT binary patch literal 4251 zcmbVPPly{;9Nz9p=w=UDDHTe64$@OnW|B>^8|CBsr zu`KkaRD0-d1(D)S1TP|Zkb3fLP8Ltz6a>XnyL$Hb=FfX?GMW4_w@luA-|zkYeP7;a zzk6!+3_kGOi@wn%J$(249fc12L}@A>Fk2@BzOQSdl*P7* zvoE9Du7thkQfwm@Ao8sb2(ld19#Y~b;l{w41l-3SwOBkYzfX)7F!R7Ihw`X(_z_*b zf&!nIh`MOtS#&(!iUe!84kh9HaTa8-SEC<&U zXcCkhPH3SL@=eF6>Jn8LO3Qa}-|xCKRgm<>;uEYYw2NtUhbb&-HN(O>#cvsIkXNb8 zEH4NO=?zhGOv;hEE1mzjX#g&VdBw!|Nl7L)Pxn6wo*?Nek^rqqjwynV0iKiBO9v;Ha zFzh|pb^Q7Ga`bBIM$*%%oyhh8wj3XZlmREWZP&F4cHlv@6!*LUwp9mVXBe%oE*nZ1 z9@|DdPIQ3nV0`IpN*yT3Lg@6Jl!WuvEK=-D;511G2R2ojET;6L*pNc7Vf?*T8zxFD z(j==&w2L|ao5|`$;aezt3w2x%VYqfUn=C>3#IisH{r>+T4&PdwSUbFRSW;lxdaxQ| zQQ0`;7t@1Y55vY_S0UKwigZrBkJdH4vVF_Z(qGojSq1@(*pFrY`xkj9gE|KW)49k1gqKlM z>K==6)(FHh?}08@9>_-%pa^gA1!SM~PsE~SS-9Zm@T8Vit^p7f6;8niOOpN^5?Q?` zTLDg>H8_DV0dv4EVhRLVjguuc7j__rG?g%Y0ec`=>zsA6F_?#WPNTp{)hZ{QWD(|} z)aS4af^U6{FKrfb_~b^(ng;c&NLVSFy4vOf_NqDYB;F3gbV=|loeOj1;klsxC; zC)IozYr%upM6r2(7KZ2fk?eqA5amtMvoJxDo=NzsSOa{|Fry*z(!sXR*CpS_Hke2_ zHl{87f#gB>=9d16MO$=}cdl>*k^K5|gIUgCk{uBl-Izcd8WF*ox-)kqvuGz6 z62S^$75_s`dyiaF%K>d>LLB}boyUX-GNDgtzp*e`zVS#|3xUh(1)d`cdWyk-*I$N) zW4=VP4bL&xwsiWh#~(hpeDd9Uzw9>mF4P|`A1(j-<8|Bq@|)Hd*B^fVpmpQX*R4mN z9p6}a>G#vckFQm>D$kzn>Lf4cYdt52({s_i`f;qUH~r~m%B_QKm|&#sPsJ9Y0{c&y%j#n``l S^v=b`M}I6`tEx+x*!}|((4WKr literal 0 HcmV?d00001 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_single/data/00000-2-1b3d5352-57ce-4968-b83d-30b13e1e7151-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_single/data/00000-2-1b3d5352-57ce-4968-b83d-30b13e1e7151-0-00001.parquet new file mode 100644 index 00000000000..6394c55b6b2 --- /dev/null +++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_single/data/00000-2-1b3d5352-57ce-4968-b83d-30b13e1e7151-0-00001.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:bd1139eaf58ad06a1aaaf2d9d8505225d4dd7ac9e7a40f8f4b324ec364e76de5 +size 729342 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_single/metadata/00000-3a7beb71-ae4e-42dd-ab04-884f6fd52372.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_single/metadata/00000-3a7beb71-ae4e-42dd-ab04-884f6fd52372.metadata.json new file mode 100644 index 00000000000..0e56acb0f0d --- /dev/null +++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_single/metadata/00000-3a7beb71-ae4e-42dd-ab04-884f6fd52372.metadata.json @@ -0,0 +1,91 @@ +{ + "format-version" : 2, + "table-uuid" : "f1d2074d-58b0-4087-8edb-8d85f1472553", + "location" : "s3://warehouse/sales/sales_single", + "last-sequence-number" : 1, + "last-updated-ms" : 1716234544074, + "last-column-id" : 5, + "current-schema-id" : 0, + "schemas" : [ { + "type" : "struct", + "schema-id" : 0, + "fields" : [ { + "id" : 1, + "name" : "Region", + "required" : false, + "type" : "string" + }, { + "id" : 2, + "name" : "Item_Type", + "required" : false, + "type" : "string" + }, { + "id" : 3, + "name" : "Units_Sold", + "required" : false, + "type" : "int" + }, { + "id" : 4, + "name" : "Unit_Price", + "required" : false, + "type" : "double" + }, { + "id" : 5, + "name" : "Order_Date", + "required" : false, + "type" : "timestamptz" + } ] + } ], + "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" : { + "owner" : "root", + "created-at" : "2024-05-20T19:49:02.681248048Z", + "write.format.default" : "parquet", + "write.parquet.compression-codec" : "zstd" + }, + "current-snapshot-id" : 5481576066981634597, + "refs" : { + "main" : { + "snapshot-id" : 5481576066981634597, + "type" : "branch" + } + }, + "snapshots" : [ { + "sequence-number" : 1, + "snapshot-id" : 5481576066981634597, + "timestamp-ms" : 1716234544074, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1716234462547", + "added-data-files" : "1", + "added-records" : "100000", + "added-files-size" : "729342", + "changed-partition-count" : "1", + "total-records" : "100000", + "total-files-size" : "729342", + "total-data-files" : "1", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_single/metadata/snap-5481576066981634597-1-3395af41-65d8-41f0-8674-13624d6129c6.avro", + "schema-id" : 0 + } ], + "statistics" : [ ], + "partition-statistics" : [ ], + "snapshot-log" : [ { + "timestamp-ms" : 1716234544074, + "snapshot-id" : 5481576066981634597 + } ], + "metadata-log" : [ ] +} \ No newline at end of file diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_single/metadata/3395af41-65d8-41f0-8674-13624d6129c6-m0.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_single/metadata/3395af41-65d8-41f0-8674-13624d6129c6-m0.avro new file mode 100644 index 0000000000000000000000000000000000000000..3a70db60f6c201e993ad9e8019a795ae71a756d5 GIT binary patch literal 6982 zcmb_gZEVzJ9G4vijj$LHCOYx49}I?@UfaFgE-qVuEkhB;2D%JpX?wk`Tdyr`Z)A`e zV(=3&3iyE--6t_fm8kgZ*&nc)Lm*o66O?sBgq>@}t*Nm*oWp*Fr z@Do1CnQ|VV@7G3j%Y^fewy&T&8obQOhN~qRfx0=uO(p%CLU?nFr{$%cSZmdn#qjbj zQ}Vp3W}kHw_C#7+nucDNSkFZ?zX4rHP@5#ws(M= z8lGAsPdL_#KA^{1LZ}^0wH$1J5z4%5>N&V4X{P5)!gB`n;t>$)6QNz3USdu ze24*CEmbB8cJ*&T`*m*=^$Z}APuOGPjW0KqVo)J+bC#2r8NVWud%zXspFucpG6?5t z5?PLl{~%j-Aoit6#;4fifD<0^KD53v_I#=Y{o$#=!M;EU!k8kjU>jP}_fSTP>}pvA z@M7#kb;z|fHR8R%c<)!2q*?_wtR(LPt=mXCzZDU%TK z(09ME0tGuQPc}%pAQz5f@xgI6Xlyd5mQl!?50BH1SujMa?tMzl{1ZJJE5U2C&5T8nR8CQv`9@CbriI79RDuULmg#K`m^L zeg(%So5(f~!`cr!lLDB-3K=w<1@ttb73W!74bTa#l?$~@XBOIO!jAZhh`+HQ2`!8@ z1>0U+kZ?B>Yw2bJ+bAN%SOz4qwsuDRtp+4E+o6j{dCimYmM|xiIN!K%-gKL)Gij5( z3GSib9#RZTiH(cMb|cq@HBPc}Ial+l&)#4{!a|sby|TFViF6naV6mPIBm-(JkX^YQ ziR@|jW;`7n|_D z2yv)OH3Id6-ZoNXWLGwX>r-{)7wB+6u>v|q$uH1n%6+l7xWr2-vs*&U=+#o*O!kK0 zG9|uqiBnYqDE%d@phhW_yt9mp8o)X~7gQ2G<{N@YjiU6cys*WmV*#>!iiVRwQzT*) z?1<2(IATUM(61##LrBT0)6r8UMWgy?n8NIb>ME}-CQOS!&!|l_gCoSOyjem2^!aZ* zQKL#|Lit%9p=i~psz}+WmULME5wb?*my~h{F{VRF4rY-*4)`|m4}4(oTQeE8#aE!i zpS+m>3I8osfoly2mk&^t)QbAh9{ND!QW8KFRObT-!mE859|S0@#qlAY^Mp?Ew35rw zn|XRBK-E_A3<_+ap3Na6p&KHkMRC5A;T`kzLWD%B;DrdGTeuewWC@{P0{w^@Aq4aE zNocL;lK|KSJ}J5>lh3H+K@(N-z{+9wcWOQ@`^c(JC0A84LgkVXnUbm9e0H-A%@^qq zf_z|?=nCd@p#gr2iQT5D8ajKTi*p4%VRtli7k_6gT~J|c6dyo~DSk=`PEdg*5iPS> zQ}bvJMwVzQtb7p5S_g?ptt;KHsk{>7;F(kB;@0&DUgjR!F!j1N*)TJ{`n!`~zw+icnYvGxs!HbOKIiX0)*gKR+TQHm zz3HC@4ddh)X?pO3506SiAAK%;bmI8X^5_0MBHq2>_TL*=`RdAhE4sR7duLOZ(?8DK zn(3XLdAWVp><>4tZfv`JG2K0T3thgRzL~oD`}LW=*{+69e(O7f&Ye2jeE#rD7x?w5 znU>oZlGB~nj&9wv_5PhN&t6Smy7JY?^wOVC|N6zAwtrTvJ#?{aat+EQlzMchPI*6- zcr5XBVp*agkyy2AS)%vwrE_WVX!kEK@SpX6-_+Up&bx`*4fRW|o%nB)^v3#j`_U%{ z*A2b(?}`nB=QrB=G1FFhlpV_S+v?DcHAi{}#}15bQC40*ZZy{Yg-)Z7pL%cT(9qqw SXFEFYuRFDR?3u(Il>P^)2O69J literal 0 HcmV?d00001 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_single/metadata/snap-5481576066981634597-1-3395af41-65d8-41f0-8674-13624d6129c6.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_single/metadata/snap-5481576066981634597-1-3395af41-65d8-41f0-8674-13624d6129c6.avro new file mode 100644 index 0000000000000000000000000000000000000000..a7720ef0bd70d2af15a81796bd8d1e8b200849bf GIT binary patch literal 4227 zcmbVPO>Y}T7;bXnkVe9w5~QjOE5yl0yK(#xOFbY8Qd9>}oT92xWsP^o@mBj`c4lKt z6p3>o2UHFLMZIx>12@DkfCI^eUOBb=0-zEXB;MJNnOU#b9~`}&eV%vTkLP{%gWiLc z^=tSj@ZX~5fVlXx8^;&0%H1srIj@gyZpC{J-bgR>-KGjz~jD69E)nLgX7$Z zA!Y+OkpNK^*xn)hwtN$C+P;aI?R)6X-VPesY=GYDpom?dR%}aa)pY%&oX2*Ecc`K} zzJk5)GwdK9AfooC1lb@PW&fDejl*`L1}7RNWW=ehRevfX%UCyA~zZe1tlMc z#4}0P3*8LkbW8s}byQUu&RqAb?7X*b2 z#wb0e2SWSN13f6qwTT#!w0Y5e^ z1BTl4A_H?lMFt9S)ncUym9XkO?z8puvEwKxdy82*M~cR}mK z^GR_GYI-y6>5L|F9l*9nVMsY}ire#jhhPsL#G<$#grHV^guQVrUjsgrFg%WfM4add zJK?10Y%3io$s*|NowStm{yb9bP2sdiABPUpT0Ew#P&Abg92ozg+k=UcLYih(i?vwr zf0#-yirzxeTd3~`2*b6b+2jc-CRP_jI2`^D;^?jAiH(!{CuI$$%>`*lmP+$PT&xqi zE(XQmS0UIO$aK!UkKO>-m!oZL`W7+Q;pz@%BV{fVTIMN0SQ(hTW7ol0QHNs0@SS6gwapWO>u{EKQK6XBxgPWkBc*HySc8ee6&nmqL#nFp+R$cM`T8uQ)nY2B3Uzc7LH^d?G!^ISz%cv zU#MB{QAlcgu$%cJjy^{hF(Hyn?x(EZSemTZcx0@F#O3t@&k+Yb!(hPcZ$QIwU!vHC z7r1M?@~qVOe*dF8ul)WZ`RB*o2Y;+ST{-yWuH$_BL-+CBufF-Rd++?a?)llHduy*h z`@H(sZ*=cD?c6SPO1DeTPfshmFFk3mg_rG4neDDUS!0!pi(Pj0^zw`L-o@H!`Ozh# pSN|^k`tZ-=;MI+@v-iXE^M6M_Ki;Wse7&);e(`kWp>ZV_!+(Rtm;C?$ literal 0 HcmV?d00001 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/ac528ac1-dea2-472b-b72b-48c4fdccc8bd.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/ac528ac1-dea2-472b-b72b-48c4fdccc8bd.parquet new file mode 100644 index 00000000000..6bcedd0c6c5 --- /dev/null +++ b/extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/ac528ac1-dea2-472b-b72b-48c4fdccc8bd.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:1d28097102526b4f6ecbcac552da69348448a28f8909c155527a6b1faccffef6 +size 4239 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/metadata/00000-bbb283f2-4659-434a-b422-227b8b15dfb9.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/metadata/00000-bbb283f2-4659-434a-b422-227b8b15dfb9.metadata.json new file mode 100644 index 00000000000..ec2c53dc065 --- /dev/null +++ b/extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/metadata/00000-bbb283f2-4659-434a-b422-227b8b15dfb9.metadata.json @@ -0,0 +1,100 @@ +{ + "format-version" : 2, + "table-uuid" : "059cfc0e-7f89-4be3-913d-b1d451736783", + "location" : "s3://warehouse/sample/all_types", + "last-sequence-number" : 0, + "last-updated-ms" : 1717446052895, + "last-column-id" : 13, + "current-schema-id" : 0, + "schemas" : [ { + "type" : "struct", + "schema-id" : 0, + "fields" : [ { + "id" : 1, + "name" : "booleanField", + "required" : false, + "type" : "boolean" + }, { + "id" : 2, + "name" : "integerField", + "required" : false, + "type" : "int" + }, { + "id" : 3, + "name" : "longField", + "required" : false, + "type" : "long" + }, { + "id" : 4, + "name" : "floatField", + "required" : false, + "type" : "float" + }, { + "id" : 5, + "name" : "doubleField", + "required" : false, + "type" : "double" + }, { + "id" : 6, + "name" : "stringField", + "required" : false, + "type" : "string" + }, { + "id" : 7, + "name" : "dateField", + "required" : false, + "type" : "date" + }, { + "id" : 8, + "name" : "timeField", + "required" : false, + "type" : "time" + }, { + "id" : 9, + "name" : "timestampField", + "required" : false, + "type" : "timestamp" + }, { + "id" : 10, + "name" : "decimalField", + "required" : false, + "type" : "decimal(9, 4)" + }, { + "id" : 11, + "name" : "fixedField", + "required" : false, + "type" : "fixed[10]" + }, { + "id" : 12, + "name" : "binaryField", + "required" : false, + "type" : "binary" + }, { + "id" : 13, + "name" : "instantField", + "required" : false, + "type" : "timestamptz" + } ] + } ], + "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" : { + "write.parquet.compression-codec" : "zstd" + }, + "current-snapshot-id" : -1, + "refs" : { }, + "snapshots" : [ ], + "statistics" : [ ], + "partition-statistics" : [ ], + "snapshot-log" : [ ], + "metadata-log" : [ ] +} \ No newline at end of file diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/metadata/00001-15b8eb1c-5cf6-4266-9f0c-5ae723cd77ba.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/metadata/00001-15b8eb1c-5cf6-4266-9f0c-5ae723cd77ba.metadata.json new file mode 100644 index 00000000000..c9e2ea56748 --- /dev/null +++ b/extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/metadata/00001-15b8eb1c-5cf6-4266-9f0c-5ae723cd77ba.metadata.json @@ -0,0 +1,130 @@ +{ + "format-version" : 2, + "table-uuid" : "059cfc0e-7f89-4be3-913d-b1d451736783", + "location" : "s3://warehouse/sample/all_types", + "last-sequence-number" : 1, + "last-updated-ms" : 1717446059625, + "last-column-id" : 13, + "current-schema-id" : 0, + "schemas" : [ { + "type" : "struct", + "schema-id" : 0, + "fields" : [ { + "id" : 1, + "name" : "booleanField", + "required" : false, + "type" : "boolean" + }, { + "id" : 2, + "name" : "integerField", + "required" : false, + "type" : "int" + }, { + "id" : 3, + "name" : "longField", + "required" : false, + "type" : "long" + }, { + "id" : 4, + "name" : "floatField", + "required" : false, + "type" : "float" + }, { + "id" : 5, + "name" : "doubleField", + "required" : false, + "type" : "double" + }, { + "id" : 6, + "name" : "stringField", + "required" : false, + "type" : "string" + }, { + "id" : 7, + "name" : "dateField", + "required" : false, + "type" : "date" + }, { + "id" : 8, + "name" : "timeField", + "required" : false, + "type" : "time" + }, { + "id" : 9, + "name" : "timestampField", + "required" : false, + "type" : "timestamp" + }, { + "id" : 10, + "name" : "decimalField", + "required" : false, + "type" : "decimal(9, 4)" + }, { + "id" : 11, + "name" : "fixedField", + "required" : false, + "type" : "fixed[10]" + }, { + "id" : 12, + "name" : "binaryField", + "required" : false, + "type" : "binary" + }, { + "id" : 13, + "name" : "instantField", + "required" : false, + "type" : "timestamptz" + } ] + } ], + "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" : { + "write.parquet.compression-codec" : "zstd" + }, + "current-snapshot-id" : 6186754175552482648, + "refs" : { + "main" : { + "snapshot-id" : 6186754175552482648, + "type" : "branch" + } + }, + "snapshots" : [ { + "sequence-number" : 1, + "snapshot-id" : 6186754175552482648, + "timestamp-ms" : 1717446059625, + "summary" : { + "operation" : "append", + "added-data-files" : "1", + "added-records" : "10", + "added-files-size" : "4239", + "changed-partition-count" : "1", + "total-records" : "10", + "total-files-size" : "4239", + "total-data-files" : "1", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sample/all_types/metadata/snap-6186754175552482648-1-388d67e5-5760-40c1-a460-c8c4ee5b8629.avro", + "schema-id" : 0 + } ], + "statistics" : [ ], + "partition-statistics" : [ ], + "snapshot-log" : [ { + "timestamp-ms" : 1717446059625, + "snapshot-id" : 6186754175552482648 + } ], + "metadata-log" : [ { + "timestamp-ms" : 1717446052895, + "metadata-file" : "s3://warehouse/sample/all_types/metadata/00000-bbb283f2-4659-434a-b422-227b8b15dfb9.metadata.json" + } ] +} \ No newline at end of file diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/metadata/388d67e5-5760-40c1-a460-c8c4ee5b8629-m0.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/metadata/388d67e5-5760-40c1-a460-c8c4ee5b8629-m0.avro new file mode 100644 index 0000000000000000000000000000000000000000..ebe516d17798c8942989e13e9ba59e803350bc4b GIT binary patch literal 7582 zcmb_gYiu0V6;54BTndPXNg)a1c6cRWHr<(V{8$r63Nf$5m^v;2<8JTH-n9qM&TM98 zof^mw0))6yBOWb}gqPq45=u%D6>U|NK!|9mXlWp7)moxWk(w%1rSwNB2wL^rxik0P zS?|khOID=Jp67SJbM86!=G>iMIO}xZ94AHm{uy^G!BmMT8O0C0vKc6F=QBk*#IRBOp@1wola9Y$T>^SD68w8?c;! zj1jMq0TAQL23NLSXRMJXmEo$LT(w}kIB29B3UTE%TsiNSb2e$D&7`<`7gq<6t;T^f zDX!kl)$=%L>>FgbY7bWptYSk|D#KNK6RLiIi={?sQ~gko8_|G45nuAew2IhnWFrz= z(W|1)mBNnTUBUV;$v3hR3XN##QDH$4FK0Q}n`&f&NeO+yJ{tarF-x4ln}Y8JQGk&Mcn``x{1jC#t&m8LRV}Tx zSk}QMf7}hSP`t6ih~kuO8~z&>$f&C1NdcGbK*`YWvL#Kn1O-kUlVqK8F-N=>tQnHE zf@PDeKW~$rl1V5IoTYYoXf;9hbdZ@1tm#5a3c*Yun|DpF)fR|#= z-66k3%!=%FDSK93S<@}c#~SiJ_Bs&KnVu>W_L0S$P~<91&t|Z77s*;qHa7()&U0sl z6+77M2G|bif_xCC=3{aB&}1^0u3auVSt>59#%0Tq#h!<0hp+-tgd3WFuWTt3n%142 z)U=*B8+M&X9b$q|8wtWoVQJkGnv>Wri|UHAl%;#1(b2v#W|`RD%m#}hE<92UgQ9!jNV8a36=47P{eD>ZML>GL3!L1GKz6aJkyvL4)82 zGz79RWP&XCsOzGWaZFsI^akh%*DBM6z41(l&W7!GIU8kDlf?63AaBzZJ};)|ya?ye zmR@d|1bW+8w2}S#3O+tH_Yfi$TfUBoClGi_5icO}1BO^*{lsD>RFkhKi6;!PfzDo> znld20fxv8AW&LW(iddxOjZ3J?$$q+$_^{cuug^_mw*D>08!&j}eI8J#H zs}aW>7Bk^N9DQ;nYtBcrE1K{pj)}7$s;j!UI>)pC>zRm2Ft}nds~%R|KiP@bU(|#X zL8uUDx>m?5S~IR{u57}Kcv-7h)`a~;DRYSFI;1La7qRO=gV#Me7H1-MhgWU%6Zl{^ zZze{9{VH70T7!j)0o3GbC2|l?e4$;j1PBMsogr{MMN8c)Y#~iaTfJim6FyQFspGCthf#;=*r}T=2V2(NI@TguR zCmpbhbBbPZqYo+Jq+#XA^ScONlzqjhE-W`yGJ5TjF{F}dJ$!kx=HVOS5(4bxA-YER z3J-v_n7Y%na3k)X%E7sYAKo2p%ct**$pICvjpzfk$>=F1mJkLLCt78)7U2mFRt(X? zSoK4!8XXW(M3@^`~qeXQvoSDhFy|I1y-chp@9s4%RR#!{2f~3xt#KDI=q2 zMpIWi9p)DO>&4~6Z@l}be=a@z_ODKVYSk5O2fkE(@8C^ut?O8Pa@%$7e>mr_7hLqU z{8?MiesJl?$i9KsKKbG)>-YcG+ccCqdi$eG_V3-hdWW(h^Rww4X!UUEyj$P=j2ue$QamuLQSeE)wRKYml&qRS62yWpq$QpYkA%gXP)a^)-E zPW3&u;_+MW{QgTH-F9f;()SPT{^e=AuRr|A3)AKf~o4DufR@nzGm?eeZWY99LYu8-HB-1+yuw=dSd-u}tj9}M>I zyk+y?hTTh*6XVZKzj@9j58b=Iqhs*i`%Z1ood2D+!IkGPzVP7T&qvn%?RQh^yGNeg z_r%Xn{^*Y1q=vus;P7PE;x|4WI)S^kOuYV&f4{nSkN(`zS6}=nux3_n{NA2j%brLL z-#^p7buQSn=?dORl~6;NUZ#_J4Ny|DHEj{o=GEANTd0`0|khPrmix%%UGo VkA42~W2g458a#1s$AMXh{~u`@^@IQb literal 0 HcmV?d00001 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/metadata/snap-6186754175552482648-1-388d67e5-5760-40c1-a460-c8c4ee5b8629.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sample/all_types/metadata/snap-6186754175552482648-1-388d67e5-5760-40c1-a460-c8c4ee5b8629.avro new file mode 100644 index 0000000000000000000000000000000000000000..cc27bc3f4f23aacbf48b00d5e0784834a0755a2f GIT binary patch literal 4226 zcmbVPO^X~w7|!m=QxJ?RYDkJE2?)E=J=yulKn`LQ0s$oxWp{_zThm>=)0>*^vAcT5 zF)ZlSi<^zenq3nQ8p0yPllTV|y!ZnI!HXifD1tW+zSSR9)zj1S;c!1&B|ZGXncEs0_KDWi0%G}rtu1RF?(yF}?70pJnd!Kcz+=ZH)DE>~ z`*z_(AG0o;NP$8YxZW20wtWk5uK5;buJ56@u3tq1mvzzWEflj0)D_p(npIujT`po% z;%%wu)K{=?_zY9T14NP=Nb5U@nix4Ew|~fmsCZdQ3#!f}&UE<|P(Ex`C+cx1hkBgdQ$*3w^Js0vb6W=v| z08N6j!x=49LcQq-RZXFaLoq`S_rtEwas|m>EIq-i!n&9xcbLGUUo5cjP6G6O}M13tiIRRKre$oo|blfX(&rFK9nCQSC6k^Tg zIY|LOHZB8(+Vmm=Q$a-rN^#XxD3+95}=6_&z1rg9nKy-UuR4D+gh3n8;U`4aY4Ol$I(@+R zuDDn$>h&-v2EPiyR#&ET;(hcwz@Cb?vE|#uItf>&n31JS41wuF&1lr>)m5WjtyU|m zjY@5`aT<7fn62!PAcRG>SUHoa?@A*2i^gf!BCsR2(l!6xS<%U`og<6yxx@g(m(g?E zJzd5{Bap870QQ0#!1`zc6cH^x$J*!pldu>?7P0V4cyeM@Y5*ifMNo*rQdWNniMqYV zTLD2})CGYw0W-kPV+tf$Rgh&hmv*3pG?%c}9QHu6)&%Q#V=xQzj7C9_>K6s+IEyd~ zWorh@Ao*5D`0{3%ng3i#YxqoyCMmGP$4feq(O3a^sP+77~}&3p_^>^aO(eub+d4 z2N?eEtp{pzuKf9;cdcg&5WWAo^s zy?6Kc!NJ{U% { + /** + * The {@link TableDefinition} that will be used for the table. + */ + final TableDefinition tableDef; + + /** + * The Iceberg {@link Table} to discover locations for. + */ + final Table table; + + /** + * The {@link Snapshot} to discover locations for. + */ + final Snapshot snapshot; + + /** + * The {@link FileIO} to use for passing to the catalog reading manifest data files. + */ + final FileIO fileIO; + + /** + * The instructions for customizations while reading. + */ + final IcebergInstructions instructions; + + /** + * A cache of {@link IcebergTableLocationKey IcebergTableLocationKeys} keyed by the URI of the file they represent. + */ + final Map cache; + + /** + * The {@link ParquetInstructions} object that will be used to read any Parquet data files in this table. Only + * accessed while synchronized on {@code this}. + */ + ParquetInstructions parquetInstructions; + + protected IcebergTableLocationKey locationKey( + final org.apache.iceberg.FileFormat format, + final URI fileUri, + @Nullable final Map> partitions) { + + if (format == org.apache.iceberg.FileFormat.PARQUET) { + if (parquetInstructions == null) { + // Start with user-supplied instructions (if provided). + final ParquetInstructions.Builder builder = new ParquetInstructions.Builder(); + + // Add the table definition. + builder.setTableDefinition(tableDef); + + // Add any column rename mappings. + if (!instructions.columnRenames().isEmpty()) { + for (Map.Entry entry : instructions.columnRenames().entrySet()) { + builder.addColumnNameMapping(entry.getKey(), entry.getValue()); + } + } + + // Add the data instructions. + instructions.dataInstructions().ifPresent(builder::setSpecialInstructions); + + parquetInstructions = builder.build(); + } + return new IcebergTableParquetLocationKey(fileUri, 0, partitions, parquetInstructions); + } + throw new UnsupportedOperationException(String.format("%s:%d - an unsupported file format %s for URI '%s'", + table, snapshot.snapshotId(), format, fileUri)); + } + + /** + * @param tableDef The {@link TableDefinition} that will be used for the table. + * @param table The {@link Table} to discover locations for. + * @param tableSnapshot The {@link Snapshot} from which to discover data files. + * @param fileIO The file IO to use for reading manifest data files. + * @param instructions The instructions for customizations while reading. + */ + public IcebergBaseLayout( + @NotNull final TableDefinition tableDef, + @NotNull final Table table, + @NotNull final Snapshot tableSnapshot, + @NotNull final FileIO fileIO, + @NotNull final IcebergInstructions instructions) { + this.tableDef = tableDef; + this.table = table; + this.snapshot = tableSnapshot; + this.fileIO = fileIO; + this.instructions = instructions; + + this.cache = new HashMap<>(); + } + + abstract IcebergTableLocationKey keyFromDataFile(DataFile df, URI fileUri); + + @Override + public synchronized void findKeys(@NotNull final Consumer locationKeyObserver) { + try { + // Retrieve the manifest files from the snapshot + final List manifestFiles = snapshot.allManifests(fileIO); + for (final ManifestFile manifestFile : manifestFiles) { + // Currently only can process manifest files with DATA content type. + if (manifestFile.content() != ManifestContent.DATA) { + throw new TableDataException( + String.format("%s:%d - only DATA manifest files are currently supported, encountered %s", + table, snapshot.snapshotId(), manifestFile.content())); + } + try (final ManifestReader reader = ManifestFiles.read(manifestFile, fileIO)) { + for (DataFile df : reader) { + final URI fileUri = FileUtils.convertToURI(df.path().toString(), false); + final IcebergTableLocationKey locationKey = + cache.computeIfAbsent(fileUri, uri -> keyFromDataFile(df, fileUri)); + if (locationKey != null) { + locationKeyObserver.accept(locationKey); + } + } + } + } + } catch (final Exception e) { + throw new TableDataException( + String.format("%s:%d - error finding Iceberg locations", table, snapshot.snapshotId()), e); + } + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java new file mode 100644 index 00000000000..ac4c19283f9 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java @@ -0,0 +1,46 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.layout; + +import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; +import io.deephaven.iceberg.location.IcebergTableLocationKey; +import io.deephaven.iceberg.util.IcebergInstructions; +import org.apache.iceberg.*; +import org.apache.iceberg.io.FileIO; +import org.jetbrains.annotations.NotNull; + +import java.net.URI; + +/** + * Iceberg {@link TableLocationKeyFinder location finder} for tables without partitions that will discover data files + * from a {@link Snapshot} + */ +public final class IcebergFlatLayout extends IcebergBaseLayout { + /** + * @param tableDef The {@link TableDefinition} that will be used for the table. + * @param table The {@link Table} to discover locations for. + * @param tableSnapshot The {@link Snapshot} from which to discover data files. + * @param fileIO The file IO to use for reading manifest data files. + * @param instructions The instructions for customizations while reading. + */ + public IcebergFlatLayout( + @NotNull final TableDefinition tableDef, + @NotNull final Table table, + @NotNull final Snapshot tableSnapshot, + @NotNull final FileIO fileIO, + @NotNull final IcebergInstructions instructions) { + super(tableDef, table, tableSnapshot, fileIO, instructions); + } + + @Override + public String toString() { + return IcebergFlatLayout.class.getSimpleName() + '[' + table.name() + ']'; + } + + @Override + IcebergTableLocationKey keyFromDataFile(DataFile df, URI fileUri) { + return locationKey(df.format(), fileUri, null); + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java new file mode 100644 index 00000000000..47ec05dfd74 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java @@ -0,0 +1,104 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.layout; + +import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.table.impl.locations.TableDataException; +import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; +import io.deephaven.iceberg.location.IcebergTableLocationKey; +import io.deephaven.iceberg.util.IcebergInstructions; +import io.deephaven.util.type.TypeUtils; +import org.apache.commons.lang3.mutable.MutableInt; +import org.apache.iceberg.*; +import org.apache.iceberg.io.FileIO; +import org.jetbrains.annotations.NotNull; + +import java.net.URI; +import java.util.*; +import java.util.stream.Collectors; + +/** + * Iceberg {@link TableLocationKeyFinder location finder} for tables with partitions that will discover data files from + * a {@link Snapshot} + */ +public final class IcebergKeyValuePartitionedLayout extends IcebergBaseLayout { + private class ColumnData { + final String name; + final Class type; + final int index; + + public ColumnData(String name, Class type, int index) { + this.name = name; + this.type = type; + this.index = index; + } + } + + private final List outputPartitioningColumns; + + /** + * @param tableDef The {@link TableDefinition} that will be used for the table. + * @param table The {@link Table} to discover locations for. + * @param tableSnapshot The {@link Snapshot} from which to discover data files. + * @param fileIO The file IO to use for reading manifest data files. + * @param partitionSpec The Iceberg {@link PartitionSpec partition spec} for the table. + * @param instructions The instructions for customizations while reading. + */ + public IcebergKeyValuePartitionedLayout( + @NotNull final TableDefinition tableDef, + @NotNull final org.apache.iceberg.Table table, + @NotNull final org.apache.iceberg.Snapshot tableSnapshot, + @NotNull final FileIO fileIO, + @NotNull final PartitionSpec partitionSpec, + @NotNull final IcebergInstructions instructions) { + super(tableDef, table, tableSnapshot, fileIO, instructions); + + // We can assume due to upstream validation that there are no duplicate names (after renaming) that are included + // in the output definition, so we can ignore duplicates. + final MutableInt icebergIndex = new MutableInt(0); + final Map availablePartitioningColumns = partitionSpec.fields().stream() + .map(PartitionField::name) + .map(name -> instructions.columnRenames().getOrDefault(name, name)) + .collect(Collectors.toMap( + name -> name, + name -> icebergIndex.getAndIncrement(), + (v1, v2) -> v1, + LinkedHashMap::new)); + + outputPartitioningColumns = tableDef.getColumnStream() + .map((final ColumnDefinition columnDef) -> { + final Integer index = availablePartitioningColumns.get(columnDef.getName()); + if (index == null) { + return null; + } + return new ColumnData(columnDef.getName(), TypeUtils.getBoxedType(columnDef.getDataType()), index); + }) + .filter(Objects::nonNull) + .collect(Collectors.toList()); + } + + @Override + public String toString() { + return IcebergKeyValuePartitionedLayout.class.getSimpleName() + '[' + table.name() + ']'; + } + + @Override + IcebergTableLocationKey keyFromDataFile(DataFile df, URI fileUri) { + final Map> partitions = new LinkedHashMap<>(); + + final PartitionData partitionData = (PartitionData) df.partition(); + for (final ColumnData colData : outputPartitioningColumns) { + final String colName = colData.name; + final Object colValue = partitionData.get(colData.index); + if (colValue != null && !colData.type.isAssignableFrom(colValue.getClass())) { + throw new TableDataException("Partitioning column " + colName + + " has type " + colValue.getClass().getName() + + " but expected " + colData.type.getName()); + } + partitions.put(colName, (Comparable) colValue); + } + return locationKey(df.format(), fileUri, partitions); + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java new file mode 100644 index 00000000000..dc91d1c45fd --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationFactory.java @@ -0,0 +1,33 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.location; + +import io.deephaven.engine.table.impl.locations.TableKey; +import io.deephaven.engine.table.impl.locations.TableLocation; +import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory; +import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; +import io.deephaven.parquet.table.ParquetInstructions; +import io.deephaven.parquet.table.location.ParquetTableLocation; +import io.deephaven.parquet.table.location.ParquetTableLocationKey; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +/** + * {@link TableLocationFactory} for Iceberg {@link TableLocation}s. + */ +public final class IcebergTableLocationFactory implements TableLocationFactory { + public IcebergTableLocationFactory() {} + + @Override + @NotNull + public TableLocation makeLocation(@NotNull final TableKey tableKey, + @NotNull final IcebergTableLocationKey locationKey, + @Nullable final TableDataRefreshService refreshService) { + if (locationKey instanceof IcebergTableParquetLocationKey) { + return new ParquetTableLocation(tableKey, (ParquetTableLocationKey) locationKey, + (ParquetInstructions) locationKey.readInstructions()); + } + throw new UnsupportedOperationException("Unsupported location key type: " + locationKey.getClass()); + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java new file mode 100644 index 00000000000..d6d5d4fb514 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java @@ -0,0 +1,18 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.location; + +import io.deephaven.engine.table.impl.locations.TableLocationKey; + +/** + * {@link TableLocationKey} implementation for use with data stored in Iceberg tables. + */ +public interface IcebergTableLocationKey extends TableLocationKey { + /** + * Get the read instructions for the location. + * + * @return the read instructions + */ + Object readInstructions(); +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocationKey.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocationKey.java new file mode 100644 index 00000000000..e356d0ecb92 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableParquetLocationKey.java @@ -0,0 +1,51 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.location; + +import io.deephaven.engine.table.impl.locations.TableLocationKey; +import io.deephaven.parquet.table.ParquetInstructions; +import io.deephaven.parquet.table.location.ParquetTableLocationKey; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.net.URI; +import java.util.Map; + +/** + * {@link TableLocationKey} implementation for use with data stored in Iceberg tables in the parquet format. + */ +public class IcebergTableParquetLocationKey extends ParquetTableLocationKey implements IcebergTableLocationKey { + private static final String IMPLEMENTATION_NAME = IcebergTableParquetLocationKey.class.getSimpleName(); + + private final ParquetInstructions readInstructions; + + /** + * Construct a new IcebergTableParquetLocationKey for the supplied {@code fileUri} and {@code partitions}. + * + * @param fileUri The file that backs the keyed location + * @param order Explicit ordering index, taking precedence over other fields + * @param partitions The table partitions enclosing the table location keyed by {@code this}. Note that if this + * parameter is {@code null}, the location will be a member of no partitions. An ordered copy of the map will + * be made, so the calling code is free to mutate the map after this call + * @param readInstructions the instructions for customizations while reading + */ + public IcebergTableParquetLocationKey( + @NotNull final URI fileUri, + final int order, + @Nullable final Map> partitions, + @NotNull final ParquetInstructions readInstructions) { + super(fileUri, order, partitions, readInstructions); + this.readInstructions = readInstructions; + } + + @Override + public String getImplementationName() { + return IMPLEMENTATION_NAME; + } + + @Override + public ParquetInstructions readInstructions() { + return readInstructions; + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java new file mode 100644 index 00000000000..c379c715c6d --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java @@ -0,0 +1,462 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.util; + +import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.PartitionAwareSourceTable; +import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.locations.TableDataException; +import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocationProvider; +import io.deephaven.engine.table.impl.locations.impl.PollingTableLocationProvider; +import io.deephaven.engine.table.impl.locations.impl.StandaloneTableKey; +import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; +import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; +import io.deephaven.engine.table.impl.sources.InMemoryColumnSource; +import io.deephaven.engine.table.impl.sources.regioned.RegionedTableComponentFactoryImpl; +import io.deephaven.engine.updategraph.UpdateSourceRegistrar; +import io.deephaven.iceberg.layout.IcebergFlatLayout; +import io.deephaven.iceberg.layout.IcebergKeyValuePartitionedLayout; +import io.deephaven.iceberg.location.IcebergTableLocationFactory; +import io.deephaven.iceberg.location.IcebergTableLocationKey; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.time.Instant; +import java.time.LocalDateTime; +import java.util.*; +import java.util.stream.Collectors; + +public class IcebergCatalogAdapter { + private final Catalog catalog; + private final FileIO fileIO; + + /** + * Construct an IcebergCatalogAdapter from a catalog and file IO. + */ + IcebergCatalogAdapter( + @NotNull final Catalog catalog, + @NotNull final FileIO fileIO) { + this.catalog = catalog; + this.fileIO = fileIO; + } + + /** + * Create a single {@link TableDefinition} from a given Schema, PartitionSpec, and TableDefinition. Takes into + * account {@link Map column rename instructions} + * + * @param schema The schema of the table. + * @param partitionSpec The partition specification of the table. + * @param tableDefinition The table definition. + * @param columnRename The map for renaming columns. + * @return The generated TableDefinition. + */ + private static TableDefinition fromSchema( + @NotNull final Schema schema, + @NotNull final PartitionSpec partitionSpec, + @Nullable final TableDefinition tableDefinition, + @NotNull final Map columnRename) { + + final Set columnNames = tableDefinition != null + ? tableDefinition.getColumnNameSet() + : null; + + final Set partitionNames = + partitionSpec.fields().stream() + .map(PartitionField::name) + .map(colName -> columnRename.getOrDefault(colName, colName)) + .collect(Collectors.toSet()); + + final List> columns = new ArrayList<>(); + + for (final Types.NestedField field : schema.columns()) { + final String name = columnRename.getOrDefault(field.name(), field.name()); + // Skip columns that are not in the provided table definition. + if (columnNames != null && !columnNames.contains(name)) { + continue; + } + final Type type = field.type(); + final io.deephaven.qst.type.Type qstType = convertPrimitiveType(type); + final ColumnDefinition column; + if (partitionNames.contains(name)) { + column = ColumnDefinition.of(name, qstType).withPartitioning(); + } else { + column = ColumnDefinition.of(name, qstType); + } + columns.add(column); + } + + return TableDefinition.of(columns); + } + + /** + * Convert an Iceberg data type to a Deephaven type. + * + * @param icebergType The Iceberg data type to be converted. + * @return The converted Deephaven type. + */ + static io.deephaven.qst.type.Type convertPrimitiveType(@NotNull final Type icebergType) { + final Type.TypeID typeId = icebergType.typeId(); + switch (typeId) { + case BOOLEAN: + return io.deephaven.qst.type.Type.booleanType().boxedType(); + case DOUBLE: + return io.deephaven.qst.type.Type.doubleType(); + case FLOAT: + return io.deephaven.qst.type.Type.floatType(); + case INTEGER: + return io.deephaven.qst.type.Type.intType(); + case LONG: + return io.deephaven.qst.type.Type.longType(); + case STRING: + return io.deephaven.qst.type.Type.stringType(); + case TIMESTAMP: + final Types.TimestampType timestampType = (Types.TimestampType) icebergType; + return timestampType.shouldAdjustToUTC() + ? io.deephaven.qst.type.Type.find(Instant.class) + : io.deephaven.qst.type.Type.find(LocalDateTime.class); + case DATE: + return io.deephaven.qst.type.Type.find(java.time.LocalDate.class); + case TIME: + return io.deephaven.qst.type.Type.find(java.time.LocalTime.class); + case DECIMAL: + return io.deephaven.qst.type.Type.find(java.math.BigDecimal.class); + case FIXED: // Fall through + case BINARY: + return io.deephaven.qst.type.Type.find(byte[].class); + case UUID: // Fall through + case STRUCT: // Fall through + case LIST: // Fall through + case MAP: // Fall through + default: + throw new TableDataException("Unsupported iceberg column type " + typeId.name()); + } + } + + /** + * List all {@link Namespace namespaces} in the catalog. This method is only supported if the catalog implements + * {@link SupportsNamespaces} for namespace discovery. See {@link SupportsNamespaces#listNamespaces(Namespace)}. + * + * @return A list of all namespaces. + */ + public List listNamespaces() { + return listNamespaces(Namespace.empty()); + } + + /** + * List all {@link Namespace namespaces} in a given namespace. This method is only supported if the catalog + * implements {@link SupportsNamespaces} for namespace discovery. See + * {@link SupportsNamespaces#listNamespaces(Namespace)}. + * + * @param namespace The namespace to list namespaces in. + * @return A list of all namespaces in the given namespace. + */ + public List listNamespaces(@NotNull final Namespace namespace) { + if (catalog instanceof SupportsNamespaces) { + final SupportsNamespaces nsCatalog = (SupportsNamespaces) catalog; + return nsCatalog.listNamespaces(namespace); + } + throw new UnsupportedOperationException(String.format( + "%s does not implement org.apache.iceberg.catalog.SupportsNamespaces", catalog.getClass().getName())); + } + + /** + * List all {@link Namespace namespaces} in the catalog as a Deephaven {@link Table table}. The resulting table will + * be static and contain the same information as {@link #listNamespaces()}. + * + * @return A {@link Table table} of all namespaces. + */ + public Table listNamespacesAsTable() { + return listNamespacesAsTable(Namespace.empty()); + } + + /** + * List all {@link Namespace namespaces} in a given namespace as a Deephaven {@link Table table}. The resulting + * table will be static and contain the same information as {@link #listNamespaces(Namespace)}. + * + * @return A {@link Table table} of all namespaces. + */ + public Table listNamespacesAsTable(@NotNull final Namespace namespace) { + final List namespaces = listNamespaces(namespace); + final long size = namespaces.size(); + + // Create and return a table containing the namespaces as strings + final Map> columnSourceMap = new LinkedHashMap<>(); + + // Create the column source(s) + final String[] namespaceArr = new String[(int) size]; + columnSourceMap.put("namespace", + InMemoryColumnSource.getImmutableMemoryColumnSource(namespaceArr, String.class, null)); + + final Namespace[] namespaceObjectArr = new Namespace[(int) size]; + columnSourceMap.put("namespace_object", + InMemoryColumnSource.getImmutableMemoryColumnSource(namespaceObjectArr, Namespace.class, null)); + + // Populate the column source arrays + for (int i = 0; i < size; i++) { + final Namespace ns = namespaces.get(i); + namespaceArr[i] = ns.toString(); + namespaceObjectArr[i] = ns; + } + + // Create and return the table + return new QueryTable(RowSetFactory.flat(size).toTracking(), columnSourceMap); + } + + /** + * List all Iceberg {@link TableIdentifier tables} in a given namespace. + * + * @param namespace The namespace to list tables in. + * @return A list of all tables in the given namespace. + */ + public List listTables(@NotNull final Namespace namespace) { + return catalog.listTables(namespace); + } + + /** + * List all Iceberg {@link TableIdentifier tables} in a given namespace as a Deephaven {@link Table table}. The + * resulting table will be static and contain the same information as {@link #listTables(Namespace)}. + * + * @param namespace The namespace from which to gather the tables + * @return A list of all tables in the given namespace. + */ + public Table listTablesAsTable(@NotNull final Namespace namespace) { + final List tableIdentifiers = listTables(namespace); + final long size = tableIdentifiers.size(); + + // Create and return a table containing the namespaces as strings + final Map> columnSourceMap = new LinkedHashMap<>(); + + // Create the column source(s) + final String[] namespaceArr = new String[(int) size]; + columnSourceMap.put("namespace", + InMemoryColumnSource.getImmutableMemoryColumnSource(namespaceArr, String.class, null)); + + final String[] tableNameArr = new String[(int) size]; + columnSourceMap.put("table_name", + InMemoryColumnSource.getImmutableMemoryColumnSource(tableNameArr, String.class, null)); + + final TableIdentifier[] tableIdentifierArr = new TableIdentifier[(int) size]; + columnSourceMap.put("table_identifier_object", + InMemoryColumnSource.getImmutableMemoryColumnSource(tableIdentifierArr, TableIdentifier.class, null)); + + // Populate the column source arrays + for (int i = 0; i < size; i++) { + final TableIdentifier tableIdentifier = tableIdentifiers.get(i); + namespaceArr[i] = tableIdentifier.namespace().toString(); + tableNameArr[i] = tableIdentifier.name(); + tableIdentifierArr[i] = tableIdentifier; + } + + // Create and return the table + return new QueryTable(RowSetFactory.flat(size).toTracking(), columnSourceMap); + } + + /** + * List all {@link Snapshot snapshots} of a given Iceberg table. + * + * @param tableIdentifier The identifier of the table from which to gather snapshots. + * @return A list of all snapshots of the given table. + */ + public List listSnapshots(@NotNull final TableIdentifier tableIdentifier) { + final List snapshots = new ArrayList<>(); + catalog.loadTable(tableIdentifier).snapshots().forEach(snapshots::add); + return snapshots; + } + + /** + * List all {@link Snapshot snapshots} of a given Iceberg table as a Deephaven {@link Table table}. The resulting + * table will be static and contain the same information as {@link #listSnapshots(TableIdentifier)}. + * + * @param tableIdentifier The identifier of the table from which to gather snapshots. + * @return A list of all tables in the given namespace. + */ + public Table listSnapshotsAsTable(@NotNull final TableIdentifier tableIdentifier) { + final List snapshots = listSnapshots(tableIdentifier); + final long size = snapshots.size(); + + // Create and return a table containing the namespaces as strings + final Map> columnSourceMap = new LinkedHashMap<>(); + + // Create the column source(s) + final long[] idArr = new long[(int) size]; + columnSourceMap.put("id", InMemoryColumnSource.getImmutableMemoryColumnSource(idArr, long.class, null)); + + final long[] timestampArr = new long[(int) size]; + columnSourceMap.put("timestamp_ms", + InMemoryColumnSource.getImmutableMemoryColumnSource(timestampArr, long.class, null)); + + final String[] operatorArr = new String[(int) size]; + columnSourceMap.put("operation", + InMemoryColumnSource.getImmutableMemoryColumnSource(operatorArr, String.class, null)); + + final Map[] summaryArr = new Map[(int) size]; + columnSourceMap.put("summary", + InMemoryColumnSource.getImmutableMemoryColumnSource(summaryArr, Map.class, null)); + + final Snapshot[] snapshotArr = new Snapshot[(int) size]; + columnSourceMap.put("snapshot_object", + InMemoryColumnSource.getImmutableMemoryColumnSource(snapshotArr, Snapshot.class, null)); + + // Populate the column source(s) + for (int i = 0; i < size; i++) { + final Snapshot snapshot = snapshots.get(i); + idArr[i] = snapshot.snapshotId(); + timestampArr[i] = snapshot.timestampMillis(); + operatorArr[i] = snapshot.operation(); + summaryArr[i] = snapshot.summary(); + snapshotArr[i] = snapshot; + } + + // Create and return the table + return new QueryTable(RowSetFactory.flat(size).toTracking(), columnSourceMap); + } + + /** + * Read the latest static snapshot of an Iceberg table from the Iceberg catalog. + * + * @param tableIdentifier The table identifier to load + * @param instructions The instructions for customizations while reading + * @return The loaded table + */ + @SuppressWarnings("unused") + public Table readTable( + @NotNull final TableIdentifier tableIdentifier, + @NotNull final IcebergInstructions instructions) { + return readTableInternal(tableIdentifier, null, instructions); + } + + /** + * Read a static snapshot of an Iceberg table from the Iceberg catalog. + * + * @param tableIdentifier The table identifier to load + * @param tableSnapshotId The snapshot id to load + * @param instructions The instructions for customizations while reading + * @return The loaded table + */ + @SuppressWarnings("unused") + public Table readTable( + @NotNull final TableIdentifier tableIdentifier, + final long tableSnapshotId, + @NotNull final IcebergInstructions instructions) { + + // Find the snapshot with the given snapshot id + final Snapshot tableSnapshot = listSnapshots(tableIdentifier).stream() + .filter(snapshot -> snapshot.snapshotId() == tableSnapshotId) + .findFirst() + .orElseThrow(() -> new IllegalArgumentException("Snapshot with id " + tableSnapshotId + " not found")); + + return readTableInternal(tableIdentifier, tableSnapshot, instructions); + } + + /** + * Read a static snapshot of an Iceberg table from the Iceberg catalog. + * + * @param tableIdentifier The table identifier to load + * @param tableSnapshot The {@link Snapshot snapshot} to load + * @param instructions The instructions for customizations while reading + * @return The loaded table + */ + @SuppressWarnings("unused") + public Table readTable( + @NotNull final TableIdentifier tableIdentifier, + @NotNull final Snapshot tableSnapshot, + @NotNull final IcebergInstructions instructions) { + return readTableInternal(tableIdentifier, tableSnapshot, instructions); + } + + private Table readTableInternal( + @NotNull final TableIdentifier tableIdentifier, + @Nullable final Snapshot tableSnapshot, + @NotNull final IcebergInstructions instructions) { + + // Load the table from the catalog + final org.apache.iceberg.Table table = catalog.loadTable(tableIdentifier); + + // Do we want the latest or a specific snapshot? + final Snapshot snapshot = tableSnapshot != null ? tableSnapshot : table.currentSnapshot(); + final Schema schema = table.schemas().get(snapshot.schemaId()); + + // Load the partitioning schema + final org.apache.iceberg.PartitionSpec partitionSpec = table.spec(); + + // Get the user supplied table definition. + final TableDefinition userTableDef = instructions.tableDefinition().orElse(null); + + // Get the table definition from the schema (potentially limited by the user supplied table definition and + // applying column renames). + final TableDefinition icebergTableDef = + fromSchema(schema, partitionSpec, userTableDef, instructions.columnRenames()); + + // If the user supplied a table definition, make sure it's fully compatible. + final TableDefinition tableDef; + if (userTableDef != null) { + tableDef = icebergTableDef.checkCompatibility(userTableDef); + + // Ensure that the user has not marked non-partitioned columns as partitioned. + final Set userPartitionColumns = userTableDef.getPartitioningColumns().stream() + .map(ColumnDefinition::getName) + .collect(Collectors.toSet()); + final Set partitionColumns = tableDef.getPartitioningColumns().stream() + .map(ColumnDefinition::getName) + .collect(Collectors.toSet()); + + // The working partitioning column set must be a super-set of the user-supplied set. + if (!partitionColumns.containsAll(userPartitionColumns)) { + final Set invalidColumns = new HashSet<>(userPartitionColumns); + invalidColumns.removeAll(partitionColumns); + + throw new TableDataException("The following columns are not partitioned in the Iceberg table: " + + invalidColumns); + } + } else { + // Use the snapshot schema as the table definition. + tableDef = icebergTableDef; + } + + final String description; + final TableLocationKeyFinder keyFinder; + final TableDataRefreshService refreshService; + final UpdateSourceRegistrar updateSourceRegistrar; + + if (partitionSpec.isUnpartitioned()) { + // Create the flat layout location key finder + keyFinder = new IcebergFlatLayout(tableDef, table, snapshot, fileIO, instructions); + } else { + // Create the partitioning column location key finder + keyFinder = new IcebergKeyValuePartitionedLayout(tableDef, table, snapshot, fileIO, partitionSpec, + instructions); + } + + refreshService = null; + updateSourceRegistrar = null; + description = "Read static iceberg table with " + keyFinder; + + final AbstractTableLocationProvider locationProvider = new PollingTableLocationProvider<>( + StandaloneTableKey.getInstance(), + keyFinder, + new IcebergTableLocationFactory(), + refreshService); + + final PartitionAwareSourceTable result = new PartitionAwareSourceTable( + tableDef, + description, + RegionedTableComponentFactoryImpl.INSTANCE, + locationProvider, + updateSourceRegistrar); + + return result; + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java new file mode 100644 index 00000000000..4788e0e8714 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java @@ -0,0 +1,56 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.util; + +import io.deephaven.annotations.BuildableStyle; +import io.deephaven.engine.table.TableDefinition; +import org.immutables.value.Value.Immutable; + +import java.util.Map; +import java.util.Optional; + +/** + * This class provides instructions intended for reading Iceberg catalogs and tables. The default values documented in + * this class may change in the future. As such, callers may wish to explicitly set the values. + */ +@Immutable +@BuildableStyle +public abstract class IcebergInstructions { + public static Builder builder() { + return ImmutableIcebergInstructions.builder(); + } + + /** + * The {@link TableDefinition} to use when reading Iceberg data files. + */ + public abstract Optional tableDefinition(); + + /** + * The data instructions to use for reading the Iceberg data files (might be S3Instructions or other cloud + * provider-specific instructions). + */ + public abstract Optional dataInstructions(); + + /** + * A {@link Map map} of rename instructions from Iceberg to Deephaven column names to use when reading the Iceberg + * data files. + */ + public abstract Map columnRenames(); + + public interface Builder { + @SuppressWarnings("unused") + Builder tableDefinition(TableDefinition tableDefinition); + + @SuppressWarnings("unused") + Builder dataInstructions(Object s3Instructions); + + @SuppressWarnings("unused") + Builder putColumnRenames(String key, String value); + + @SuppressWarnings("unused") + Builder putAllColumnRenames(Map entries); + + IcebergInstructions build(); + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java new file mode 100644 index 00000000000..bcdda326dca --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java @@ -0,0 +1,19 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.util; + +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.io.FileIO; + +/** + * Tools for accessing tables in the Iceberg table format. + */ +public abstract class IcebergTools { + @SuppressWarnings("unused") + public static IcebergCatalogAdapter createAdapter( + final Catalog catalog, + final FileIO fileIO) { + return new IcebergCatalogAdapter(catalog, fileIO); + } +} diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestCatalog.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestCatalog.java new file mode 100644 index 00000000000..e62fbd282e0 --- /dev/null +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestCatalog.java @@ -0,0 +1,106 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.TestCatalog; + +import org.apache.iceberg.*; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NamespaceNotEmptyException; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.io.FileIO; + +import java.io.File; +import java.util.*; + +public class IcebergTestCatalog implements Catalog, SupportsNamespaces { + private final Map> namespaceTableMap; + private final Map tableMap; + + private IcebergTestCatalog(final String path, final FileIO fileIO) { + namespaceTableMap = new HashMap<>(); + tableMap = new HashMap<>(); + + // Assume first level is namespace. + final File root = new File(path); + for (final File namespaceFile : root.listFiles()) { + if (namespaceFile.isDirectory()) { + final Namespace namespace = Namespace.of(namespaceFile.getName()); + namespaceTableMap.putIfAbsent(namespace, new HashMap<>()); + for (final File tableFile : namespaceFile.listFiles()) { + if (tableFile.isDirectory()) { + // Second level is table name. + final TableIdentifier tableId = TableIdentifier.of(namespace, tableFile.getName()); + final Table table = IcebergTestTable.loadFromMetadata(tableFile.getAbsolutePath(), fileIO); + + // Add it to the maps. + namespaceTableMap.get(namespace).put(tableId, table); + tableMap.put(tableId, table); + } + } + } + } + } + + public static IcebergTestCatalog create(final String path, final FileIO fileIO) { + return new IcebergTestCatalog(path, fileIO); + } + + @Override + public List listTables(Namespace namespace) { + if (namespaceTableMap.containsKey(namespace)) { + return new ArrayList<>(namespaceTableMap.get(namespace).keySet()); + } + return List.of(); + } + + @Override + public boolean dropTable(TableIdentifier tableIdentifier, boolean b) { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public void renameTable(TableIdentifier tableIdentifier, TableIdentifier tableIdentifier1) { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public Table loadTable(TableIdentifier tableIdentifier) { + if (tableMap.containsKey(tableIdentifier)) { + return tableMap.get(tableIdentifier); + } + return null; + } + + @Override + public void createNamespace(Namespace namespace, Map map) { + + } + + @Override + public List listNamespaces(Namespace namespace) throws NoSuchNamespaceException { + return new ArrayList<>(namespaceTableMap.keySet()); + } + + @Override + public Map loadNamespaceMetadata(Namespace namespace) throws NoSuchNamespaceException { + return Map.of(); + } + + @Override + public boolean dropNamespace(Namespace namespace) throws NamespaceNotEmptyException { + return false; + } + + @Override + public boolean setProperties(Namespace namespace, Map map) throws NoSuchNamespaceException { + return false; + } + + @Override + public boolean removeProperties(Namespace namespace, Set set) throws NoSuchNamespaceException { + return false; + } +} diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestFileIO.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestFileIO.java new file mode 100644 index 00000000000..03be6ca1b5e --- /dev/null +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestFileIO.java @@ -0,0 +1,49 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.TestCatalog; + +import org.apache.iceberg.inmemory.InMemoryFileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.HashSet; +import java.util.Set; + +public class IcebergTestFileIO extends InMemoryFileIO { + private final Set inputFiles; + private final String matchPrefix; + private final String replacePrefix; + + public IcebergTestFileIO(final String matchPrefix, final String replacePrefix) { + this.matchPrefix = matchPrefix; + this.replacePrefix = replacePrefix; + inputFiles = new HashSet<>(); + } + + @Override + public InputFile newInputFile(String s) { + if (!inputFiles.contains(s)) { + try { + final String replaced = s.replace(matchPrefix, replacePrefix); + final byte[] data = Files.readAllBytes(Path.of(replaced)); + addFile(s, data); + inputFiles.add(s); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + return super.newInputFile(s); + } + + @Override + public OutputFile newOutputFile(String s) { + return null; + } + + @Override + public void deleteFile(String s) {} +} diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestTable.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestTable.java new file mode 100644 index 00000000000..d1cf5c2ee0e --- /dev/null +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergTestTable.java @@ -0,0 +1,239 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.TestCatalog; + +import org.apache.iceberg.*; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.LocationProvider; +import org.jetbrains.annotations.NotNull; +import org.testcontainers.shaded.org.apache.commons.lang3.NotImplementedException; + +import java.io.File; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class IcebergTestTable implements Table { + private final TableMetadata metadata; + private final FileIO fileIO; + + private IcebergTestTable(@NotNull final String path, @NotNull final FileIO fileIO) { + final File metadataRoot = new File(path, "metadata"); + this.fileIO = fileIO; + + final List metadataFiles = new ArrayList<>(); + + // Get a list of the JSON files. + for (final File file : metadataRoot.listFiles()) { + if (!file.isDirectory() && file.getName().endsWith(".json")) { + metadataFiles.add(file.getAbsolutePath()); + } + } + + // The last entry after sorting will be the newest / current. + metadataFiles.sort(String::compareTo); + final Path tablePath = Path.of(metadataFiles.get(metadataFiles.size() - 1)); + try { + final String tableJson = new String(java.nio.file.Files.readAllBytes(tablePath)); + metadata = TableMetadataParser.fromJson(tableJson); + } catch (Exception e) { + throw new RuntimeException("Failed to read table file: " + tablePath, e); + } + } + + public static IcebergTestTable loadFromMetadata(@NotNull final String path, @NotNull final FileIO fileIO) { + return new IcebergTestTable(path, fileIO); + } + + @Override + public void refresh() {} + + @Override + public TableScan newScan() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public Schema schema() { + return metadata.schema(); + } + + @Override + public Map schemas() { + final Map schemaMap = new java.util.HashMap<>(); + final List schemas = metadata.schemas(); + for (int i = 0; i < schemas.size(); i++) { + schemaMap.put(i, schemas.get(i)); + } + return schemaMap; + } + + @Override + public PartitionSpec spec() { + return metadata.spec(); + } + + @Override + public Map specs() { + final List partitionSpecs = metadata.specs(); + final Map specMap = new java.util.HashMap<>(); + for (int i = 0; i < partitionSpecs.size(); i++) { + specMap.put(i, partitionSpecs.get(i)); + } + return specMap; + } + + @Override + public SortOrder sortOrder() { + return metadata.sortOrder(); + } + + @Override + public Map sortOrders() { + final List sortOrders = metadata.sortOrders(); + final Map sortOrderMap = new java.util.HashMap<>(); + for (int i = 0; i < sortOrders.size(); i++) { + sortOrderMap.put(i, sortOrders.get(i)); + } + return sortOrderMap; + } + + @Override + public Map properties() { + return metadata.properties(); + } + + @Override + public String location() { + return metadata.location(); + } + + @Override + public Snapshot currentSnapshot() { + return metadata.currentSnapshot(); + } + + @Override + public Snapshot snapshot(long l) { + final List snapshots = metadata.snapshots(); + for (final Snapshot snapshot : snapshots) { + if (snapshot.snapshotId() == l) { + return snapshot; + } + } + return null; + } + + @Override + public Iterable snapshots() { + return metadata.snapshots(); + } + + @Override + public List history() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public UpdateSchema updateSchema() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public UpdatePartitionSpec updateSpec() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public UpdateProperties updateProperties() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public ReplaceSortOrder replaceSortOrder() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public UpdateLocation updateLocation() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public AppendFiles newAppend() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public RewriteFiles newRewrite() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public RewriteManifests rewriteManifests() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public OverwriteFiles newOverwrite() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public RowDelta newRowDelta() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public ReplacePartitions newReplacePartitions() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public DeleteFiles newDelete() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public ExpireSnapshots expireSnapshots() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public ManageSnapshots manageSnapshots() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public Transaction newTransaction() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public FileIO io() { + return fileIO; + } + + @Override + public EncryptionManager encryption() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public LocationProvider locationProvider() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public List statisticsFiles() { + return metadata.statisticsFiles(); + } + + @Override + public Map refs() { + throw new NotImplementedException("Not implemented"); + } +} diff --git a/extensions/parquet/table/build.gradle b/extensions/parquet/table/build.gradle index 6d594e29a41..1c80142c1d8 100644 --- a/extensions/parquet/table/build.gradle +++ b/extensions/parquet/table/build.gradle @@ -48,9 +48,7 @@ dependencies { testImplementation TestTools.projectDependency(project, 'extensions-s3') Classpaths.inheritJUnitClassic(project, 'testImplementation') - testImplementation "org.testcontainers:testcontainers:1.19.4" - testImplementation "org.testcontainers:localstack:1.19.4" - testImplementation "org.testcontainers:minio:1.19.4" + Classpaths.inheritTestContainers(project) testRuntimeOnly project(':log-to-slf4j'), project(path: ':configs'), diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetInstructions.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetInstructions.java index 3b28644c691..069ddd4dac6 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetInstructions.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetInstructions.java @@ -87,7 +87,7 @@ public static void setDefaultMaximumDictionarySize(final int maximumDictionarySi /** * @return The default for {@link #getMaximumDictionarySize()} */ - public static int getDefaltMaximumDictionarySize() { + public static int getDefaultMaximumDictionarySize() { return defaultMaximumDictionarySize; } @@ -147,7 +147,7 @@ public enum ParquetFileLayout { *
  • A single parquet {@value ParquetUtils#COMMON_METADATA_FILE_NAME} file * */ - METADATA_PARTITIONED; + METADATA_PARTITIONED } private static final boolean DEFAULT_GENERATE_METADATA_FILES = false; @@ -257,10 +257,10 @@ public static boolean sameColumnNamesAndCodecMappings(final ParquetInstructions if (i2 == EMPTY) { return true; } - return ((ReadOnly) i2).columnNameToInstructions.size() == 0; + return ((ReadOnly) i2).columnNameToInstructions.isEmpty(); } if (i2 == EMPTY) { - return ((ReadOnly) i1).columnNameToInstructions.size() == 0; + return ((ReadOnly) i1).columnNameToInstructions.isEmpty(); } return ReadOnly.sameCodecMappings((ReadOnly) i1, (ReadOnly) i2); } @@ -723,7 +723,7 @@ public Builder(final ParquetInstructions parquetInstructions) { } private void newColumnNameToInstructionsMap() { - columnNameToInstructions = new KeyedObjectHashMap<>(new KeyedObjectKey.Basic() { + columnNameToInstructions = new KeyedObjectHashMap<>(new KeyedObjectKey.Basic<>() { @Override public String getKey(@NotNull final ColumnInstructions value) { return value.getColumnName(); @@ -733,7 +733,7 @@ public String getKey(@NotNull final ColumnInstructions value) { private void newParquetColumnNameToInstructionsMap() { parquetColumnNameToInstructions = - new KeyedObjectHashMap<>(new KeyedObjectKey.Basic() { + new KeyedObjectHashMap<>(new KeyedObjectKey.Basic<>() { @Override public String getKey(@NotNull final ColumnInstructions value) { return value.getParquetColumnName(); diff --git a/extensions/s3/build.gradle b/extensions/s3/build.gradle index 64954314387..7d1343c052b 100644 --- a/extensions/s3/build.gradle +++ b/extensions/s3/build.gradle @@ -13,9 +13,7 @@ dependencies { implementation project(':Configuration') implementation project(':log-factory') - implementation platform('software.amazon.awssdk:bom:2.23.19') - implementation 'software.amazon.awssdk:s3' - implementation 'software.amazon.awssdk:aws-crt-client' + Classpaths.inheritAWSSDK(project) compileOnly depAnnotations @@ -32,10 +30,7 @@ dependencies { testRuntimeOnly 'org.junit.platform:junit-platform-launcher' testImplementation 'software.amazon.awssdk:s3-transfer-manager' - testImplementation "org.testcontainers:testcontainers:1.19.4" - testImplementation "org.testcontainers:junit-jupiter:1.19.4" - testImplementation "org.testcontainers:localstack:1.19.4" - testImplementation "org.testcontainers:minio:1.19.4" + Classpaths.inheritTestContainers(project) testRuntimeOnly project(':test-configs') testRuntimeOnly project(':log-to-slf4j') diff --git a/py/embedded-server/java-runtime/build.gradle b/py/embedded-server/java-runtime/build.gradle index 56111b847c5..b64ddaf2958 100644 --- a/py/embedded-server/java-runtime/build.gradle +++ b/py/embedded-server/java-runtime/build.gradle @@ -32,6 +32,7 @@ dependencies { if (!hasProperty('excludeS3')) { runtimeOnly project(':extensions-s3') + runtimeOnly project(':extensions-iceberg:s3') } } diff --git a/server/build.gradle b/server/build.gradle index 27f6f18aa0a..d3ccceabb13 100644 --- a/server/build.gradle +++ b/server/build.gradle @@ -9,6 +9,7 @@ dependencies { implementation project(':engine-table') implementation project(':extensions-csv') implementation project(':extensions-arrow') + implementation project(':extensions-iceberg') implementation project(':extensions-parquet-table') implementation project(':extensions-performance') implementation project(':extensions-jdbc') diff --git a/server/jetty-app-custom/build.gradle b/server/jetty-app-custom/build.gradle index ed64f1cdab7..11917d41326 100644 --- a/server/jetty-app-custom/build.gradle +++ b/server/jetty-app-custom/build.gradle @@ -55,6 +55,7 @@ if (!hasProperty('excludeSql')) { if (!hasProperty('excludeS3')) { dependencies { runtimeOnly project(':extensions-s3') + runtimeOnly project(':extensions-iceberg:s3') } } diff --git a/server/jetty-app/build.gradle b/server/jetty-app/build.gradle index 407235af4ae..39bb137af16 100644 --- a/server/jetty-app/build.gradle +++ b/server/jetty-app/build.gradle @@ -54,6 +54,7 @@ if (!hasProperty('excludeSql')) { if (!hasProperty('excludeS3')) { dependencies { runtimeOnly project(':extensions-s3') + runtimeOnly project(':extensions-iceberg:s3') } } diff --git a/server/netty-app/build.gradle b/server/netty-app/build.gradle index 15a180d998b..cd5b23ea1bb 100644 --- a/server/netty-app/build.gradle +++ b/server/netty-app/build.gradle @@ -54,6 +54,7 @@ if (!hasProperty('excludeSql')) { if (!hasProperty('excludeS3')) { dependencies { runtimeOnly project(':extensions-s3') + runtimeOnly project(':extensions-iceberg:s3') } } diff --git a/settings.gradle b/settings.gradle index 852e08360bd..9a240a942db 100644 --- a/settings.gradle +++ b/settings.gradle @@ -267,6 +267,12 @@ project(':extensions-trackedfile').projectDir = file('extensions/trackedfile') include(':extensions-s3') project(':extensions-s3').projectDir = file('extensions/s3') +include(':extensions-iceberg') +project(':extensions-iceberg').projectDir = file('extensions/iceberg') + +include(':extensions-iceberg:s3') +project(':extensions-iceberg:s3').projectDir = file('extensions/iceberg/s3') + include(':plugin') include(':plugin-dagger')