From ecdc8e7ad9772018cdcb93267206649fb15b9632 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Sat, 23 Nov 2024 03:30:54 +0530 Subject: [PATCH] feat: Added support to write iceberg tables (#5989) Closes: #6125 Also moves existing Iceberg tests from Junit4 to Junit5. --- .../iceberg/util/IcebergToolsS3.java | 34 +- .../iceberg/util/IcebergToolsTest.java | 86 +- .../util/IcebergWriteInstructionsTest.java | 51 + .../LocalstackWarehouseSqliteCatalogTest.java | 9 +- .../util/MinioWarehouseSqliteCatalogTest.java | 9 +- .../util/S3WarehouseSqliteCatalogBase.java | 7 +- .../util/TableParquetWriterOptionsTest.java | 160 ++++ .../deephaven/iceberg/base/IcebergUtils.java | 320 +++++++ .../iceberg/layout/IcebergBaseLayout.java | 48 +- .../iceberg/util/IcebergCatalogAdapter.java | 68 +- .../iceberg/util/IcebergReadInstructions.java | 3 +- .../iceberg/util/IcebergTableAdapter.java | 72 +- .../iceberg/util/IcebergTableWriter.java | 525 ++++++++++ .../util/IcebergWriteInstructions.java | 76 ++ .../iceberg/util/SchemaProvider.java | 33 + .../iceberg/util/SchemaProviderInternal.java | 117 +++ .../util/TableParquetWriterOptions.java | 130 +++ .../iceberg/util/TableWriterOptions.java | 114 +++ .../io/deephaven/iceberg/PyIceberg1Test.java | 2 +- .../FileWarehouseSqliteCatalogTest.java | 9 +- .../iceberg/junit5/SqliteCatalogBase.java | 898 +++++++++++++++++- .../parquet/base/ParquetFileWriter.java | 7 + .../parquet/table/CompletedParquetWrite.java | 61 ++ .../parquet/table/ParquetInstructions.java | 46 +- .../parquet/table/ParquetTableWriter.java | 23 +- .../deephaven/parquet/table/ParquetTools.java | 13 +- .../table/ParquetTableReadWriteTest.java | 34 + .../table/S3ParquetLocalStackTest.java | 1 - py/server/deephaven/experimental/iceberg.py | 325 ++++++- py/server/deephaven/experimental/s3.py | 1 + py/server/tests/test_iceberg.py | 67 +- 31 files changed, 3157 insertions(+), 192 deletions(-) create mode 100644 extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergWriteInstructionsTest.java create mode 100644 extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/TableParquetWriterOptionsTest.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/base/IcebergUtils.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableWriter.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteInstructions.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/SchemaProvider.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/SchemaProviderInternal.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/TableParquetWriterOptions.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/TableWriterOptions.java create mode 100644 extensions/parquet/table/src/main/java/io/deephaven/parquet/table/CompletedParquetWrite.java 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 index 22e25a2a23c..ef4f98b5d11 100644 --- 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 @@ -13,7 +13,9 @@ import org.apache.iceberg.aws.AwsProperties; import org.apache.iceberg.aws.HttpClientProperties; import org.apache.iceberg.aws.glue.GlueCatalog; +import org.apache.iceberg.aws.s3.S3FileIO; import org.apache.iceberg.aws.s3.S3FileIOProperties; +import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.rest.RESTCatalog; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -22,9 +24,8 @@ import java.util.Map; /** - * Tools for accessing tables in the Iceberg table format. + * Tools for accessing tables in the Iceberg table format from S3. */ -@SuppressWarnings("unused") public final class IcebergToolsS3 { /** @@ -52,14 +53,6 @@ public static IcebergCatalogAdapter createS3Rest( // 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); @@ -71,10 +64,9 @@ public static IcebergCatalogAdapter createS3Rest( properties.put(S3FileIOProperties.ENDPOINT, endpointOverride); } - final String catalogName = name != null ? name : "IcebergCatalog-" + catalogURI; + final RESTCatalog catalog = new RESTCatalog(); catalog.setConf(new Configuration()); - catalog.initialize(catalogName, properties); - return IcebergCatalogAdapter.of(catalog); + return createAdapterCommon(name, catalogURI, warehouseLocation, catalog, properties); } /** @@ -97,16 +89,28 @@ public static IcebergCatalogAdapter createGlue( final Map properties = new HashMap<>(); final GlueCatalog catalog = new GlueCatalog(); + catalog.setConf(new Configuration()); + return createAdapterCommon(name, catalogURI, warehouseLocation, catalog, properties); + } + private static IcebergCatalogAdapter createAdapterCommon( + @Nullable final String name, + @NotNull final String catalogURI, + @NotNull final String warehouseLocation, + @NotNull final Catalog catalog, + @NotNull final Map properties) { properties.put(CatalogProperties.CATALOG_IMPL, catalog.getClass().getName()); properties.put(CatalogProperties.URI, catalogURI); properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation); + // Following is needed to write new manifest files when writing new data. + // Not setting this will result in using ResolvingFileIO. + properties.put(CatalogProperties.FILE_IO_IMPL, S3FileIO.class.getName()); + final String catalogName = name != null ? name : "IcebergCatalog-" + catalogURI; - catalog.setConf(new Configuration()); catalog.initialize(catalogName, properties); - return new IcebergCatalogAdapter(catalog, properties); + return IcebergCatalogAdapter.of(catalog, properties); } /** 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 index fbd36ce1f77..0920d803fae 100644 --- 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 @@ -15,9 +15,11 @@ import io.deephaven.extensions.s3.S3Instructions; import io.deephaven.iceberg.TestCatalog.IcebergTestCatalog; import org.apache.iceberg.Schema; +import io.deephaven.iceberg.base.IcebergUtils; import org.apache.iceberg.Snapshot; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.types.Type; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -31,6 +33,7 @@ import java.io.File; import java.math.BigDecimal; +import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; @@ -99,7 +102,7 @@ public abstract class IcebergToolsTest { ColumnDefinition.ofString("ColumnType"), ColumnDefinition.ofBoolean("IsPartitioning")); - IcebergReadInstructions instructions; + private IcebergReadInstructions instructions; public abstract S3AsyncClient s3AsyncClient(); @@ -115,10 +118,10 @@ public abstract class IcebergToolsTest { private String warehousePath; private IcebergTestCatalog resourceCatalog; - public final EngineCleanup framework = new EngineCleanup(); + private final EngineCleanup framework = new EngineCleanup(); @BeforeEach - public void setUp() throws Exception { + void setUp() throws Exception { framework.setUp(); bucket = "warehouse"; asyncClient = s3AsyncClient(); @@ -137,7 +140,7 @@ public void setUp() throws Exception { } @AfterEach - public void tearDown() throws Exception { + void tearDown() throws Exception { resourceCatalog.close(); for (String key : keys) { asyncClient.deleteObject(DeleteObjectRequest.builder().bucket(bucket).key(key).build()).get(); @@ -195,7 +198,7 @@ private void uploadSalesRenamed() throws ExecutionException, InterruptedExceptio } @Test - public void testListNamespaces() { + void testListNamespaces() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); final Collection namespaces = adapter.listNamespaces(); @@ -212,7 +215,7 @@ public void testListNamespaces() { } @Test - public void testListTables() { + void testListTables() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); @@ -236,7 +239,7 @@ public void testListTables() { } @Test - public void testGetTableAdapter() { + void testGetTableAdapter() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); // Test the overloads of the load() method. @@ -252,7 +255,7 @@ public void testGetTableAdapter() { } @Test - public void testListSnapshots() { + void testListSnapshots() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_multi"); @@ -275,7 +278,7 @@ public void testListSnapshots() { } @Test - public void testOpenTableA() throws ExecutionException, InterruptedException, TimeoutException { + void testOpenTableA() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); @@ -301,7 +304,7 @@ public void testOpenTableB() throws ExecutionException, InterruptedException, Ti } @Test - public void testOpenTableC() throws ExecutionException, InterruptedException, TimeoutException { + void testOpenTableC() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesSingle(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); @@ -314,7 +317,7 @@ public void testOpenTableC() throws ExecutionException, InterruptedException, Ti } @Test - public void testOpenTableS3Only() throws ExecutionException, InterruptedException, TimeoutException { + void testOpenTableS3Only() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); @@ -327,7 +330,7 @@ public void testOpenTableS3Only() throws ExecutionException, InterruptedExceptio } @Test - public void testOpenTableDefinition() throws ExecutionException, InterruptedException, TimeoutException { + void testOpenTableDefinition() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); @@ -340,7 +343,7 @@ public void testOpenTableDefinition() throws ExecutionException, InterruptedExce } @Test - public void testOpenTablePartitionTypeException() { + void testOpenTablePartitionTypeException() { final TableDefinition tableDef = TableDefinition.of( ColumnDefinition.ofLong("year").withPartitioning(), ColumnDefinition.ofInt("month").withPartitioning(), @@ -372,7 +375,7 @@ public void testOpenTablePartitionTypeException() { } @Test - public void testOpenTableDefinitionRename() throws ExecutionException, InterruptedException, TimeoutException { + void testOpenTableDefinitionRename() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); final TableDefinition renamed = TableDefinition.of( @@ -406,7 +409,7 @@ public void testOpenTableDefinitionRename() throws ExecutionException, Interrupt } @Test - public void testSkippedPartitioningColumn() throws ExecutionException, InterruptedException, TimeoutException { + void testSkippedPartitioningColumn() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); final TableDefinition tableDef = TableDefinition.of( @@ -434,7 +437,7 @@ public void testSkippedPartitioningColumn() throws ExecutionException, Interrupt @Test - public void testReorderedPartitioningColumn() throws ExecutionException, InterruptedException, TimeoutException { + void testReorderedPartitioningColumn() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); final TableDefinition tableDef = TableDefinition.of( @@ -461,7 +464,7 @@ public void testReorderedPartitioningColumn() throws ExecutionException, Interru } @Test - public void testZeroPartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { + void testZeroPartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); final IcebergReadInstructions localInstructions = IcebergReadInstructions.builder() @@ -479,7 +482,7 @@ public void testZeroPartitioningColumns() throws ExecutionException, Interrupted } @Test - public void testIncorrectPartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { + void testIncorrectPartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { final TableDefinition tableDef = TableDefinition.of( ColumnDefinition.ofInt("month").withPartitioning(), ColumnDefinition.ofInt("year").withPartitioning(), @@ -512,7 +515,7 @@ public void testIncorrectPartitioningColumns() throws ExecutionException, Interr } @Test - public void testMissingPartitioningColumns() { + void testMissingPartitioningColumns() { final TableDefinition tableDef = TableDefinition.of( ColumnDefinition.ofInt("__year").withPartitioning(), // Incorrect name ColumnDefinition.ofInt("__month").withPartitioning(), // Incorrect name @@ -544,7 +547,7 @@ public void testMissingPartitioningColumns() { } @Test - public void testOpenTableColumnRename() throws ExecutionException, InterruptedException, TimeoutException { + void testOpenTableColumnRename() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); final IcebergReadInstructions localInstructions = IcebergReadInstructions.builder() @@ -562,7 +565,7 @@ public void testOpenTableColumnRename() throws ExecutionException, InterruptedEx } @Test - public void testOpenTableColumnLegalization() throws ExecutionException, InterruptedException, TimeoutException { + void testOpenTableColumnLegalization() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesRenamed(); final IcebergReadInstructions localInstructions = IcebergReadInstructions.builder() @@ -579,7 +582,7 @@ public void testOpenTableColumnLegalization() throws ExecutionException, Interru } @Test - public void testOpenTableColumnLegalizationRename() + void testOpenTableColumnLegalizationRename() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesRenamed(); @@ -607,7 +610,7 @@ public void testOpenTableColumnLegalizationRename() } @Test - public void testOpenTableColumnLegalizationPartitionException() { + void testOpenTableColumnLegalizationPartitionException() { final TableDefinition tableDef = TableDefinition.of( ColumnDefinition.ofInt("Year").withPartitioning(), ColumnDefinition.ofInt("Month").withPartitioning()); @@ -636,7 +639,7 @@ public void testOpenTableColumnLegalizationPartitionException() { } @Test - public void testOpenTableColumnRenamePartitioningColumns() + void testOpenTableColumnRenamePartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); @@ -666,7 +669,7 @@ public void testOpenTableColumnRenamePartitioningColumns() } @Test - public void testOpenTableSnapshot() throws ExecutionException, InterruptedException, TimeoutException { + void testOpenTableSnapshot() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesMulti(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); @@ -706,7 +709,7 @@ public void testOpenTableSnapshot() throws ExecutionException, InterruptedExcept } @Test - public void testOpenTableSnapshotByID() throws ExecutionException, InterruptedException, TimeoutException { + void testOpenTableSnapshotByID() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesMulti(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); @@ -748,7 +751,7 @@ public void testOpenTableSnapshotByID() throws ExecutionException, InterruptedEx } @Test - public void testOpenAllTypesTable() throws ExecutionException, InterruptedException, TimeoutException { + void testOpenAllTypesTable() throws ExecutionException, InterruptedException, TimeoutException { uploadAllTypes(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); @@ -761,7 +764,7 @@ public void testOpenAllTypesTable() throws ExecutionException, InterruptedExcept } @Test - public void testTableDefinition() { + void testTableDefinition() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_multi"); final List snapshots = tableAdapter.listSnapshots(); @@ -788,7 +791,7 @@ public void testTableDefinition() { } @Test - public void testTableSchema() { + void testTableSchema() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_multi"); @@ -810,7 +813,7 @@ public void testTableSchema() { } @Test - public void testTableDefinitionTable() { + void testTableDefinitionTable() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_multi"); final List snapshots = tableAdapter.listSnapshots(); @@ -842,7 +845,7 @@ public void testTableDefinitionTable() { } @Test - public void testTableDefinitionWithInstructions() { + void testTableDefinitionWithInstructions() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_multi"); @@ -884,7 +887,7 @@ public void testTableDefinitionWithInstructions() { } @Test - public void testManualRefreshingTable() throws ExecutionException, InterruptedException, TimeoutException { + void testManualRefreshingTable() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesMulti(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); @@ -925,4 +928,23 @@ public void testManualRefreshingTable() throws ExecutionException, InterruptedEx updateGraph.runWithinUnitTestCycle(table::refresh); Assert.eq(table.size(), "table.size()", 0, "expected rows in the table"); } + + @Test + void testConvertToIcebergTypeAndBack() { + final Class[] javaTypes = { + Boolean.class, double.class, float.class, int.class, long.class, String.class, Instant.class, + LocalDateTime.class, LocalDate.class, LocalTime.class, byte[].class + }; + + for (final Class javaType : javaTypes) { + // Java type -> Iceberg type + final Type icebergType = IcebergUtils.convertToIcebergType(javaType); + + // Iceberg type -> Deephaven type + final io.deephaven.qst.type.Type deephavenType = IcebergUtils.convertToDHType(icebergType); + + // Deephaven type == Java type + Assert.eq(javaType, javaType.getName(), deephavenType.clazz(), deephavenType.clazz().getName()); + } + } } diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergWriteInstructionsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergWriteInstructionsTest.java new file mode 100644 index 00000000000..631fea540f1 --- /dev/null +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergWriteInstructionsTest.java @@ -0,0 +1,51 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.util; + +import io.deephaven.engine.table.Table; +import io.deephaven.engine.util.TableTools; +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.failBecauseExceptionWasNotThrown; +import static org.assertj.core.api.Assertions.assertThat; + +class IcebergWriteInstructionsTest { + + @Test + void testSetDhTables() { + final Table table1 = TableTools.emptyTable(3); + final Table table2 = TableTools.emptyTable(4); + final IcebergWriteInstructions instructions = IcebergWriteInstructions.builder() + .addTables(table1) + .addTables(table2) + .build(); + assertThat(instructions.tables()).hasSize(2); + assertThat(instructions.tables()).contains(table1); + assertThat(instructions.tables()).contains(table2); + } + + @Test + void testSetPartitionPaths() { + final Table table1 = TableTools.emptyTable(3); + final String pp1 = "P1C=1/PC2=2"; + final Table table2 = TableTools.emptyTable(4); + final String pp2 = "P1C=2/PC2=3"; + try { + final IcebergWriteInstructions instructions = IcebergWriteInstructions.builder() + .addPartitionPaths(pp1, pp2) + .build(); + failBecauseExceptionWasNotThrown(IllegalArgumentException.class); + } catch (final IllegalArgumentException e) { + assertThat(e).hasMessageContaining("Partition path must be provided for each table"); + } + + final IcebergWriteInstructions instructions = IcebergWriteInstructions.builder() + .addTables(table1, table2) + .addPartitionPaths(pp1, pp2) + .build(); + assertThat(instructions.partitionPaths()).hasSize(2); + assertThat(instructions.partitionPaths()).contains(pp1); + assertThat(instructions.partitionPaths()).contains(pp2); + } +} diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/LocalstackWarehouseSqliteCatalogTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/LocalstackWarehouseSqliteCatalogTest.java index fb9c8e020f7..29b27a8485f 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/LocalstackWarehouseSqliteCatalogTest.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/LocalstackWarehouseSqliteCatalogTest.java @@ -9,17 +9,20 @@ import org.junit.jupiter.api.Tag; import software.amazon.awssdk.services.s3.S3AsyncClient; +import java.time.Duration; + @Tag("testcontainers") -public final class LocalstackWarehouseSqliteCatalogTest extends S3WarehouseSqliteCatalogBase { +final class LocalstackWarehouseSqliteCatalogTest extends S3WarehouseSqliteCatalogBase { @BeforeAll - public static void initContainer() { + static void initContainer() { // ensure container is started so container startup time isn't associated with a specific test LocalStack.init(); } @Override public S3Instructions s3Instructions() { - return LocalStack.s3Instructions(S3Instructions.builder()).build(); + return LocalStack.s3Instructions(S3Instructions.builder() + .readTimeout(Duration.ofSeconds(10))).build(); } @Override diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/MinioWarehouseSqliteCatalogTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/MinioWarehouseSqliteCatalogTest.java index 1a569ab88ab..ed4f8560c57 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/MinioWarehouseSqliteCatalogTest.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/MinioWarehouseSqliteCatalogTest.java @@ -11,10 +11,12 @@ import org.junit.jupiter.api.Tag; import software.amazon.awssdk.services.s3.S3AsyncClient; +import java.time.Duration; + @Tag("testcontainers") -public final class MinioWarehouseSqliteCatalogTest extends S3WarehouseSqliteCatalogBase { +final class MinioWarehouseSqliteCatalogTest extends S3WarehouseSqliteCatalogBase { @BeforeAll - public static void initContainer() { + 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 @@ -23,7 +25,8 @@ public static void initContainer() { @Override public S3Instructions s3Instructions() { - return MinIO.s3Instructions(S3Instructions.builder()).build(); + return MinIO.s3Instructions(S3Instructions.builder() + .readTimeout(Duration.ofSeconds(10))).build(); } @Override diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/S3WarehouseSqliteCatalogBase.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/S3WarehouseSqliteCatalogBase.java index acd529037f5..8beca64e938 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/S3WarehouseSqliteCatalogBase.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/S3WarehouseSqliteCatalogBase.java @@ -20,12 +20,17 @@ import static io.deephaven.extensions.s3.testlib.S3Helper.TIMEOUT_SECONDS; -public abstract class S3WarehouseSqliteCatalogBase extends SqliteCatalogBase { +abstract class S3WarehouseSqliteCatalogBase extends SqliteCatalogBase { public abstract S3Instructions s3Instructions(); public abstract S3AsyncClient s3AsyncClient(); + @Override + public final Object dataInstructions() { + return s3Instructions(); + } + @Override protected IcebergCatalogAdapter catalogAdapter(TestInfo testInfo, Path rootDir, Map properties) throws ExecutionException, InterruptedException, TimeoutException { diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/TableParquetWriterOptionsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/TableParquetWriterOptionsTest.java new file mode 100644 index 00000000000..d6d555321c0 --- /dev/null +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/TableParquetWriterOptionsTest.java @@ -0,0 +1,160 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.util; + +import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.TableDefinition; +import io.deephaven.parquet.table.ParquetInstructions; +import org.junit.jupiter.api.Test; + +import java.util.Map; + +import static org.assertj.core.api.Assertions.failBecauseExceptionWasNotThrown; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +class TableParquetWriterOptionsTest { + + /** + * Create a new TableParquetWriterOptions builder with an empty table definition. + */ + private static TableParquetWriterOptions.Builder instructions() { + return TableParquetWriterOptions.builder().tableDefinition(TableDefinition.of( + ColumnDefinition.ofInt("someCol"))); + } + + @Test + void defaults() { + final TableParquetWriterOptions instructions = instructions().build(); + assertThat(instructions.dataInstructions()).isEmpty(); + assertThat(instructions.compressionCodecName()).isEqualTo("SNAPPY"); + assertThat(instructions.maximumDictionaryKeys()).isEqualTo(1048576); + assertThat(instructions.maximumDictionarySize()).isEqualTo(1048576); + assertThat(instructions.targetPageSize()).isEqualTo(65536); + } + + @Test + void testSetTableDefinition() { + final TableDefinition definition = TableDefinition.of( + ColumnDefinition.ofInt("PC1").withPartitioning(), + ColumnDefinition.ofInt("PC2").withPartitioning(), + ColumnDefinition.ofLong("I")); + assertThat(TableParquetWriterOptions.builder() + .tableDefinition(definition) + .build() + .tableDefinition()) + .isEqualTo(definition); + } + + @Test + void testEmptyTableDefinition() { + try { + TableParquetWriterOptions.builder() + .tableDefinition(TableDefinition.of()) + .build(); + failBecauseExceptionWasNotThrown(IllegalArgumentException.class); + } catch (IllegalArgumentException e) { + assertThat(e).hasMessageContaining("table definition"); + } + } + + @Test + void testSetCompressionCodecName() { + assertThat(instructions() + .compressionCodecName("GZIP") + .build() + .compressionCodecName()) + .isEqualTo("GZIP"); + } + + @Test + void testSetMaximumDictionaryKeys() { + assertThat(instructions() + .maximumDictionaryKeys(100) + .build() + .maximumDictionaryKeys()) + .isEqualTo(100); + } + + @Test + void testSetMaximumDictionarySize() { + assertThat(instructions() + .maximumDictionarySize(100) + .build() + .maximumDictionarySize()) + .isEqualTo(100); + } + + @Test + void testSetTargetPageSize() { + assertThat(instructions() + .targetPageSize(1 << 20) + .build() + .targetPageSize()) + .isEqualTo(1 << 20); + } + + @Test + void testMinMaximumDictionaryKeys() { + + try { + instructions() + .maximumDictionaryKeys(-1) + .build(); + failBecauseExceptionWasNotThrown(IllegalArgumentException.class); + } catch (IllegalArgumentException e) { + assertThat(e).hasMessageContaining("maximumDictionaryKeys"); + } + } + + @Test + void testMinMaximumDictionarySize() { + try { + instructions() + .maximumDictionarySize(-1) + .build(); + failBecauseExceptionWasNotThrown(IllegalArgumentException.class); + } catch (IllegalArgumentException e) { + assertThat(e).hasMessageContaining("maximumDictionarySize"); + } + } + + @Test + void testMinTargetPageSize() { + try { + instructions() + .targetPageSize(1024) + .build(); + failBecauseExceptionWasNotThrown(IllegalArgumentException.class); + } catch (IllegalArgumentException e) { + assertThat(e).hasMessageContaining("targetPageSize"); + } + } + + @Test + void toParquetInstructionTest() { + final TableParquetWriterOptions writeInstructions = instructions() + .compressionCodecName("GZIP") + .maximumDictionaryKeys(100) + .maximumDictionarySize(200) + .targetPageSize(1 << 20) + .build(); + final TableDefinition definition = TableDefinition.of( + ColumnDefinition.ofInt("PC1").withPartitioning(), + ColumnDefinition.ofInt("PC2").withPartitioning(), + ColumnDefinition.ofLong("I")); + final Map fieldIdToName = Map.of(2, "field2", 3, "field3"); + final ParquetInstructions parquetInstructions = writeInstructions.toParquetInstructions( + null, definition, fieldIdToName); + + assertThat(parquetInstructions.getCompressionCodecName()).isEqualTo("GZIP"); + assertThat(parquetInstructions.getMaximumDictionaryKeys()).isEqualTo(100); + assertThat(parquetInstructions.getMaximumDictionarySize()).isEqualTo(200); + assertThat(parquetInstructions.getTargetPageSize()).isEqualTo(1 << 20); + assertThat(parquetInstructions.getFieldId("field1")).isEmpty(); + assertThat(parquetInstructions.getFieldId("field2")).hasValue(2); + assertThat(parquetInstructions.getFieldId("field3")).hasValue(3); + assertThat(parquetInstructions.onWriteCompleted()).isEmpty(); + assertThat(parquetInstructions.getTableDefinition()).hasValue(definition); + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/base/IcebergUtils.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/base/IcebergUtils.java new file mode 100644 index 00000000000..925c7a3a923 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/base/IcebergUtils.java @@ -0,0 +1,320 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.base; + +import io.deephaven.base.Pair; +import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.context.QueryScope; +import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.table.impl.locations.TableDataException; +import io.deephaven.iceberg.util.IcebergReadInstructions; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.ManifestContent; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.PartitionData; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.NamespaceNotEmptyException; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.math.BigDecimal; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; + +public final class IcebergUtils { + + private static final Map, Type> DH_TO_ICEBERG_TYPE_MAP = new HashMap<>(); + + static { + DH_TO_ICEBERG_TYPE_MAP.put(Boolean.class, Types.BooleanType.get()); + DH_TO_ICEBERG_TYPE_MAP.put(double.class, Types.DoubleType.get()); + DH_TO_ICEBERG_TYPE_MAP.put(float.class, Types.FloatType.get()); + DH_TO_ICEBERG_TYPE_MAP.put(int.class, Types.IntegerType.get()); + DH_TO_ICEBERG_TYPE_MAP.put(long.class, Types.LongType.get()); + DH_TO_ICEBERG_TYPE_MAP.put(String.class, Types.StringType.get()); + DH_TO_ICEBERG_TYPE_MAP.put(Instant.class, Types.TimestampType.withZone()); + DH_TO_ICEBERG_TYPE_MAP.put(LocalDateTime.class, Types.TimestampType.withoutZone()); + DH_TO_ICEBERG_TYPE_MAP.put(LocalDate.class, Types.DateType.get()); + DH_TO_ICEBERG_TYPE_MAP.put(LocalTime.class, Types.TimeType.get()); + DH_TO_ICEBERG_TYPE_MAP.put(byte[].class, Types.BinaryType.get()); + // TODO (deephaven-core#6327) Add support for more types like ZonedDateTime, Big Decimals, and Lists + } + + /** + * Get a stream of all {@link DataFile} objects from the given {@link Table} and {@link Snapshot}. + * + * @param table The {@link Table} to retrieve data files for. + * @param snapshot The {@link Snapshot} to retrieve data files from. + * + * @return A stream of {@link DataFile} objects. + */ + public static Stream allDataFiles(@NotNull final Table table, @NotNull final Snapshot snapshot) { + return allManifestFiles(table, snapshot) + .map(manifestFile -> ManifestFiles.read(manifestFile, table.io())) + .flatMap(IcebergUtils::toStream); + } + + /** + * Get a stream of all {@link ManifestFile} objects from the given {@link Table} and {@link Snapshot}. + * + * @param table The {@link Table} to retrieve manifest files for. + * @param snapshot The {@link Snapshot} to retrieve manifest files from. + * + * @return A stream of {@link ManifestFile} objects. + */ + public static Stream allManifestFiles(@NotNull final Table table, @NotNull final Snapshot snapshot) { + return allManifests(table, snapshot).stream() + .peek(manifestFile -> { + 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())); + } + }); + } + + /** + * Retrieves a {@link List} of manifest files from the given {@link Table} and {@link Snapshot}. + * + * @param table The {@link Table} to retrieve manifest files for. + * @param snapshot The {@link Snapshot} to retrieve manifest files from. + * + * @return A {@link List} of {@link ManifestFile} objects. + * @throws TableDataException if there is an error retrieving the manifest files. + */ + static List allManifests(@NotNull final Table table, @NotNull final Snapshot snapshot) { + try { + return snapshot.allManifests(table.io()); + } catch (final RuntimeException e) { + throw new TableDataException( + String.format("%s:%d - error retrieving manifest files", table, snapshot.snapshotId()), e); + } + } + + /** + * Convert a {@link org.apache.iceberg.io.CloseableIterable} to a {@link Stream} that will close the iterable when + * the stream is closed. + */ + public static Stream toStream(final org.apache.iceberg.io.CloseableIterable iterable) { + return StreamSupport.stream(iterable.spliterator(), false).onClose(() -> { + try { + iterable.close(); + } catch (final IOException e) { + throw new UncheckedIOException(e); + } + }); + } + + /** + * Convert an Iceberg data type to a Deephaven type. + * + * @param icebergType The Iceberg data type to be converted. + * @return The converted Deephaven type. + */ + public static io.deephaven.qst.type.Type convertToDHType(@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; + if (timestampType == Types.TimestampType.withZone()) { + return io.deephaven.qst.type.Type.find(Instant.class); + } + return io.deephaven.qst.type.Type.find(LocalDateTime.class); + case DATE: + return io.deephaven.qst.type.Type.find(LocalDate.class); + case TIME: + return io.deephaven.qst.type.Type.find(LocalTime.class); + case DECIMAL: + return io.deephaven.qst.type.Type.find(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()); + } + } + + /** + * Convert a Deephaven type to an Iceberg type. + * + * @param columnType The Deephaven type to be converted. + * @return The converted Iceberg type. + */ + public static Type convertToIcebergType(final Class columnType) { + final Type icebergType = DH_TO_ICEBERG_TYPE_MAP.get(columnType); + if (icebergType != null) { + return icebergType; + } else { + throw new TableDataException("Unsupported deephaven column type " + columnType.getName()); + } + } + + /** + * Used to hold a {@link Schema}, {@link PartitionSpec} and {@link IcebergReadInstructions} together. + */ + public static final class SpecAndSchema { + public final Schema schema; + public final PartitionSpec partitionSpec; + public final IcebergReadInstructions readInstructions; + + public SpecAndSchema( + @NotNull final Schema schema, + @NotNull final PartitionSpec partitionSpec, + @Nullable final IcebergReadInstructions readInstructions) { + this.schema = schema; + this.partitionSpec = partitionSpec; + this.readInstructions = readInstructions; + } + } + + /** + * Create {@link PartitionSpec} and {@link Schema} from a {@link TableDefinition}. + * + * @return A {@link SpecAndSchema} object containing the partition spec and schema, and {@code null} for read + * instructions. + */ + public static SpecAndSchema createSpecAndSchema(@NotNull final TableDefinition tableDefinition) { + final Collection partitioningColumnNames = new ArrayList<>(); + final List fields = new ArrayList<>(); + int fieldID = 1; // Iceberg field IDs start from 1 + + // Create the schema first and use it to build the partition spec + for (final ColumnDefinition columnDefinition : tableDefinition.getColumns()) { + final String dhColumnName = columnDefinition.getName(); + final Type icebergType = convertToIcebergType(columnDefinition.getDataType()); + fields.add(Types.NestedField.optional(fieldID, dhColumnName, icebergType)); + if (columnDefinition.isPartitioning()) { + partitioningColumnNames.add(dhColumnName); + } + fieldID++; + } + final Schema schema = new Schema(fields); + + final PartitionSpec partitionSpec = createPartitionSpec(schema, partitioningColumnNames); + return new SpecAndSchema(schema, partitionSpec, null); + } + + public static PartitionSpec createPartitionSpec( + @NotNull final Schema schema, + @NotNull final Iterable partitionColumnNames) { + final PartitionSpec.Builder partitionSpecBuilder = PartitionSpec.builderFor(schema); + for (final String partitioningColumnName : partitionColumnNames) { + partitionSpecBuilder.identity(partitioningColumnName); + } + return partitionSpecBuilder.build(); + } + + public static boolean createNamespaceIfNotExists( + @NotNull final Catalog catalog, + @NotNull final Namespace namespace) { + if (catalog instanceof SupportsNamespaces) { + final SupportsNamespaces nsCatalog = (SupportsNamespaces) catalog; + try { + nsCatalog.createNamespace(namespace); + return true; + } catch (final AlreadyExistsException | UnsupportedOperationException e) { + return false; + } + } + return false; + } + + public static boolean dropNamespaceIfExists( + @NotNull final Catalog catalog, + @NotNull final Namespace namespace) { + if (catalog instanceof SupportsNamespaces) { + final SupportsNamespaces nsCatalog = (SupportsNamespaces) catalog; + try { + return nsCatalog.dropNamespace(namespace); + } catch (final NamespaceNotEmptyException e) { + return false; + } + } + return false; + } + + /** + * Check that all required fields are present in the table definition + */ + public static void verifyRequiredFields(final Schema tableSchema, final TableDefinition tableDefinition) { + final List columnNames = tableDefinition.getColumnNames(); + for (final Types.NestedField field : tableSchema.columns()) { + if (field.isRequired() && !columnNames.contains(field.name())) { + // TODO (deephaven-core#6343): Add check for writeDefault() not set for required fields + throw new IllegalArgumentException("Field " + field + " is required in the table schema, but is not " + + "present in the table definition, table schema " + tableSchema + ", tableDefinition " + + tableDefinition); + } + } + } + + /** + * Check that all the partitioning columns from the partition spec are present in the Table Definition. + */ + public static void verifyPartitioningColumns( + final PartitionSpec tablePartitionSpec, + final TableDefinition tableDefinition) { + final List partitioningColumnNamesFromDefinition = tableDefinition.getColumnStream() + .filter(ColumnDefinition::isPartitioning) + .map(ColumnDefinition::getName) + .collect(Collectors.toList()); + final List partitionFieldsFromSchema = tablePartitionSpec.fields(); + if (partitionFieldsFromSchema.size() != partitioningColumnNamesFromDefinition.size()) { + throw new IllegalArgumentException("Partition spec contains " + partitionFieldsFromSchema.size() + + " fields, but the table definition contains " + partitioningColumnNamesFromDefinition.size() + + " fields, partition spec " + tablePartitionSpec + ", table definition " + tableDefinition); + } + for (int colIdx = 0; colIdx < partitionFieldsFromSchema.size(); colIdx += 1) { + final PartitionField partitionField = partitionFieldsFromSchema.get(colIdx); + if (!partitioningColumnNamesFromDefinition.get(colIdx).equals(partitionField.name())) { + throw new IllegalArgumentException("Partitioning column " + partitionField.name() + " is not present " + + "in the table definition at idx " + colIdx + ", table definition " + tableDefinition + + ", partition spec " + tablePartitionSpec); + } + } + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java index 9cab712df7f..d711119641b 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java @@ -7,6 +7,7 @@ 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.base.IcebergUtils; import io.deephaven.iceberg.location.IcebergTableLocationKey; import io.deephaven.iceberg.location.IcebergTableParquetLocationKey; import io.deephaven.iceberg.relative.RelativeFileIO; @@ -26,8 +27,12 @@ import java.net.URI; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.UUID; import java.util.function.Consumer; +import java.util.stream.Stream; + +import static io.deephaven.iceberg.base.IcebergUtils.allManifestFiles; public abstract class IcebergBaseLayout implements TableLocationKeyFinder { /** @@ -176,32 +181,23 @@ public synchronized void findKeys(@NotNull final Consumer manifestFiles = snapshot.allManifests(table.io()); - 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, table.io())) { - for (final DataFile dataFile : reader) { - final URI fileUri = dataFileUri(table, dataFile); - if (!uriScheme.equals(fileUri.getScheme())) { - throw new TableDataException(String.format( - "%s:%d - multiple URI schemes are not currently supported. uriScheme=%s, fileUri=%s", - table, snapshot.snapshotId(), uriScheme, fileUri)); - } - final IcebergTableLocationKey locationKey = keyFromDataFile(manifestFile, dataFile, fileUri); - if (locationKey != null) { - locationKeyObserver.accept(locationKey); - } - } - } - } - } catch (final Exception e) { + try (final Stream manifestFiles = allManifestFiles(table, snapshot)) { + manifestFiles.forEach(manifestFile -> { + final ManifestReader reader = ManifestFiles.read(manifestFile, table.io()); + IcebergUtils.toStream(reader) + .map(dataFile -> { + final URI fileUri = dataFileUri(table, dataFile); + if (!uriScheme.equals(fileUri.getScheme())) { + throw new TableDataException(String.format( + "%s:%d - multiple URI schemes are not currently supported. uriScheme=%s, " + + "fileUri=%s", + table, snapshot.snapshotId(), uriScheme, fileUri)); + } + return keyFromDataFile(manifestFile, dataFile, fileUri); + }) + .forEach(locationKeyObserver); + }); + } catch (final RuntimeException e) { throw new TableDataException( String.format("%s:%d - error finding Iceberg locations", tableAdapter, snapshot.snapshotId()), e); } 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 index a096d21570c..2c0206c0c89 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java @@ -7,19 +7,27 @@ import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.engine.table.impl.sources.InMemoryColumnSource; +import io.deephaven.iceberg.base.IcebergUtils; import io.deephaven.iceberg.internal.DataInstructionsProviderLoader; import io.deephaven.iceberg.internal.DataInstructionsProviderPlugin; import io.deephaven.util.annotations.VisibleForTesting; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.TableProperties; 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.AlreadyExistsException; import org.jetbrains.annotations.NotNull; import org.apache.iceberg.rest.RESTCatalog; import org.apache.iceberg.rest.ResourcePaths; import java.util.*; +import static io.deephaven.iceberg.base.IcebergUtils.createNamespaceIfNotExists; +import static io.deephaven.iceberg.base.IcebergUtils.dropNamespaceIfExists; + public class IcebergCatalogAdapter { @VisibleForTesting @@ -86,7 +94,6 @@ static IcebergCatalogAdapter of(RESTCatalog restCatalog) { dataInstructionsProvider = DataInstructionsProviderLoader.create(Map.copyOf(properties)); } - /** * 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)}. @@ -249,8 +256,65 @@ public IcebergTableAdapter loadTable(@NotNull final TableIdentifier tableIdentif /** * Returns the underlying Iceberg {@link Catalog catalog} used by this adapter. */ - @SuppressWarnings("unused") public Catalog catalog() { return catalog; } + + /** + * Create a new Iceberg table in the catalog with the given table identifier and definition. + *

+ * All columns of type {@link ColumnDefinition.ColumnType#Partitioning partitioning} will be used to create the + * partition spec for the table. + * + * @param tableIdentifier The identifier string of the new table. + * @param definition The {@link TableDefinition} of the new table. + * @return The {@link IcebergTableAdapter table adapter} for the new Iceberg table. + * @throws AlreadyExistsException if the table already exists + */ + public IcebergTableAdapter createTable( + @NotNull final String tableIdentifier, + @NotNull final TableDefinition definition) { + return createTable(TableIdentifier.parse(tableIdentifier), definition); + } + + /** + * Create a new Iceberg table in the catalog with the given table identifier and definition. + *

+ * All columns of type {@link ColumnDefinition.ColumnType#Partitioning partitioning} will be used to create the + * partition spec for the table. + * + * @param tableIdentifier The identifier of the new table. + * @param definition The {@link TableDefinition} of the new table. + * @return The {@link IcebergTableAdapter table adapter} for the new Iceberg table. + * @throws AlreadyExistsException if the table already exists + */ + public IcebergTableAdapter createTable( + @NotNull final TableIdentifier tableIdentifier, + @NotNull final TableDefinition definition) { + final IcebergUtils.SpecAndSchema specAndSchema = IcebergUtils.createSpecAndSchema(definition); + return createTable(tableIdentifier, specAndSchema.schema, specAndSchema.partitionSpec); + } + + private IcebergTableAdapter createTable( + @NotNull final TableIdentifier tableIdentifier, + @NotNull final Schema schema, + @NotNull final PartitionSpec partitionSpec) { + final boolean newNamespaceCreated = createNamespaceIfNotExists(catalog, tableIdentifier.namespace()); + try { + final org.apache.iceberg.Table table = + catalog.createTable(tableIdentifier, schema, partitionSpec, + Map.of(TableProperties.DEFAULT_FILE_FORMAT, TableProperties.DEFAULT_FILE_FORMAT_DEFAULT)); + return new IcebergTableAdapter(catalog, tableIdentifier, table, dataInstructionsProvider); + } catch (final Throwable throwable) { + if (newNamespaceCreated) { + // Delete it to avoid leaving a partial namespace in the catalog + try { + dropNamespaceIfExists(catalog, tableIdentifier.namespace()); + } catch (final RuntimeException dropException) { + throwable.addSuppressed(dropException); + } + } + throw throwable; + } + } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergReadInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergReadInstructions.java index 82271590900..fa91cc9c2d3 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergReadInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergReadInstructions.java @@ -22,9 +22,8 @@ public abstract class IcebergReadInstructions { /** * The default {@link IcebergReadInstructions} to use when reading Iceberg data files. Providing this will use - * system defaults for cloud provider-specific parameters + * system defaults for cloud provider-specific parameters. */ - @SuppressWarnings("unused") public static final IcebergReadInstructions DEFAULT = builder().build(); public static Builder builder() { diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java index 7464ece4988..dff4f620a25 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java @@ -19,6 +19,7 @@ import io.deephaven.engine.table.impl.sources.regioned.RegionedTableComponentFactoryImpl; import io.deephaven.engine.updategraph.UpdateSourceRegistrar; import io.deephaven.engine.util.TableTools; +import io.deephaven.iceberg.base.IcebergUtils.SpecAndSchema; import io.deephaven.iceberg.internal.DataInstructionsProviderLoader; import io.deephaven.iceberg.layout.*; import io.deephaven.iceberg.location.IcebergTableLocationFactory; @@ -38,10 +39,11 @@ import org.jetbrains.annotations.Nullable; import java.time.Instant; -import java.time.LocalDateTime; import java.util.*; import java.util.stream.Collectors; +import static io.deephaven.iceberg.base.IcebergUtils.convertToDHType; + /** * This class manages an Iceberg {@link org.apache.iceberg.Table table} and provides methods to interact with it. */ @@ -266,24 +268,6 @@ public Snapshot getSnapshot(@NotNull final IcebergReadInstructions readInstructi return null; } - /** - * Used to hold return value for {@link #getSpecAndSchema(IcebergReadInstructions)}. - */ - private static final class SpecAndSchema { - private final Schema schema; - private final PartitionSpec partitionSpec; - private final IcebergReadInstructions readInstructions; - - private SpecAndSchema( - @NotNull final Schema schema, - @NotNull final PartitionSpec partitionSpec, - @NotNull final IcebergReadInstructions readInstructions) { - this.schema = schema; - this.partitionSpec = partitionSpec; - this.readInstructions = readInstructions; - } - } - /** * Retrieve the schema and partition spec for the table based on the provided read instructions. Also, populate the * read instructions with the requested snapshot, or the latest snapshot if none is requested. @@ -552,7 +536,7 @@ private static TableDefinition fromSchema( continue; } final Type type = field.type(); - final io.deephaven.qst.type.Type qstType = convertPrimitiveType(type); + final io.deephaven.qst.type.Type qstType = convertToDHType(type); final ColumnDefinition column; if (partitionNames.contains(name)) { column = ColumnDefinition.of(name, qstType).withPartitioning(); @@ -590,46 +574,16 @@ private static TableDefinition fromSchema( } /** - * Convert an Iceberg data type to a Deephaven type. + * Create a new {@link IcebergTableWriter} for this Iceberg table using the provided {@link TableWriterOptions}. + *

+ * This method will perform schema validation to ensure that the provided + * {@link TableWriterOptions#tableDefinition()} is compatible with the Iceberg table schema. All further writes + * performed by the returned writer will not be validated against the table's schema, and thus will be faster. * - * @param icebergType The Iceberg data type to be converted. - * @return The converted Deephaven type. + * @param tableWriterOptions The options to configure the table writer. + * @return A new instance of {@link IcebergTableWriter} configured with the provided options. */ - 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()); - } + public IcebergTableWriter tableWriter(final TableWriterOptions tableWriterOptions) { + return new IcebergTableWriter(tableWriterOptions, this); } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableWriter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableWriter.java new file mode 100644 index 00000000000..1f4fa3597af --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableWriter.java @@ -0,0 +1,525 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.util; + +import io.deephaven.base.Pair; +import io.deephaven.base.verify.Require; +import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.context.QueryScope; +import io.deephaven.engine.context.StandaloneQueryScope; +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.parquet.table.CompletedParquetWrite; +import io.deephaven.parquet.table.ParquetInstructions; +import io.deephaven.parquet.table.ParquetTools; +import io.deephaven.iceberg.util.SchemaProviderInternal.SchemaProviderImpl; +import io.deephaven.util.SafeCloseable; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.PartitionData; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.Transaction; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.mapping.MappedField; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.mapping.NameMappingParser; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.jetbrains.annotations.NotNull; + +import java.time.Instant; +import java.time.LocalDate; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; + +import static io.deephaven.iceberg.base.IcebergUtils.verifyPartitioningColumns; +import static io.deephaven.iceberg.base.IcebergUtils.verifyRequiredFields; + +/** + * This class is responsible for writing Deephaven tables to an Iceberg table. Each instance of this class is associated + * with a single {@link IcebergTableAdapter} and can be used to write multiple Deephaven tables to this Iceberg table. + */ +public class IcebergTableWriter { + + /** + * The options used to configure the behavior of this writer instance. + */ + private final TableParquetWriterOptions tableWriterOptions; + + /** + * The Iceberg table which will be written to by this instance. + */ + private final org.apache.iceberg.Table table; + + /** + * Store the partition spec of the Iceberg table at the time of creation of this writer instance and use it for all + * writes, so that even if the table spec, the writer will still work. + */ + private final PartitionSpec tableSpec; + + /** + * The table definition used for all writes by this writer instance. + */ + private final TableDefinition tableDefinition; + + /** + * The table definition consisting of non-partitioning columns from {@link #tableDefinition}. All tables written by + * this writer are expected to have a compatible definition with this. + */ + private final TableDefinition nonPartitioningTableDefinition; + + /** + * The schema to use when in conjunction with the {@link #fieldIdToColumnName} to map Deephaven columns from + * {@link #tableDefinition} to Iceberg columns. + */ + private final Schema userSchema; + + /** + * Mapping from Iceberg field IDs to Deephaven column names, populated inside the parquet file. + */ + private final Map fieldIdToColumnName; + + /** + * The factory to create new output file locations for writing data files. + */ + private final OutputFileFactory outputFileFactory; + + /** + * Characters to be used for generating random variable names of length {@link #VARIABLE_NAME_LENGTH}. + */ + private static final String CHARACTERS = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz"; + private static final int VARIABLE_NAME_LENGTH = 6; + + IcebergTableWriter( + final TableWriterOptions tableWriterOptions, + final IcebergTableAdapter tableAdapter) { + this.tableWriterOptions = verifyWriterOptions(tableWriterOptions); + this.table = tableAdapter.icebergTable(); + + this.tableSpec = table.spec(); + + this.tableDefinition = tableWriterOptions.tableDefinition(); + this.nonPartitioningTableDefinition = nonPartitioningTableDefinition(tableDefinition); + verifyRequiredFields(table.schema(), tableDefinition); + verifyPartitioningColumns(tableSpec, tableDefinition); + + this.userSchema = ((SchemaProviderImpl) tableWriterOptions.schemaProvider()).getSchema(table); + verifyFieldIdsInSchema(tableWriterOptions.fieldIdToColumnName().keySet(), userSchema); + + // Create a copy of the fieldIdToColumnName map since we might need to add new entries for columns which are not + // provided by the user. + this.fieldIdToColumnName = new HashMap<>(tableWriterOptions.fieldIdToColumnName()); + addFieldIdsForAllColumns(); + + outputFileFactory = OutputFileFactory.builderFor(table, 0, 0) + .format(FileFormat.PARQUET) + .build(); + } + + private static TableParquetWriterOptions verifyWriterOptions( + @NotNull final TableWriterOptions tableWriterOptions) { + // We ony support writing to Parquet files + if (!(tableWriterOptions instanceof TableParquetWriterOptions)) { + throw new IllegalArgumentException( + "Unsupported options of class " + tableWriterOptions.getClass() + " for" + + " writing Iceberg table, expected: " + TableParquetWriterOptions.class); + } + return (TableParquetWriterOptions) tableWriterOptions; + } + + /** + * Return a {@link TableDefinition} which contains only the non-partitioning columns from the provided table + * definition. + */ + private static TableDefinition nonPartitioningTableDefinition( + @NotNull final TableDefinition tableDefinition) { + final Collection> nonPartitioningColumns = new ArrayList<>(); + for (final ColumnDefinition columnDefinition : tableDefinition.getColumns()) { + if (!columnDefinition.isPartitioning()) { + nonPartitioningColumns.add(columnDefinition); + } + } + return TableDefinition.of(nonPartitioningColumns); + } + + /** + * Check that all the field IDs are present in the schema. + */ + private static void verifyFieldIdsInSchema(final Collection fieldIds, final Schema schema) { + if (!fieldIds.isEmpty()) { + for (final Integer fieldId : fieldIds) { + if (schema.findField(fieldId) == null) { + throw new IllegalArgumentException("Column corresponding to field ID " + fieldId + " not " + + "found in schema, available columns in schema are: " + schema.columns()); + } + } + } + } + + /** + * Populate the {@link #fieldIdToColumnName} map for all the columns in the {@link #tableDefinition} and do + * additional checks to ensure that the table definition is compatible with schema provided by user. + */ + private void addFieldIdsForAllColumns() { + final Map dhColumnNameToFieldId = tableWriterOptions.dhColumnNameToFieldId(); + Map nameMappingDefault = null; // Lazily initialized + for (final ColumnDefinition columnDefinition : tableDefinition.getColumns()) { + final String columnName = columnDefinition.getName(); + + // We are done if we already have the mapping between column name and field ID + if (dhColumnNameToFieldId.containsKey(columnName)) { + continue; + } + + // To be populated by the end of this block for each column, else throw an exception + Integer fieldId = null; + Types.NestedField nestedField; + + // Check in the schema.name_mapping.default map + if (nameMappingDefault == null) { + nameMappingDefault = readNameMappingDefault(); + } + fieldId = nameMappingDefault.get(columnName); + if (fieldId != null) { + nestedField = userSchema.findField(fieldId); + if (nestedField == null) { + throw new IllegalArgumentException("Field ID " + fieldId + " extracted for " + + "column " + columnName + " from the schema.name_mapping map not found in schema " + + userSchema); + } + } + + // Directly lookup in the user provided schema using column name + if (fieldId == null) { + nestedField = userSchema.findField(columnName); + if (nestedField != null) { + fieldId = nestedField.fieldId(); + } + } + + if (fieldId == null) { + throw new IllegalArgumentException("Column " + columnName + " not found in the schema or " + + "the name mapping for the table"); + } + + fieldIdToColumnName.put(fieldId, columnName); + } + } + + /** + * Build the mapping from column names to field IDs on demand using the + * {@value TableProperties#DEFAULT_NAME_MAPPING} map. + *

+ * Return an empty map if the table metadata is null or the mapping is not present in the table metadata. + */ + private Map readNameMappingDefault() { + final TableMetadata tableMetadata; + if (table instanceof HasTableOperations) { + tableMetadata = ((HasTableOperations) table).operations().current(); + } else { + // TableMetadata is not available, so nothing to add to the map + return Map.of(); + } + final String nameMappingJson = tableMetadata.property(TableProperties.DEFAULT_NAME_MAPPING, null); + if (nameMappingJson == null) { + return Map.of(); + } + // Iterate over all mapped fields and build a reverse map from column name to field ID + final Map nameMappingDefault = new HashMap<>(); + final NameMapping nameMapping = NameMappingParser.fromJson(nameMappingJson); + for (final MappedField field : nameMapping.asMappedFields().fields()) { + final Integer fieldId = field.id(); + for (final String name : field.names()) { + nameMappingDefault.put(name, fieldId); + } + } + return nameMappingDefault; + } + + /** + * Append the provided Deephaven {@link IcebergWriteInstructions#tables()} as new partitions to the existing Iceberg + * table in a single snapshot. This method will not perform any compatibility checks between the existing schema and + * the provided Deephaven tables. + * + * @param writeInstructions The instructions for customizations while writing. + */ + public void append(@NotNull final IcebergWriteInstructions writeInstructions) { + final List dataFilesWritten = writeDataFiles(writeInstructions); + commit(dataFilesWritten); + } + + /** + * Writes data from Deephaven {@link IcebergWriteInstructions#tables()} to an Iceberg table without creating a new + * snapshot. This method returns a list of data files that were written. Users can use this list to create a + * transaction/snapshot if needed. This method will not perform any compatibility checks between the existing schema + * and the provided Deephaven tables. + * + * @param writeInstructions The instructions for customizations while writing. + */ + public List writeDataFiles(@NotNull final IcebergWriteInstructions writeInstructions) { + verifyCompatible(writeInstructions.tables(), nonPartitioningTableDefinition); + final List partitionPaths = writeInstructions.partitionPaths(); + verifyPartitionPaths(tableSpec, partitionPaths); + final List partitionData; + final List parquetFileInfo; + // Start a new query scope to avoid polluting the existing query scope with new parameters added for + // partitioning columns + try (final SafeCloseable _ignore = + ExecutionContext.getContext().withQueryScope(new StandaloneQueryScope()).open()) { + final Pair, List> ret = partitionDataFromPaths(tableSpec, partitionPaths); + partitionData = ret.getFirst(); + final List dhTableUpdateStrings = ret.getSecond(); + parquetFileInfo = writeParquet(partitionData, dhTableUpdateStrings, writeInstructions); + } + return dataFilesFromParquet(parquetFileInfo, partitionData); + } + + /** + * Verify that all the tables are compatible with the provided table definition. + */ + private static void verifyCompatible( + @NotNull final Iterable tables, + @NotNull final TableDefinition expectedDefinition) { + for (final Table table : tables) { + expectedDefinition.checkMutualCompatibility(table.getDefinition()); + } + } + + private static void verifyPartitionPaths( + final PartitionSpec partitionSpec, + final Collection partitionPaths) { + if (partitionSpec.isPartitioned() && partitionPaths.isEmpty()) { + throw new IllegalArgumentException("Cannot write data to a partitioned table without partition paths."); + } + if (!partitionSpec.isPartitioned() && !partitionPaths.isEmpty()) { + throw new IllegalArgumentException("Cannot write data to an un-partitioned table with partition paths."); + } + } + + /** + * Creates a list of {@link PartitionData} and corresponding update strings for Deephaven tables from partition + * paths and spec. Also, validates that the partition paths are compatible with the provided partition spec. + * + * @param partitionSpec The partition spec to use for validation. + * @param partitionPaths The list of partition paths to process. + * @return A pair containing a list of PartitionData objects and a list of update strings for Deephaven tables. + * @throws IllegalArgumentException if the partition paths are not compatible with the partition spec. + * + * @implNote Check implementations of {@link DataFiles#data} and {@link Conversions#fromPartitionString} for more + * details on how partition paths should be parsed, how each type of value is parsed from a string and + * what types are allowed for partitioning columns. + */ + private static Pair, List> partitionDataFromPaths( + final PartitionSpec partitionSpec, + final Collection partitionPaths) { + final List partitionDataList = new ArrayList<>(partitionPaths.size()); + final List dhTableUpdateStringList = new ArrayList<>(partitionPaths.size()); + final int numPartitioningFields = partitionSpec.fields().size(); + final QueryScope queryScope = ExecutionContext.getContext().getQueryScope(); + for (final String partitionPath : partitionPaths) { + final String[] dhTableUpdateString = new String[numPartitioningFields]; + try { + final String[] partitions = partitionPath.split("/", -1); + if (partitions.length != numPartitioningFields) { + throw new IllegalArgumentException("Expecting " + numPartitioningFields + " number of fields, " + + "found " + partitions.length); + } + final PartitionData partitionData = new PartitionData(partitionSpec.partitionType()); + for (int colIdx = 0; colIdx < partitions.length; colIdx += 1) { + final String[] parts = partitions[colIdx].split("=", 2); + if (parts.length != 2) { + throw new IllegalArgumentException("Expecting key=value format, found " + partitions[colIdx]); + } + final PartitionField field = partitionSpec.fields().get(colIdx); + if (!field.name().equals(parts[0])) { + throw new IllegalArgumentException("Expecting field name " + field.name() + " at idx " + + colIdx + ", found " + parts[0]); + } + final Type type = partitionData.getType(colIdx); + dhTableUpdateString[colIdx] = getTableUpdateString(field.name(), type, parts[1], queryScope); + partitionData.set(colIdx, Conversions.fromPartitionString(partitionData.getType(colIdx), parts[1])); + } + } catch (final Exception e) { + throw new IllegalArgumentException("Failed to parse partition path: " + partitionPath + " using" + + " partition spec " + partitionSpec + ", check cause for more details ", e); + } + dhTableUpdateStringList.add(dhTableUpdateString); + partitionDataList.add(DataFiles.data(partitionSpec, partitionPath)); + } + return new Pair<>(partitionDataList, dhTableUpdateStringList); + } + + /** + * This method would convert a partitioning column info to a string which can be used in + * {@link io.deephaven.engine.table.Table#updateView(Collection) Table#updateView} method. For example, if the + * partitioning column of name "partitioningColumnName" if of type {@link Types.TimestampType} and the value is + * "2021-01-01T00:00:00Z", then this method would: + *
    + *
  • Add a new parameter to the query scope with a random name and value as {@link Instant} parsed from the string + * "2021-01-01T00:00:00Z"
  • + *
  • Return the string "partitioningColumnName = randomName"
  • + *
+ * + * @param colName The name of the partitioning column + * @param colType The type of the partitioning column + * @param value The value of the partitioning column + * @param queryScope The query scope to add the parameter to + */ + private static String getTableUpdateString( + @NotNull final String colName, + @NotNull final Type colType, + @NotNull final String value, + @NotNull final QueryScope queryScope) { + // Randomly generated name to be added to the query scope for each value to avoid repeated casts + // TODO(deephaven-core#6418): Find a better way to handle these table updates instead of using query scope + final String paramName = generateRandomAlphabetString(VARIABLE_NAME_LENGTH); + final Type.TypeID typeId = colType.typeId(); + if (typeId == Type.TypeID.BOOLEAN) { + queryScope.putParam(paramName, Boolean.parseBoolean(value)); + } else if (typeId == Type.TypeID.DOUBLE) { + queryScope.putParam(paramName, Double.parseDouble(value)); + } else if (typeId == Type.TypeID.FLOAT) { + queryScope.putParam(paramName, Float.parseFloat(value)); + } else if (typeId == Type.TypeID.INTEGER) { + queryScope.putParam(paramName, Integer.parseInt(value)); + } else if (typeId == Type.TypeID.LONG) { + queryScope.putParam(paramName, Long.parseLong(value)); + } else if (typeId == Type.TypeID.STRING) { + queryScope.putParam(paramName, value); + } else if (typeId == Type.TypeID.DATE) { + queryScope.putParam(paramName, LocalDate.parse(value)); + } else { + // TODO (deephaven-core#6327) Add support for more partitioning types like Big Decimals + throw new TableDataException("Unsupported partitioning column type " + typeId.name()); + } + return colName + " = " + paramName; + } + + /** + * Generate a random string of length {@code length} using just alphabets. + */ + private static String generateRandomAlphabetString(final int length) { + final StringBuilder stringBuilder = new StringBuilder(); + final Random random = new Random(); + for (int i = 0; i < length; i++) { + final int index = random.nextInt(CHARACTERS.length()); + stringBuilder.append(CHARACTERS.charAt(index)); + } + return stringBuilder.toString(); + } + + @NotNull + private List writeParquet( + @NotNull final List partitionDataList, + @NotNull final List dhTableUpdateStrings, + @NotNull final IcebergWriteInstructions writeInstructions) { + final List
dhTables = writeInstructions.tables(); + final boolean isPartitioned = tableSpec.isPartitioned(); + if (isPartitioned) { + Require.eq(dhTables.size(), "dhTables.size()", + partitionDataList.size(), "partitionDataList.size()"); + Require.eq(dhTables.size(), "dhTables.size()", + dhTableUpdateStrings.size(), "dhTableUpdateStrings.size()"); + } else { + Require.eqZero(partitionDataList.size(), "partitionDataList.size()"); + Require.eqZero(dhTableUpdateStrings.size(), "dhTableUpdateStrings.size()"); + } + + // Build the parquet instructions + final List parquetFilesWritten = new ArrayList<>(dhTables.size()); + final ParquetInstructions.OnWriteCompleted onWriteCompleted = parquetFilesWritten::add; + final ParquetInstructions parquetInstructions = tableWriterOptions.toParquetInstructions( + onWriteCompleted, tableDefinition, fieldIdToColumnName); + + // Write the data to parquet files + for (int idx = 0; idx < dhTables.size(); idx++) { + Table dhTable = dhTables.get(idx); + if (dhTable.numColumns() == 0) { + // Skip writing empty tables with no columns + continue; + } + final String newDataLocation; + if (isPartitioned) { + newDataLocation = getDataLocation(partitionDataList.get(idx)); + dhTable = dhTable.updateView(dhTableUpdateStrings.get(idx)); + } else { + newDataLocation = getDataLocation(); + } + // TODO (deephaven-core#6343): Set writeDefault() values for required columns that not present in the table + ParquetTools.writeTable(dhTable, newDataLocation, parquetInstructions); + } + return parquetFilesWritten; + } + + /** + * Generate the location string for a new data file for the given partition data. + */ + private String getDataLocation(@NotNull final PartitionData partitionData) { + final EncryptedOutputFile outputFile = outputFileFactory.newOutputFile(tableSpec, partitionData); + return outputFile.encryptingOutputFile().location(); + } + + /** + * Generate the location string for a new data file for the unpartitioned table. + */ + private String getDataLocation() { + final EncryptedOutputFile outputFile = outputFileFactory.newOutputFile(); + return outputFile.encryptingOutputFile().location(); + } + + /** + * Commit the changes to the Iceberg table by creating a snapshot. + */ + private void commit( + @NotNull final Iterable dataFiles) { + final Transaction icebergTransaction = table.newTransaction(); + + // Append the new data files to the table + final AppendFiles append = icebergTransaction.newAppend(); + dataFiles.forEach(append::appendFile); + append.commit(); + + // Commit the transaction, creating new snapshot + icebergTransaction.commitTransaction(); + } + + /** + * Generate a list of {@link DataFile} objects from a list of parquet files written. + */ + private List dataFilesFromParquet( + @NotNull final List parquetFilesWritten, + @NotNull final List partitionDataList) { + final int numFiles = parquetFilesWritten.size(); + final List dataFiles = new ArrayList<>(numFiles); + final PartitionSpec partitionSpec = tableSpec; + for (int idx = 0; idx < numFiles; idx++) { + final CompletedParquetWrite completedWrite = parquetFilesWritten.get(idx); + final DataFiles.Builder dataFileBuilder = DataFiles.builder(partitionSpec) + .withPath(completedWrite.destination().toString()) + .withFormat(FileFormat.PARQUET) + .withRecordCount(completedWrite.numRows()) + .withFileSizeInBytes(completedWrite.numBytes()); + if (partitionSpec.isPartitioned()) { + dataFileBuilder.withPartition(partitionDataList.get(idx)); + } + dataFiles.add(dataFileBuilder.build()); + } + return dataFiles; + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteInstructions.java new file mode 100644 index 00000000000..12afb5ef9e4 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteInstructions.java @@ -0,0 +1,76 @@ +// +// 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.Table; +import org.immutables.value.Value; +import org.immutables.value.Value.Immutable; + +import java.util.List; + +/** + * This class provides instructions intended for writing deephaven tables as partitions to Iceberg tables. + */ +@Immutable +@BuildableStyle +public abstract class IcebergWriteInstructions { + + public static Builder builder() { + return ImmutableIcebergWriteInstructions.builder(); + } + + /** + * The Deephaven tables to be written. + *

+ * All tables must have the same table definition as definition for non-partitioning columns specified in the + * {@link IcebergTableWriter}. For example, if an iceberg table is partitioned by "year" and "month" and has a + * non-partitioning column "data," then {@link IcebergTableWriter} should be configured with a definition that + * includes all three columns: "year," "month," and "data." But, the tables provided here should only include the + * non-partitioning column, such as "data." + */ + public abstract List

tables(); + + /** + * The partition paths where each table will be written. For example, if the table is partitioned by "year" and + * "month", the partition path could be "year=2021/month=01". + *

+ * If writing to a partitioned iceberg table, users must provide partition path for each table in {@link #tables()} + * in the same order. Else, this should be an empty list. + */ + public abstract List partitionPaths(); + + // @formatter:off + public interface Builder { + // @formatter:on + Builder addTables(Table element); + + Builder addTables(Table... elements); + + Builder addAllTables(Iterable elements); + + Builder addPartitionPaths(String element); + + Builder addPartitionPaths(String... elements); + + Builder addAllPartitionPaths(Iterable elements); + + IcebergWriteInstructions build(); + } + + @Value.Check + final void countCheckTables() { + if (tables().isEmpty()) { + throw new IllegalArgumentException("At least one table must be provided"); + } + } + + @Value.Check + final void countCheckPartitionPaths() { + if (!partitionPaths().isEmpty() && partitionPaths().size() != tables().size()) { + throw new IllegalArgumentException("Partition path must be provided for each table, partitionPaths.size()=" + + partitionPaths().size() + ", tables.size()=" + tables().size()); + } + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/SchemaProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/SchemaProvider.java new file mode 100644 index 00000000000..579c849bfe4 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/SchemaProvider.java @@ -0,0 +1,33 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.util; + +import org.apache.iceberg.Schema; + +/** + * A specification for extracting the schema from a table. + */ +public interface SchemaProvider { + + // Static factory methods for creating SchemaProvider instances + static SchemaProvider fromCurrent() { + return new SchemaProviderInternal.CurrentSchemaProvider(); + } + + static SchemaProvider fromSchemaId(final int id) { + return new SchemaProviderInternal.IdSchemaProvider(id); + } + + static SchemaProvider fromSchema(final Schema schema) { + return new SchemaProviderInternal.DirectSchemaProvider(schema); + } + + static SchemaProvider fromSnapshotId(final int snapshotId) { + return new SchemaProviderInternal.SnapshotIdSchemaProvider(snapshotId); + } + + static SchemaProvider fromCurrentSnapshot() { + return new SchemaProviderInternal.CurrentSnapshotSchemaProvider(); + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/SchemaProviderInternal.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/SchemaProviderInternal.java new file mode 100644 index 00000000000..25989d7609e --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/SchemaProviderInternal.java @@ -0,0 +1,117 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.util; + +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; + +/** + * Internal class containing the implementations of {@link SchemaProvider}. + */ +class SchemaProviderInternal { + + interface SchemaProviderImpl { + /** + * Returns the schema for the given table based on this {@link SchemaProvider}. + */ + Schema getSchema(Table table); + } + + // Implementations of SchemaProvider + static class CurrentSchemaProvider implements SchemaProvider, SchemaProviderImpl { + @Override + public Schema getSchema(final Table table) { + return getCurrentSchema(table); + } + } + + static class IdSchemaProvider implements SchemaProvider, SchemaProviderImpl { + private final int schemaId; + + IdSchemaProvider(final int schemaId) { + this.schemaId = schemaId; + } + + @Override + public Schema getSchema(final Table table) { + return getSchemaForId(table, schemaId); + } + } + + static class DirectSchemaProvider implements SchemaProvider, SchemaProviderImpl { + private final Schema schema; + + DirectSchemaProvider(final Schema schema) { + this.schema = schema; + } + + @Override + public Schema getSchema(final Table table) { + return schema; + } + } + + static class SnapshotIdSchemaProvider implements SchemaProvider, SchemaProviderImpl { + private final int snapshotId; + + SnapshotIdSchemaProvider(final int snapshotId) { + this.snapshotId = snapshotId; + } + + @Override + public Schema getSchema(final Table table) { + return getSchemaForSnapshotId(table, snapshotId); + } + } + + static class CurrentSnapshotSchemaProvider implements SchemaProvider, SchemaProviderImpl { + @Override + public Schema getSchema(final Table table) { + return getSchemaForCurrentSnapshot(table); + } + } + + // -------------------------------------------------------------------------------------------------- + + // Methods for extracting the schema from the table + private static Schema getCurrentSchema(final Table table) { + return table.schema(); + } + + private static Schema getSchemaForId(final Table table, final int schemaId) { + final Schema schema = table.schemas().get(schemaId); + if (schema == null) { + throw new IllegalArgumentException("Schema with ID " + schemaId + " not found for table " + table); + } + return schema; + } + + private static Schema getSchemaForSnapshotId(final Table table, final int snapshotId) { + final Snapshot snapshot = table.snapshot(snapshotId); + if (snapshot == null) { + throw new IllegalArgumentException("Snapshot with ID " + snapshotId + " not found for table " + + table); + } + final Integer schemaId = snapshot.schemaId(); + if (schemaId == null) { + throw new IllegalArgumentException("Snapshot with ID " + snapshotId + " does not have a schema ID"); + } + return getSchemaForId(table, schemaId); + } + + private static Schema getSchemaForCurrentSnapshot(final Table table) { + final Snapshot currentSnapshot = table.currentSnapshot(); + if (currentSnapshot == null) { + throw new IllegalArgumentException("Table " + table + " does not have a current snapshot"); + } + final Integer schemaId = currentSnapshot.schemaId(); + if (schemaId == null) { + throw new IllegalArgumentException("Current snapshot with ID " + currentSnapshot.snapshotId() + + " for table " + table + " does not have a schema ID"); + } + return getSchemaForId(table, schemaId); + } +} + diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/TableParquetWriterOptions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/TableParquetWriterOptions.java new file mode 100644 index 00000000000..98f2211907a --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/TableParquetWriterOptions.java @@ -0,0 +1,130 @@ +// +// 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 io.deephaven.parquet.table.ParquetInstructions; +import org.immutables.value.Value; +import org.jetbrains.annotations.NotNull; + +import java.util.Map; + +import static io.deephaven.parquet.table.ParquetInstructions.MIN_TARGET_PAGE_SIZE; + +/** + * This class provides instructions for building {@link IcebergTableWriter} intended for writing Iceberg tables as + * Parquet data files. The default values documented in this class may change in the future. As such, callers may wish + * to explicitly set the values. + */ +@Value.Immutable +@BuildableStyle +public abstract class TableParquetWriterOptions extends TableWriterOptions { + + public static Builder builder() { + return ImmutableTableParquetWriterOptions.builder(); + } + + /** + * The name of the compression codec to use when writing Parquet files; defaults to + * {@link ParquetInstructions#DEFAULT_COMPRESSION_CODEC_NAME}. + */ + @Value.Default + public String compressionCodecName() { + return ParquetInstructions.DEFAULT_COMPRESSION_CODEC_NAME; + } + + /** + * The maximum number of unique keys the parquet file writer should add to a dictionary page before switching to + * non-dictionary encoding; defaults to {@value ParquetInstructions#DEFAULT_MAXIMUM_DICTIONARY_KEYS}; never + * evaluated for non-String columns. + */ + @Value.Default + public int maximumDictionaryKeys() { + return ParquetInstructions.DEFAULT_MAXIMUM_DICTIONARY_KEYS; + } + + /** + * The maximum number of bytes the parquet file writer should add to a dictionary before switching to non-dictionary + * encoding; defaults to {@value ParquetInstructions#DEFAULT_MAXIMUM_DICTIONARY_SIZE}; never evaluated for + * non-String columns. + */ + @Value.Default + public int maximumDictionarySize() { + return ParquetInstructions.DEFAULT_MAXIMUM_DICTIONARY_SIZE; + } + + /** + * The target page size for writing the parquet files; defaults to + * {@link ParquetInstructions#DEFAULT_TARGET_PAGE_SIZE}, should be greater than or equal to + * {@link ParquetInstructions#MIN_TARGET_PAGE_SIZE}. + */ + @Value.Default + public int targetPageSize() { + return ParquetInstructions.DEFAULT_TARGET_PAGE_SIZE; + } + + /** + * Convert this to a {@link ParquetInstructions}. + * + * @param onWriteCompleted The callback to be invoked after writing the parquet file. + * @param tableDefinition The table definition to be populated inside the parquet file's schema + * @param fieldIdToName Mapping of field id to field name, to be populated inside the parquet file's schema + */ + ParquetInstructions toParquetInstructions( + @NotNull final ParquetInstructions.OnWriteCompleted onWriteCompleted, + @NotNull final TableDefinition tableDefinition, + @NotNull final Map fieldIdToName) { + final ParquetInstructions.Builder builder = new ParquetInstructions.Builder(); + + dataInstructions().ifPresent(builder::setSpecialInstructions); + + // Add parquet writing specific instructions. + builder.setTableDefinition(tableDefinition); + for (final Map.Entry entry : fieldIdToName.entrySet()) { + builder.setFieldId(entry.getValue(), entry.getKey()); + } + builder.setCompressionCodecName(compressionCodecName()); + builder.setMaximumDictionaryKeys(maximumDictionaryKeys()); + builder.setMaximumDictionarySize(maximumDictionarySize()); + builder.setTargetPageSize(targetPageSize()); + builder.setOnWriteCompleted(onWriteCompleted); + + return builder.build(); + } + + public interface Builder extends TableWriterOptions.Builder { + Builder compressionCodecName(String compressionCodecName); + + Builder maximumDictionaryKeys(int maximumDictionaryKeys); + + Builder maximumDictionarySize(int maximumDictionarySize); + + Builder targetPageSize(int targetPageSize); + + TableParquetWriterOptions build(); + } + + @Value.Check + final void boundsCheckMaxDictionaryKeys() { + if (maximumDictionaryKeys() < 0) { + throw new IllegalArgumentException("maximumDictionaryKeys(=" + maximumDictionaryKeys() + ") must be >= 0"); + } + } + + @Value.Check + final void boundsCheckMaxDictionarySize() { + if (maximumDictionarySize() < 0) { + throw new IllegalArgumentException("maximumDictionarySize(=" + maximumDictionarySize() + ") must be >= 0"); + } + } + + @Value.Check + final void boundsCheckTargetPageSize() { + if (targetPageSize() < MIN_TARGET_PAGE_SIZE) { + throw new IllegalArgumentException( + "targetPageSize(=" + targetPageSize() + ") must be >= " + MIN_TARGET_PAGE_SIZE); + } + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/TableWriterOptions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/TableWriterOptions.java new file mode 100644 index 00000000000..95bcb2d8036 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/TableWriterOptions.java @@ -0,0 +1,114 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.util; + +import io.deephaven.engine.table.TableDefinition; +import org.apache.iceberg.Schema; +import org.immutables.value.Value; + +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +public abstract class TableWriterOptions { + + /** + * The {@link TableDefinition} to use when writing Iceberg data files, instead of the one implied by the table being + * written itself. This definition can be used to skip some columns or add additional columns with {@code null} + * values. + */ + public abstract TableDefinition tableDefinition(); + + /** + * The data instructions to use for reading/writing the Iceberg data files (might be S3Instructions or other cloud + * provider-specific instructions). + */ + public abstract Optional dataInstructions(); + + /** + * Used to extract a {@link Schema} from a table. That schema will be used in conjunction with the + * {@link #fieldIdToColumnName()} to map Deephaven columns from {@link #tableDefinition()} to Iceberg columns. If + * {@link #fieldIdToColumnName()} is not provided, the mapping is done by column name. + *

+ * Users can specify how to extract the schema in multiple ways (by schema ID, snapshot ID, etc.). + *

+ * Defaults to {@link SchemaProvider#fromCurrent()}, which means use the current schema from the table. + */ + @Value.Default + public SchemaProvider schemaProvider() { + return SchemaProvider.fromCurrent(); + } + + /** + * A one-to-one {@link Map map} from Iceberg field IDs from the {@link #schemaProvider()} to Deephaven column names + * from the {@link #tableDefinition()}. + */ + public abstract Map fieldIdToColumnName(); + + /** + * A reverse mapping of {@link #fieldIdToColumnName()}. + */ + @Value.Lazy + Map dhColumnNameToFieldId() { + final Map reversedMap = new HashMap<>(fieldIdToColumnName().size()); + for (final Map.Entry entry : fieldIdToColumnName().entrySet()) { + reversedMap.put(entry.getValue(), entry.getKey()); + } + return reversedMap; + } + + // @formatter:off + interface Builder> { + // @formatter:on + INSTRUCTIONS_BUILDER tableDefinition(TableDefinition tableDefinition); + + INSTRUCTIONS_BUILDER dataInstructions(Object s3Instructions); + + INSTRUCTIONS_BUILDER schemaProvider(SchemaProvider schemaProvider); + + INSTRUCTIONS_BUILDER putFieldIdToColumnName(int value, String key); + + INSTRUCTIONS_BUILDER putAllFieldIdToColumnName(Map entries); + } + + /** + * Check all column names present in the {@link #fieldIdToColumnName()} map are present in the + * {@link #tableDefinition()}. + */ + @Value.Check + final void checkDhColumnsToIcebergFieldIds() { + if (!fieldIdToColumnName().isEmpty()) { + final Set columnNamesFromDefinition = tableDefinition().getColumnNameSet(); + final Map fieldIdToColumnName = fieldIdToColumnName(); + for (final String columnNameFromMap : fieldIdToColumnName.values()) { + if (!columnNamesFromDefinition.contains(columnNameFromMap)) { + throw new IllegalArgumentException("Column " + columnNameFromMap + " not found in table " + + "definition, available columns are: " + columnNamesFromDefinition); + } + } + } + } + + @Value.Check + final void checkOneToOneMapping() { + final Collection columnNames = new HashSet<>(fieldIdToColumnName().size()); + for (final String columnName : fieldIdToColumnName().values()) { + if (columnNames.contains(columnName)) { + throw new IllegalArgumentException("Duplicate mapping found: " + columnName + " in field Id to column" + + " name map, expected one-to-one mapping"); + } + columnNames.add(columnName); + } + } + + @Value.Check + final void checkNonEmptyDefinition() { + if (tableDefinition().numColumns() == 0) { + throw new IllegalArgumentException("Cannot write to an Iceberg table using empty table definition"); + } + } +} diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/PyIceberg1Test.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/PyIceberg1Test.java index 3759ebc3305..52a6f5a4b9b 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/PyIceberg1Test.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/PyIceberg1Test.java @@ -29,7 +29,7 @@ * See TESTING.md and generate-pyiceberg-1.py for more details. */ @Tag("security-manager-allow") -public class PyIceberg1Test { +class PyIceberg1Test { private static final Namespace NAMESPACE = Namespace.of("dh-default"); private static final TableIdentifier CITIES_ID = TableIdentifier.of(NAMESPACE, "cities"); diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/junit5/FileWarehouseSqliteCatalogTest.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/junit5/FileWarehouseSqliteCatalogTest.java index 75401946e50..9b4eabe929d 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/junit5/FileWarehouseSqliteCatalogTest.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/junit5/FileWarehouseSqliteCatalogTest.java @@ -6,6 +6,7 @@ import io.deephaven.iceberg.sqlite.SqliteHelper; import io.deephaven.iceberg.util.IcebergCatalogAdapter; import io.deephaven.iceberg.util.IcebergTools; +import org.jetbrains.annotations.Nullable; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.TestInfo; @@ -17,7 +18,13 @@ * served via local file IO. */ @Tag("security-manager-allow") -public final class FileWarehouseSqliteCatalogTest extends SqliteCatalogBase { +final class FileWarehouseSqliteCatalogTest extends SqliteCatalogBase { + + @Override + @Nullable + public Object dataInstructions() { + return null; + } @Override protected IcebergCatalogAdapter catalogAdapter(TestInfo testInfo, Path rootDir, Map properties) { diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/junit5/SqliteCatalogBase.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/junit5/SqliteCatalogBase.java index 1f8522e45f8..068e77b31c3 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/junit5/SqliteCatalogBase.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/junit5/SqliteCatalogBase.java @@ -3,37 +3,70 @@ // package io.deephaven.iceberg.junit5; +import io.deephaven.UncheckedDeephavenException; +import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.table.impl.PartitionAwareSourceTable; +import io.deephaven.engine.table.impl.select.FormulaEvaluationException; +import io.deephaven.engine.testutil.ControlledUpdateGraph; +import io.deephaven.engine.util.TableTools; +import io.deephaven.iceberg.base.IcebergUtils; import io.deephaven.engine.testutil.junit4.EngineCleanup; import io.deephaven.iceberg.sqlite.SqliteHelper; import io.deephaven.iceberg.util.IcebergCatalogAdapter; +import io.deephaven.iceberg.util.IcebergReadInstructions; import io.deephaven.iceberg.util.IcebergTableAdapter; +import io.deephaven.iceberg.util.IcebergTableImpl; +import io.deephaven.iceberg.util.IcebergTableWriter; +import io.deephaven.iceberg.util.IcebergUpdateMode; +import io.deephaven.iceberg.util.IcebergWriteInstructions; +import io.deephaven.iceberg.util.TableParquetWriterOptions; +import io.deephaven.parquet.table.ParquetInstructions; +import io.deephaven.parquet.table.ParquetTools; +import io.deephaven.parquet.table.location.ParquetTableLocationKey; +import io.deephaven.qst.type.Type; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.types.Types; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.jetbrains.annotations.Nullable; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.io.TempDir; +import java.net.URI; +import java.net.URISyntaxException; import java.nio.file.Path; +import java.time.LocalDate; import java.util.HashMap; import java.util.Map; +import java.util.List; +import java.util.stream.Collectors; +import static io.deephaven.engine.testutil.TstUtils.assertTableEquals; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.failBecauseExceptionWasNotThrown; public abstract class SqliteCatalogBase { - protected IcebergCatalogAdapter catalogAdapter; - private EngineCleanup engineCleanup = new EngineCleanup(); + private IcebergCatalogAdapter catalogAdapter; + private final EngineCleanup engineCleanup = new EngineCleanup(); protected abstract IcebergCatalogAdapter catalogAdapter(TestInfo testInfo, Path rootDir, Map properties) throws Exception; + @Nullable + protected abstract Object dataInstructions(); + @BeforeEach void setUp(TestInfo testInfo, @TempDir Path rootDir) throws Exception { engineCleanup.setUp(); @@ -47,6 +80,15 @@ void tearDown() throws Exception { engineCleanup.tearDown(); } + private TableParquetWriterOptions.Builder writerOptionsBuilder() { + final TableParquetWriterOptions.Builder builder = TableParquetWriterOptions.builder(); + final Object dataInstructions; + if ((dataInstructions = dataInstructions()) != null) { + return builder.dataInstructions(dataInstructions); + } + return builder; + } + @Test void empty() { assertThat(catalogAdapter.listNamespaces()).isEmpty(); @@ -79,4 +121,856 @@ void createEmptyTable() { } assertThat(table.isEmpty()).isTrue(); } + + @Test + void appendTableBasicTest() { + final Table source = TableTools.emptyTable(10) + .update("intCol = (int) 2 * i + 10", + "doubleCol = (double) 2.5 * i + 10"); + final TableIdentifier tableIdentifier = TableIdentifier.parse("MyNamespace.MyTable"); + final IcebergTableAdapter tableAdapter = catalogAdapter.createTable(tableIdentifier, source.getDefinition()); + { + final IcebergTableWriter tableWriter = tableAdapter.tableWriter(writerOptionsBuilder() + .tableDefinition(source.getDefinition()) + .build()); + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(source) + .build()); + } + + Table fromIceberg = tableAdapter.table(); + assertTableEquals(source, fromIceberg); + verifySnapshots(tableIdentifier, List.of("append")); + + // Append more data with different compression codec + final Table moreData = TableTools.emptyTable(5) + .update("intCol = (int) 3 * i + 20", + "doubleCol = (double) 3.5 * i + 20"); + final IcebergTableWriter lz4TableWriter = tableAdapter.tableWriter(writerOptionsBuilder() + .tableDefinition(source.getDefinition()) + .compressionCodecName("LZ4") + .build()); + lz4TableWriter.append(IcebergWriteInstructions.builder() + .addTables(moreData) + .build()); + + fromIceberg = tableAdapter.table(); + final Table expected = TableTools.merge(source, moreData); + assertTableEquals(expected, fromIceberg); + verifySnapshots(tableIdentifier, List.of("append", "append")); + + // Append an empty table + final Table emptyTable = TableTools.emptyTable(0) + .update("intCol = (int) 4 * i + 30", + "doubleCol = (double) 4.5 * i + 30"); + lz4TableWriter.append(IcebergWriteInstructions.builder() + .addTables(emptyTable) + .build()); + fromIceberg = tableAdapter.table(); + assertTableEquals(expected, fromIceberg); + verifySnapshots(tableIdentifier, List.of("append", "append", "append")); + + // Append multiple tables in a single call with different compression codec + final Table someMoreData = TableTools.emptyTable(3) + .update("intCol = (int) 5 * i + 40", + "doubleCol = (double) 5.5 * i + 40"); + { + final IcebergTableWriter gzipTableWriter = tableAdapter.tableWriter(writerOptionsBuilder() + .tableDefinition(source.getDefinition()) + .compressionCodecName("GZIP") + .build()); + gzipTableWriter.append(IcebergWriteInstructions.builder() + .addTables(someMoreData, moreData, emptyTable) + .build()); + } + + fromIceberg = tableAdapter.table(); + final Table expected2 = TableTools.merge(expected, someMoreData, moreData); + assertTableEquals(expected2, fromIceberg); + verifySnapshots(tableIdentifier, List.of("append", "append", "append", "append")); + } + + private void verifySnapshots(final TableIdentifier tableIdentifier, final List expectedOperations) { + final Iterable snapshots = catalogAdapter.catalog().loadTable(tableIdentifier).snapshots(); + assertThat(snapshots).map(Snapshot::operation).isEqualTo(expectedOperations); + } + + @Test + void appendWithDifferentDefinition() { + final Table source = TableTools.emptyTable(10) + .update("intCol = (int) 2 * i + 10", + "doubleCol = (double) 2.5 * i + 10"); + final TableIdentifier tableIdentifier = TableIdentifier.parse("MyNamespace.MyTable"); + + final IcebergTableAdapter tableAdapter = catalogAdapter.createTable(tableIdentifier, source.getDefinition()); + final IcebergTableWriter tableWriter = tableAdapter.tableWriter(writerOptionsBuilder() + .tableDefinition(source.getDefinition()) + .build()); + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(source) + .build()); + Table fromIceberg = tableAdapter.table(); + assertTableEquals(source, fromIceberg); + verifySnapshots(tableIdentifier, List.of("append")); + + // Append a table with just the int column + final Table expected; + { + final IcebergTableWriter tableWriterWithOneColumn = tableAdapter.tableWriter(writerOptionsBuilder() + .tableDefinition(TableDefinition.of(ColumnDefinition.ofInt("intCol"))) + .build()); + final Table singleColumnSource = TableTools.emptyTable(10) + .update("intCol = (int) 5 * i + 10"); + tableWriterWithOneColumn.append(IcebergWriteInstructions.builder() + .addTables(singleColumnSource) + .build()); + fromIceberg = tableAdapter.table(); + expected = TableTools.merge(source, singleColumnSource.update("doubleCol = NULL_DOUBLE")); + assertTableEquals(expected, fromIceberg); + verifySnapshots(tableIdentifier, List.of("append", "append")); + } + + // Append more data + final Table moreData = TableTools.emptyTable(5) + .update("intCol = (int) 3 * i + 20", + "doubleCol = (double) 3.5 * i + 20"); + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(moreData) + .build()); + fromIceberg = tableAdapter.table(); + final Table expected2 = TableTools.merge(expected, moreData); + assertTableEquals(expected2, fromIceberg); + verifySnapshots(tableIdentifier, List.of("append", "append", "append")); + + // Append an empty table + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(TableTools.emptyTable(0).update( + "intCol = (int) 3 * i + 20", + "doubleCol = (double) 3.5 * i + 20")) + .build()); + fromIceberg = tableAdapter.table(); + assertTableEquals(expected2, fromIceberg); + verifySnapshots(tableIdentifier, List.of("append", "append", "append", "append")); + } + + @Test + void appendMultipleTablesWithDifferentDefinitionTest() { + final Table source = TableTools.emptyTable(10) + .update("intCol = (int) 2 * i + 10", + "doubleCol = (double) 2.5 * i + 10"); + final TableIdentifier tableIdentifier = TableIdentifier.parse("MyNamespace.MyTable"); + final IcebergTableAdapter tableAdapter = catalogAdapter.createTable(tableIdentifier, source.getDefinition()); + final IcebergTableWriter tableWriter = tableAdapter.tableWriter(writerOptionsBuilder() + .tableDefinition(source.getDefinition()) + .build()); + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(source) + .build()); + Table fromIceberg = tableAdapter.table(); + assertTableEquals(source, fromIceberg); + + try { + final Table appendTable = TableTools.emptyTable(5) + .update("intCol = (int) 3 * i + 20", + "doubleCol = (double) 3.5 * i + 20", + "shortCol = (short) 3 * i + 20"); + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(appendTable) + .build()); + failBecauseExceptionWasNotThrown(UncheckedDeephavenException.class); + } catch (TableDefinition.IncompatibleTableDefinitionException e) { + // Table definition mismatch between table writer and append table + assertThat(e).hasMessageContaining("Table definition"); + } + } + + @Test + void appendToCatalogTableWithAllDataTypesTest() { + final Schema schema = new Schema( + Types.NestedField.required(1, "booleanCol", Types.BooleanType.get()), + Types.NestedField.required(2, "doubleCol", Types.DoubleType.get()), + Types.NestedField.required(3, "floatCol", Types.FloatType.get()), + Types.NestedField.required(4, "intCol", Types.IntegerType.get()), + Types.NestedField.required(5, "longCol", Types.LongType.get()), + Types.NestedField.required(6, "stringCol", Types.StringType.get()), + Types.NestedField.required(7, "instantCol", Types.TimestampType.withZone()), + Types.NestedField.required(8, "localDateTimeCol", Types.TimestampType.withoutZone()), + Types.NestedField.required(9, "localDateCol", Types.DateType.get()), + Types.NestedField.required(10, "localTimeCol", Types.TimeType.get()), + Types.NestedField.required(11, "binaryCol", Types.BinaryType.get())); + final Namespace myNamespace = Namespace.of("MyNamespace"); + final TableIdentifier myTableId = TableIdentifier.of(myNamespace, "MyTableWithAllDataTypes"); + catalogAdapter.catalog().createTable(myTableId, schema); + + final Table source = TableTools.emptyTable(10) + .update( + "booleanCol = i % 2 == 0", + "doubleCol = (double) 2.5 * i + 10", + "floatCol = (float) (2.5 * i + 10)", + "intCol = 2 * i + 10", + "longCol = (long) (2 * i + 10)", + "stringCol = String.valueOf(2 * i + 10)", + "instantCol = java.time.Instant.now()", + "localDateTimeCol = java.time.LocalDateTime.now()", + "localDateCol = java.time.LocalDate.now()", + "localTimeCol = java.time.LocalTime.now()", + "binaryCol = new byte[] {(byte) i}"); + final IcebergTableAdapter tableAdapter = catalogAdapter.loadTable(myTableId); + final IcebergTableWriter tableWriter = tableAdapter.tableWriter(writerOptionsBuilder() + .tableDefinition(source.getDefinition()) + .build()); + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(source) + .build()); + final Table fromIceberg = tableAdapter.table(); + assertTableEquals(source, fromIceberg); + } + + @Test + void testFailureInWrite() { + // Try creating a new iceberg table with bad data + final Table badSource = TableTools.emptyTable(5) + .updateView( + "stringCol = ii % 2 == 0 ? Long.toString(ii) : null", + "intCol = (int) stringCol.charAt(0)"); + final Namespace myNamespace = Namespace.of("MyNamespace"); + final TableIdentifier tableIdentifier = TableIdentifier.of(myNamespace, "MyTable"); + final IcebergTableAdapter tableAdapter = catalogAdapter.createTable(tableIdentifier, badSource.getDefinition()); + final IcebergTableWriter tableWriter = tableAdapter.tableWriter(writerOptionsBuilder() + .tableDefinition(badSource.getDefinition()) + .build()); + + try { + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(badSource) + .build()); + failBecauseExceptionWasNotThrown(UncheckedDeephavenException.class); + } catch (UncheckedDeephavenException e) { + // Exception expected for invalid formula in table + assertThat(e).cause().isInstanceOf(FormulaEvaluationException.class); + } + + // Now create a table with good data with same schema and append a bad source to it + final Table goodSource = TableTools.emptyTable(5) + .update("stringCol = Long.toString(ii)", + "intCol = (int) i"); + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(goodSource) + .build()); + Table fromIceberg = tableAdapter.table(); + assertTableEquals(goodSource, fromIceberg); + + try { + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(badSource) + .build()); + failBecauseExceptionWasNotThrown(UncheckedDeephavenException.class); + } catch (UncheckedDeephavenException e) { + // Exception expected for invalid formula in table + assertThat(e).cause().isInstanceOf(FormulaEvaluationException.class); + } + + try { + final IcebergTableWriter badWriter = tableAdapter.tableWriter(writerOptionsBuilder() + .tableDefinition(TableDefinition.of(ColumnDefinition.ofDouble("doubleCol"))) + .build()); + failBecauseExceptionWasNotThrown(UncheckedDeephavenException.class); + } catch (IllegalArgumentException e) { + // Exception expected because "doubleCol" is not present in the table + assertThat(e).hasMessageContaining("Column doubleCol not found in the schema"); + } + + // Make sure existing good data is not deleted + assertThat(catalogAdapter.listNamespaces()).contains(myNamespace); + assertThat(catalogAdapter.listTables(myNamespace)).containsExactly(tableIdentifier); + fromIceberg = tableAdapter.table(); + assertTableEquals(goodSource, fromIceberg); + } + + @Test + void testColumnRenameWhileWriting() throws URISyntaxException { + final Table source = TableTools.emptyTable(10) + .update("intCol = (int) 2 * i + 10", + "doubleCol = (double) 2.5 * i + 10"); + final TableIdentifier tableIdentifier = TableIdentifier.parse("MyNamespace.MyTable"); + final TableDefinition originalDefinition = source.getDefinition(); + final IcebergTableAdapter tableAdapter = catalogAdapter.createTable(tableIdentifier, originalDefinition); + { + final IcebergTableWriter tableWriter = tableAdapter.tableWriter(writerOptionsBuilder() + .tableDefinition(source.getDefinition()) + .build()); + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(source) + .build()); + + verifyDataFiles(tableIdentifier, List.of(source)); + } + + // Get field IDs for the columns for this table + final Map nameToFieldIdFromSchema = new HashMap<>(); + final Schema schema = tableAdapter.icebergTable().schema(); + for (final Types.NestedField field : schema.columns()) { + nameToFieldIdFromSchema.put(field.name(), field.fieldId()); + } + + { + final List parquetFiles = getAllParquetFilesFromDataFiles(tableIdentifier); + assertThat(parquetFiles).hasSize(1); + verifyFieldIdsFromParquetFile(parquetFiles.get(0), originalDefinition.getColumnNames(), + nameToFieldIdFromSchema); + } + + final Table moreData = TableTools.emptyTable(5) + .update("newIntCol = (int) 3 * i + 20", + "newDoubleCol = (double) 3.5 * i + 20"); + { + // Now append more data to it but with different column names and field Id mapping + final IcebergTableWriter tableWriter = tableAdapter.tableWriter(writerOptionsBuilder() + .tableDefinition(moreData.getDefinition()) + .putFieldIdToColumnName(nameToFieldIdFromSchema.get("intCol"), "newIntCol") + .putFieldIdToColumnName(nameToFieldIdFromSchema.get("doubleCol"), "newDoubleCol") + .build()); + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(moreData) + .build()); + + verifyDataFiles(tableIdentifier, List.of(moreData, source)); + + final Map newNameToFieldId = new HashMap<>(); + newNameToFieldId.put("newIntCol", nameToFieldIdFromSchema.get("intCol")); + newNameToFieldId.put("newDoubleCol", nameToFieldIdFromSchema.get("doubleCol")); + + final List parquetFiles = getAllParquetFilesFromDataFiles(tableIdentifier); + assertThat(parquetFiles).hasSize(2); + verifyFieldIdsFromParquetFile(parquetFiles.get(0), moreData.getDefinition().getColumnNames(), + newNameToFieldId); + verifyFieldIdsFromParquetFile(parquetFiles.get(1), originalDefinition.getColumnNames(), + nameToFieldIdFromSchema); + } + + // TODO: This is failing because we don't map columns based on the column ID when reading. Uncomment this + // when #6156 is merged + // final Table fromIceberg = tableAdapter.table(); + // assertTableEquals(TableTools.merge(source, + // moreData.renameColumns("intCol = newIntCol", "doubleCol = newDoubleCol")), fromIceberg); + } + + /** + * Verify that the schema of the parquet file read from the provided path has the provided column and corresponding + * field IDs. + */ + private void verifyFieldIdsFromParquetFile( + final String path, + final List columnNames, + final Map nameToFieldId) throws URISyntaxException { + final ParquetMetadata metadata = + new ParquetTableLocationKey(new URI(path), 0, null, ParquetInstructions.builder() + .setSpecialInstructions(dataInstructions()) + .build()) + .getMetadata(); + final List columnsMetadata = metadata.getFileMetaData().getSchema().getColumns(); + + final int numColumns = columnNames.size(); + for (int colIdx = 0; colIdx < numColumns; colIdx++) { + final String columnName = columnNames.get(colIdx); + final String columnNameFromParquetFile = columnsMetadata.get(colIdx).getPath()[0]; + assertThat(columnName).isEqualTo(columnNameFromParquetFile); + + final int expectedFieldId = nameToFieldId.get(columnName); + final int fieldIdFromParquetFile = columnsMetadata.get(colIdx).getPrimitiveType().getId().intValue(); + assertThat(fieldIdFromParquetFile).isEqualTo(expectedFieldId); + } + } + + /** + * Verify that the data files in the table match the Deephaven tables in the given sequence. + */ + private void verifyDataFiles( + final TableIdentifier tableIdentifier, + final List

dhTables) { + final org.apache.iceberg.Table table = catalogAdapter.catalog().loadTable(tableIdentifier); + final List dataFileList = IcebergUtils.allDataFiles(table, table.currentSnapshot()) + .collect(Collectors.toList()); + assertThat(dataFileList).hasSize(dhTables.size()); + + // Check that each Deephaven table matches the corresponding data file in sequence + for (int i = 0; i < dhTables.size(); i++) { + final Table dhTable = dhTables.get(i); + final DataFile dataFile = dataFileList.get(i); + final String parquetFilePath = dataFile.path().toString(); + final Table fromParquet = ParquetTools.readTable(parquetFilePath, ParquetInstructions.builder() + .setSpecialInstructions(dataInstructions()) + .build()); + assertTableEquals(dhTable, fromParquet); + } + } + + /** + * Get all the parquet files in the table. + */ + private List getAllParquetFilesFromDataFiles(final TableIdentifier tableIdentifier) { + final org.apache.iceberg.Table table = catalogAdapter.catalog().loadTable(tableIdentifier); + return IcebergUtils.allDataFiles(table, table.currentSnapshot()) + .map(dataFile -> dataFile.path().toString()) + .collect(Collectors.toList()); + } + + @Test + void writeDataFilesBasicTest() { + final Table source = TableTools.emptyTable(10) + .update("intCol = (int) 2 * i + 10", + "doubleCol = (double) 2.5 * i + 10"); + final Table anotherSource = TableTools.emptyTable(5) + .update("intCol = (int) 3 * i + 20", + "doubleCol = (double) 3.5 * i + 20"); + + final TableIdentifier tableIdentifier = TableIdentifier.parse("MyNamespace.MyTable"); + final IcebergTableAdapter tableAdapter = catalogAdapter.createTable(tableIdentifier, source.getDefinition()); + + final IcebergTableWriter tableWriter = tableAdapter.tableWriter(writerOptionsBuilder() + .tableDefinition(source.getDefinition()) + .build()); + + final List dataFilesWritten = tableWriter.writeDataFiles(IcebergWriteInstructions.builder() + .addTables(source, anotherSource) + .build()); + verifySnapshots(tableIdentifier, List.of()); + assertThat(dataFilesWritten).hasSize(2); + + // Append some data to the table + final Table moreData = TableTools.emptyTable(5) + .update("intCol = (int) 3 * i + 20", + "doubleCol = (double) 3.5 * i + 20"); + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(moreData) + .build()); + { + final Table fromIceberg = tableAdapter.table(); + assertTableEquals(moreData, fromIceberg); + verifySnapshots(tableIdentifier, List.of("append")); + verifyDataFiles(tableIdentifier, List.of(moreData)); + } + + // Now commit those data files to the table + final org.apache.iceberg.Table icebergTable = catalogAdapter.catalog().loadTable(tableIdentifier); + final AppendFiles append = icebergTable.newAppend(); + dataFilesWritten.forEach(append::appendFile); + append.commit(); + + // Verify that the data files are now in the table + verifySnapshots(tableIdentifier, List.of("append", "append")); + verifyDataFiles(tableIdentifier, List.of(source, anotherSource, moreData)); + + { + // Verify thaty we read the data files in the correct order + final Table fromIceberg = tableAdapter.table(); + assertTableEquals(TableTools.merge(moreData, source, anotherSource), fromIceberg); + } + } + + @Test + void testPartitionedAppendBasic() { + final Table part1 = TableTools.emptyTable(10) + .update("intCol = (int) 2 * i + 10", + "doubleCol = (double) 2.5 * i + 10"); + final Table part2 = TableTools.emptyTable(5) + .update("intCol = (int) 3 * i + 20", + "doubleCol = (double) 3.5 * i + 20"); + final List partitionPaths = List.of("PC=cat", "PC=apple"); + final TableIdentifier tableIdentifier = TableIdentifier.parse("MyNamespace.MyTable"); + { + final TableDefinition tableDefinition = part1.getDefinition(); + final IcebergTableAdapter tableAdapter = catalogAdapter.createTable(tableIdentifier, tableDefinition); + final IcebergTableWriter tableWriter = tableAdapter.tableWriter(writerOptionsBuilder() + .tableDefinition(tableDefinition) + .build()); + try { + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(part1, part2) + .addAllPartitionPaths(partitionPaths) + .build()); + failBecauseExceptionWasNotThrown(IllegalArgumentException.class); + } catch (IllegalArgumentException e) { + // Exception expected since partition paths provided with non partitioned table + assertThat(e).hasMessageContaining("partition paths"); + } + catalogAdapter.catalog().dropTable(tableIdentifier, true); + } + + final TableDefinition partitioningTableDef = TableDefinition.of( + ColumnDefinition.ofInt("intCol"), + ColumnDefinition.ofDouble("doubleCol"), + ColumnDefinition.ofString("PC").withPartitioning()); + final IcebergTableAdapter tableAdapter = catalogAdapter.createTable(tableIdentifier, partitioningTableDef); + final IcebergTableWriter tableWriter = tableAdapter.tableWriter(writerOptionsBuilder() + .tableDefinition(partitioningTableDef) + .build()); + + try { + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(part1, part2) + .build()); + failBecauseExceptionWasNotThrown(IllegalArgumentException.class); + } catch (IllegalArgumentException e) { + // Exception expected since partition paths not provided with a partitioned table + assertThat(e).hasMessageContaining("partition paths"); + } + + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(part1, part2) + .addAllPartitionPaths(partitionPaths) + .build()); + final Table fromIceberg = tableAdapter.table(); + assertThat(tableAdapter.definition()).isEqualTo(partitioningTableDef); + assertThat(fromIceberg.getDefinition()).isEqualTo(partitioningTableDef); + assertThat(fromIceberg).isInstanceOf(PartitionAwareSourceTable.class); + final Table expected = TableTools.merge( + part1.update("PC = `cat`"), + part2.update("PC = `apple`")); + assertTableEquals(expected, fromIceberg.select()); + + final Table part3 = TableTools.emptyTable(5) + .update("intCol = (int) 4 * i + 30", + "doubleCol = (double) 4.5 * i + 30"); + final String partitionPath = "PC=boy"; + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(part3) + .addPartitionPaths(partitionPath) + .build()); + final Table fromIceberg2 = tableAdapter.table(); + final Table expected2 = TableTools.merge( + part1.update("PC = `cat`"), + part2.update("PC = `apple`"), + part3.update("PC = `boy`")); + assertTableEquals(expected2, fromIceberg2.select()); + } + + @Test + void testPartitionedAppendBasicIntegerPartitions() { + final Table part1 = TableTools.emptyTable(10) + .update("intCol = (int) 2 * i + 10", + "doubleCol = (double) 2.5 * i + 10"); + final Table part2 = TableTools.emptyTable(5) + .update("intCol = (int) 3 * i + 20", + "doubleCol = (double) 3.5 * i + 20"); + final TableIdentifier tableIdentifier = TableIdentifier.parse("MyNamespace.MyTable"); + + final TableDefinition tableDefinition = TableDefinition.of( + ColumnDefinition.ofInt("intCol"), + ColumnDefinition.ofDouble("doubleCol"), + ColumnDefinition.ofInt("PC").withPartitioning()); + final IcebergTableAdapter tableAdapter = catalogAdapter.createTable(tableIdentifier, tableDefinition); + final IcebergTableWriter tableWriter = tableAdapter.tableWriter(writerOptionsBuilder() + .tableDefinition(tableDefinition) + .build()); + + { + // Add partition paths of incorrect type + try { + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(part1, part2) + .addAllPartitionPaths(List.of("PC=cat", "PC=apple")) + .build()); + failBecauseExceptionWasNotThrown(IllegalArgumentException.class); + } catch (IllegalArgumentException e) { + // Exception expected since partition paths provided of incorrect type + assertThat(e).hasMessageContaining("partition path"); + } + } + + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(part1, part2) + .addAllPartitionPaths(List.of("PC=3", "PC=1")) + .build()); + final Table fromIceberg = tableAdapter.table(); + assertThat(tableAdapter.definition()).isEqualTo(tableDefinition); + assertThat(fromIceberg.getDefinition()).isEqualTo(tableDefinition); + assertThat(fromIceberg).isInstanceOf(PartitionAwareSourceTable.class); + final Table expected = TableTools.merge( + part1.update("PC = 3"), + part2.update("PC = 1")); + assertTableEquals(expected, fromIceberg.select()); + + final Table part3 = TableTools.emptyTable(5) + .update("intCol = (int) 4 * i + 30", + "doubleCol = (double) 4.5 * i + 30"); + final String partitionPath = "PC=2"; + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(part3) + .addPartitionPaths(partitionPath) + .build()); + final Table fromIceberg2 = tableAdapter.table(); + final Table expected2 = TableTools.merge( + part1.update("PC = 3"), + part2.update("PC = 1"), + part3.update("PC = 2")); + assertTableEquals(expected2, fromIceberg2.select()); + } + + @Test + void testPartitionedAppendWithAllPartitioningTypes() { + final TableDefinition definition = TableDefinition.of( + ColumnDefinition.ofString("StringPC").withPartitioning(), + ColumnDefinition.ofBoolean("BooleanPC").withPartitioning(), + ColumnDefinition.ofInt("IntegerPC").withPartitioning(), + ColumnDefinition.ofLong("LongPC").withPartitioning(), + ColumnDefinition.ofFloat("FloatPC").withPartitioning(), + ColumnDefinition.ofDouble("DoublePC").withPartitioning(), + ColumnDefinition.of("LocalDatePC", Type.find(LocalDate.class)).withPartitioning(), + ColumnDefinition.ofInt("data")); + + final TableIdentifier tableIdentifier = TableIdentifier.parse("MyNamespace.MyTable"); + final IcebergTableAdapter tableAdapter = catalogAdapter.createTable(tableIdentifier, definition); + + final Table source = TableTools.emptyTable(10) + .update("data = (int) 2 * i + 10"); + final IcebergTableWriter tableWriter = tableAdapter.tableWriter(writerOptionsBuilder() + .tableDefinition(definition) + .build()); + + final List partitionPaths = List.of( + "StringPC=AA/" + + "BooleanPC=true/" + + "IntegerPC=1/" + + "LongPC=2/" + + "FloatPC=3.0/" + + "DoublePC=4.0/" + + "LocalDatePC=2023-10-01"); + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(source) + .addAllPartitionPaths(partitionPaths) + .build()); + final Table fromIceberg = tableAdapter.table(); + assertThat(tableAdapter.definition()).isEqualTo(definition); + assertThat(fromIceberg.getDefinition()).isEqualTo(definition); + assertThat(fromIceberg).isInstanceOf(PartitionAwareSourceTable.class); + + final Table expected = source.updateView( + "StringPC = `AA`", + "BooleanPC = (Boolean) true", + "IntegerPC = (int) 1", + "LongPC = (long) 2", + "FloatPC = (float) 3.0", + "DoublePC = (double) 4.0", + "LocalDatePC = LocalDate.parse(`2023-10-01`)") + .moveColumns(7, "data"); + + // TODO (deephaven-core#6419) Dropping the local data column since it is not supported on the read side. + // Remove this when the issue is fixed. + final TableDefinition tableDefinitionWithoutLocalDate = fromIceberg.dropColumns("LocalDatePC").getDefinition(); + final Table fromIcebergWithoutLocalDate = tableAdapter.table(IcebergReadInstructions.builder() + .tableDefinition(tableDefinitionWithoutLocalDate) + .build()); + assertTableEquals(expected.dropColumns("LocalDatePC"), fromIcebergWithoutLocalDate); + } + + @Test + void testManualRefreshingAppend() { + final Table source = TableTools.emptyTable(10) + .update("intCol = (int) 2 * i + 10", + "doubleCol = (double) 2.5 * i + 10"); + final TableIdentifier tableIdentifier = TableIdentifier.parse("MyNamespace.MyTable"); + final IcebergTableAdapter tableAdapter = catalogAdapter.createTable(tableIdentifier, source.getDefinition()); + { + final IcebergTableWriter tableWriter = tableAdapter.tableWriter(writerOptionsBuilder() + .tableDefinition(source.getDefinition()) + .build()); + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(source) + .build()); + } + + final ControlledUpdateGraph updateGraph = ExecutionContext.getContext().getUpdateGraph().cast(); + + final IcebergTableImpl fromIcebergRefreshing = + (IcebergTableImpl) tableAdapter.table(IcebergReadInstructions.builder() + .updateMode(IcebergUpdateMode.manualRefreshingMode()) + .build()); + assertTableEquals(source, fromIcebergRefreshing); + verifySnapshots(tableIdentifier, List.of("append")); + + + // Append more data with different compression codec + final Table moreData = TableTools.emptyTable(5) + .update("intCol = (int) 3 * i + 20", + "doubleCol = (double) 3.5 * i + 20"); + { + final IcebergTableWriter tableWriter = tableAdapter.tableWriter(writerOptionsBuilder() + .tableDefinition(source.getDefinition()) + .compressionCodecName("LZ4") + .build()); + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(moreData) + .build()); + } + + fromIcebergRefreshing.update(); + updateGraph.runWithinUnitTestCycle(fromIcebergRefreshing::refresh); + + final Table expected = TableTools.merge(source, moreData); + assertTableEquals(expected, fromIcebergRefreshing); + verifySnapshots(tableIdentifier, List.of("append", "append")); + + assertTableEquals(expected, tableAdapter.table()); + } + + @Test + void testAutomaticRefreshingAppend() throws InterruptedException { + final Table source = TableTools.emptyTable(10) + .update("intCol = (int) 2 * i + 10", + "doubleCol = (double) 2.5 * i + 10"); + final TableIdentifier tableIdentifier = TableIdentifier.parse("MyNamespace.MyTable"); + final IcebergTableAdapter tableAdapter = catalogAdapter.createTable(tableIdentifier, source.getDefinition()); + { + final IcebergTableWriter tableWriter = tableAdapter.tableWriter(writerOptionsBuilder() + .tableDefinition(source.getDefinition()) + .build()); + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(source) + .build()); + } + + final IcebergTableImpl fromIcebergRefreshing = + (IcebergTableImpl) tableAdapter.table(IcebergReadInstructions.builder() + .updateMode(IcebergUpdateMode.autoRefreshingMode(10)) + .build()); + assertTableEquals(source, fromIcebergRefreshing); + verifySnapshots(tableIdentifier, List.of("append")); + + // Append more data with different compression codec + final Table moreData = TableTools.emptyTable(5) + .update("intCol = (int) 3 * i + 20", + "doubleCol = (double) 3.5 * i + 20"); + { + final IcebergTableWriter tableWriter = tableAdapter.tableWriter(writerOptionsBuilder() + .tableDefinition(source.getDefinition()) + .compressionCodecName("LZ4") + .build()); + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(moreData) + .build()); + } + + // Sleep for 0.5 second + Thread.sleep(500); + + final ControlledUpdateGraph updateGraph = ExecutionContext.getContext().getUpdateGraph().cast(); + updateGraph.runWithinUnitTestCycle(fromIcebergRefreshing::refresh); + + final Table expected = TableTools.merge(source, moreData); + assertTableEquals(expected, fromIcebergRefreshing); + verifySnapshots(tableIdentifier, List.of("append", "append")); + + assertTableEquals(expected, tableAdapter.table()); + } + + @Test + void testManualRefreshingPartitionedAppend() { + final Table part1 = TableTools.emptyTable(10) + .update("intCol = (int) 2 * i + 10", + "doubleCol = (double) 2.5 * i + 10"); + final Table part2 = TableTools.emptyTable(5) + .update("intCol = (int) 3 * i + 20", + "doubleCol = (double) 3.5 * i + 20"); + final List partitionPaths = List.of("PC=apple", "PC=boy"); + final TableIdentifier tableIdentifier = TableIdentifier.parse("MyNamespace.MyTable"); + + final TableDefinition tableDefinition = TableDefinition.of( + ColumnDefinition.ofInt("intCol"), + ColumnDefinition.ofDouble("doubleCol"), + ColumnDefinition.ofString("PC").withPartitioning()); + final IcebergTableAdapter tableAdapter = catalogAdapter.createTable(tableIdentifier, tableDefinition); + final IcebergTableWriter tableWriter = tableAdapter.tableWriter(writerOptionsBuilder() + .tableDefinition(tableDefinition) + .build()); + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(part1, part2) + .addAllPartitionPaths(partitionPaths) + .build()); + + final ControlledUpdateGraph updateGraph = ExecutionContext.getContext().getUpdateGraph().cast(); + + final IcebergTableImpl fromIcebergRefreshing = + (IcebergTableImpl) tableAdapter.table(IcebergReadInstructions.builder() + .updateMode(IcebergUpdateMode.manualRefreshingMode()) + .build()); + assertThat(tableAdapter.definition()).isEqualTo(tableDefinition); + assertThat(fromIcebergRefreshing.getDefinition()).isEqualTo(tableDefinition); + assertThat(fromIcebergRefreshing).isInstanceOf(PartitionAwareSourceTable.class); + final Table expected = TableTools.merge( + part1.update("PC = `apple`"), + part2.update("PC = `boy`")); + assertTableEquals(expected, fromIcebergRefreshing.select()); + + final Table part3 = TableTools.emptyTable(5) + .update("intCol = (int) 4 * i + 30", + "doubleCol = (double) 4.5 * i + 30"); + final String partitionPath = "PC=cat"; + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(part3) + .addPartitionPaths(partitionPath) + .build()); + + fromIcebergRefreshing.update(); + updateGraph.runWithinUnitTestCycle(fromIcebergRefreshing::refresh); + + final Table expected2 = TableTools.merge(expected, part3.update("PC = `cat`")); + assertTableEquals(expected2, fromIcebergRefreshing.select()); + } + + @Test + void testAutoRefreshingPartitionedAppend() throws InterruptedException { + final Table part1 = TableTools.emptyTable(10) + .update("intCol = (int) 2 * i + 10", + "doubleCol = (double) 2.5 * i + 10"); + final Table part2 = TableTools.emptyTable(5) + .update("intCol = (int) 3 * i + 20", + "doubleCol = (double) 3.5 * i + 20"); + final List partitionPaths = List.of("PC=apple", "PC=boy"); + final TableIdentifier tableIdentifier = TableIdentifier.parse("MyNamespace.MyTable"); + + final TableDefinition tableDefinition = TableDefinition.of( + ColumnDefinition.ofInt("intCol"), + ColumnDefinition.ofDouble("doubleCol"), + ColumnDefinition.ofString("PC").withPartitioning()); + final IcebergTableAdapter tableAdapter = catalogAdapter.createTable(tableIdentifier, tableDefinition); + final IcebergTableWriter tableWriter = tableAdapter.tableWriter(writerOptionsBuilder() + .tableDefinition(tableDefinition) + .build()); + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(part1, part2) + .addAllPartitionPaths(partitionPaths) + .build()); + + final ControlledUpdateGraph updateGraph = ExecutionContext.getContext().getUpdateGraph().cast(); + + final IcebergTableImpl fromIcebergRefreshing = + (IcebergTableImpl) tableAdapter.table(IcebergReadInstructions.builder() + .updateMode(IcebergUpdateMode.autoRefreshingMode(10)) + .build()); + assertThat(tableAdapter.definition()).isEqualTo(tableDefinition); + assertThat(fromIcebergRefreshing.getDefinition()).isEqualTo(tableDefinition); + assertThat(fromIcebergRefreshing).isInstanceOf(PartitionAwareSourceTable.class); + final Table expected = TableTools.merge( + part1.update("PC = `apple`"), + part2.update("PC = `boy`")); + assertTableEquals(expected, fromIcebergRefreshing.select()); + + final Table part3 = TableTools.emptyTable(5) + .update("intCol = (int) 4 * i + 30", + "doubleCol = (double) 4.5 * i + 30"); + final String partitionPath = "PC=cat"; + tableWriter.append(IcebergWriteInstructions.builder() + .addTables(part3) + .addPartitionPaths(partitionPath) + .build()); + + // Sleep for 0.5 second + Thread.sleep(500); + + updateGraph.runWithinUnitTestCycle(fromIcebergRefreshing::refresh); + + final Table expected2 = TableTools.merge(expected, part3.update("PC = `cat`")); + assertTableEquals(expected2, fromIcebergRefreshing.select()); + } } diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileWriter.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileWriter.java index 8cf51a65e7e..d14916a6f93 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileWriter.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileWriter.java @@ -72,6 +72,13 @@ public RowGroupWriter addRowGroup(final long size) { return rowGroupWriter; } + /** + * Get the number of bytes written to the parquet file so far. + */ + public long bytesWritten() { + return countingOutput.getCount(); + } + @Override public void close() throws IOException { serializeOffsetIndexes(); diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/CompletedParquetWrite.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/CompletedParquetWrite.java new file mode 100644 index 00000000000..90ab4989a03 --- /dev/null +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/CompletedParquetWrite.java @@ -0,0 +1,61 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.parquet.table; + +import io.deephaven.annotations.BuildableStyle; +import org.immutables.value.Value; + +import java.net.URI; + +/** + * This class is used as a return POJO for the result of a Parquet write operation. + *

+ * It is intended to be used with the {@link ParquetInstructions#onWriteCompleted()} callback. + */ +@Value.Immutable +@BuildableStyle +public abstract class CompletedParquetWrite { + /** + * The destination URI of the written Parquet file. + */ + public abstract URI destination(); + + /** + * The number of rows written to the Parquet file. + */ + public abstract long numRows(); + + /** + * The number of bytes written to the Parquet file. + */ + public abstract long numBytes(); + + public static Builder builder() { + return ImmutableCompletedParquetWrite.builder(); + } + + interface Builder { + Builder destination(URI destination); + + Builder numRows(long numRows); + + Builder numBytes(long numBytes); + + CompletedParquetWrite build(); + } + + @Value.Check + final void numRowsBoundsCheck() { + if (numRows() < 0) { + throw new IllegalArgumentException("numRows must be non-negative"); + } + } + + @Value.Check + final void numBytesBoundsCheck() { + if (numBytes() <= 0) { + throw new IllegalArgumentException("numBytes must be positive"); + } + } +} 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 07bb2df5549..d8fa3ff9aa7 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 @@ -59,6 +59,10 @@ static TableDefinition ensureDefinition(final ParquetInstructions parquetInstruc private static final boolean DEFAULT_IS_REFRESHING = false; + public interface OnWriteCompleted { + void onWriteCompleted(CompletedParquetWrite completedParquetWrite); + } + public enum ParquetFileLayout { /** * A single parquet file. @@ -196,6 +200,12 @@ public abstract ParquetInstructions withTableDefinitionAndLayout(final TableDefi */ public abstract String baseNameForPartitionedParquetData(); + /** + * @return A callback to be executed when on completing each parquet data file write (excluding the index and + * metadata files). This callback gets invoked by the writing thread in a linear fashion. + */ + public abstract Optional onWriteCompleted(); + @VisibleForTesting public static boolean sameColumnNamesAndCodecMappings(final ParquetInstructions i1, final ParquetInstructions i2) { if (i1 == EMPTY) { @@ -323,7 +333,7 @@ public ParquetInstructions withTableDefinitionAndLayout( return new ReadOnly(null, null, getCompressionCodecName(), getMaximumDictionaryKeys(), getMaximumDictionarySize(), isLegacyParquet(), getTargetPageSize(), isRefreshing(), getSpecialInstructions(), generateMetadataFiles(), baseNameForPartitionedParquetData(), - useLayout, useDefinition, null); + useLayout, useDefinition, null, null); } @Override @@ -331,7 +341,12 @@ ParquetInstructions withIndexColumns(final Collection> indexColumns return new ReadOnly(null, null, getCompressionCodecName(), getMaximumDictionaryKeys(), getMaximumDictionarySize(), isLegacyParquet(), getTargetPageSize(), isRefreshing(), getSpecialInstructions(), generateMetadataFiles(), baseNameForPartitionedParquetData(), - null, null, indexColumns); + null, null, indexColumns, null); + } + + @Override + public Optional onWriteCompleted() { + return Optional.empty(); } }; @@ -442,6 +457,7 @@ private static final class ReadOnly extends ParquetInstructions { private final ParquetFileLayout fileLayout; private final TableDefinition tableDefinition; private final Collection> indexColumns; + private final OnWriteCompleted onWriteCompleted; private ReadOnly( final KeyedObjectHashMap columnNameToInstructions, @@ -457,7 +473,8 @@ private ReadOnly( final String baseNameForPartitionedParquetData, final ParquetFileLayout fileLayout, final TableDefinition tableDefinition, - final Collection> indexColumns) { + final Collection> indexColumns, + final OnWriteCompleted onWriteCompleted) { this.columnNameToInstructions = columnNameToInstructions; this.parquetColumnNameToInstructions = parquetColumnNameToColumnName; this.compressionCodecName = compressionCodecName; @@ -475,6 +492,7 @@ private ReadOnly( : indexColumns.stream() .map(List::copyOf) .collect(Collectors.toUnmodifiableList()); + this.onWriteCompleted = onWriteCompleted; } private T getOrDefault(final String columnName, final T defaultValue, @@ -617,7 +635,7 @@ public ParquetInstructions withTableDefinitionAndLayout( getCompressionCodecName(), getMaximumDictionaryKeys(), getMaximumDictionarySize(), isLegacyParquet(), getTargetPageSize(), isRefreshing(), getSpecialInstructions(), generateMetadataFiles(), baseNameForPartitionedParquetData(), useLayout, useDefinition, - indexColumns); + indexColumns, onWriteCompleted); } @Override @@ -626,7 +644,12 @@ ParquetInstructions withIndexColumns(final Collection> useIndexColu getCompressionCodecName(), getMaximumDictionaryKeys(), getMaximumDictionarySize(), isLegacyParquet(), getTargetPageSize(), isRefreshing(), getSpecialInstructions(), generateMetadataFiles(), baseNameForPartitionedParquetData(), fileLayout, - tableDefinition, useIndexColumns); + tableDefinition, useIndexColumns, onWriteCompleted); + } + + @Override + public Optional onWriteCompleted() { + return Optional.ofNullable(onWriteCompleted); } KeyedObjectHashMap copyColumnNameToInstructions() { @@ -685,6 +708,7 @@ public static class Builder { private ParquetFileLayout fileLayout; private TableDefinition tableDefinition; private Collection> indexColumns; + private OnWriteCompleted onWriteCompleted; /** * For each additional field added, make sure to update the copy constructor builder @@ -712,6 +736,7 @@ public Builder(final ParquetInstructions parquetInstructions) { fileLayout = readOnlyParquetInstructions.getFileLayout().orElse(null); tableDefinition = readOnlyParquetInstructions.getTableDefinition().orElse(null); indexColumns = readOnlyParquetInstructions.getIndexColumns().orElse(null); + onWriteCompleted = readOnlyParquetInstructions.onWriteCompleted().orElse(null); } public Builder addColumnNameMapping(final String parquetColumnName, final String columnName) { @@ -940,6 +965,15 @@ public Builder addAllIndexColumns(final Iterable> indexColumns) { return this; } + /** + * Adds a callback to be executed when on completing each parquet data file write (excluding the index and + * metadata files). + */ + public Builder setOnWriteCompleted(final OnWriteCompleted onWriteCompleted) { + this.onWriteCompleted = onWriteCompleted; + return this; + } + public ParquetInstructions build() { final KeyedObjectHashMap columnNameToInstructionsOut = columnNameToInstructions; columnNameToInstructions = null; @@ -949,7 +983,7 @@ public ParquetInstructions build() { return new ReadOnly(columnNameToInstructionsOut, parquetColumnNameToColumnNameOut, compressionCodecName, maximumDictionaryKeys, maximumDictionarySize, isLegacyParquet, targetPageSize, isRefreshing, specialInstructions, generateMetadataFiles, baseNameForPartitionedParquetData, fileLayout, - tableDefinition, indexColumns); + tableDefinition, indexColumns, onWriteCompleted); } } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTableWriter.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTableWriter.java index ebb1d17571d..c4fdfbb11d8 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTableWriter.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTableWriter.java @@ -173,8 +173,14 @@ static void write( if (!sortedColumns.isEmpty()) { tableInfoBuilder.addSortingColumns(SortColumnInfo.of(sortedColumns.get(0))); } - write(t, definition, writeInstructions, dest, destOutputStream, incomingMeta, + final long numBytes = write(t, definition, writeInstructions, dest, destOutputStream, incomingMeta, tableInfoBuilder, metadataFileWriter, computedCache); + writeInstructions.onWriteCompleted() + .ifPresent(callback -> callback.onWriteCompleted(CompletedParquetWrite.builder() + .destination(dest) + .numRows(t.size()) + .numBytes(numBytes) + .build())); } /** @@ -191,9 +197,11 @@ static void write( * @param metadataFileWriter The writer for the {@value ParquetUtils#METADATA_FILE_NAME} and * {@value ParquetUtils#COMMON_METADATA_FILE_NAME} files * @param computedCache Per column cache tags + * @return The number of bytes written + * * @throws IOException For file writing related errors */ - private static void write( + private static long write( @NotNull final Table table, @NotNull final TableDefinition definition, @NotNull final ParquetInstructions writeInstructions, @@ -207,13 +215,18 @@ private static void write( final Table t = pretransformTable(table, definition); final TrackingRowSet tableRowSet = t.getRowSet(); final Map> columnSourceMap = t.getColumnSourceMap(); - try (final ParquetFileWriter parquetFileWriter = getParquetFileWriter(computedCache, definition, - tableRowSet, columnSourceMap, dest, destOutputStream, writeInstructions, tableMeta, - tableInfoBuilder, metadataFileWriter)) { + final long numBytesWritten; + { + final ParquetFileWriter parquetFileWriter = getParquetFileWriter(computedCache, definition, + tableRowSet, columnSourceMap, dest, destOutputStream, writeInstructions, tableMeta, + tableInfoBuilder, metadataFileWriter); // Given the transformation, do not use the original table's "definition" for writing write(t, writeInstructions, parquetFileWriter, computedCache); + parquetFileWriter.close(); + numBytesWritten = parquetFileWriter.bytesWritten(); } destOutputStream.done(); + return numBytesWritten; } } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTools.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTools.java index a03294a28ed..aaf3cf616d8 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTools.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTools.java @@ -596,11 +596,12 @@ private static void writeTablesImpl( // Write the tables without any index info for (int tableIdx = 0; tableIdx < sources.length; tableIdx++) { final Table source = sources[tableIdx]; + final URI tableDestination = destinations[tableIdx]; final CompletableOutputStream outputStream = channelsProvider.getOutputStream( - destinations[tableIdx], PARQUET_OUTPUT_BUFFER_SIZE); + tableDestination, PARQUET_OUTPUT_BUFFER_SIZE); outputStreams.add(outputStream); - ParquetTableWriter.write(source, definition, writeInstructions, destinations[tableIdx], - outputStream, Collections.emptyMap(), (List) null, + ParquetTableWriter.write(source, definition, writeInstructions, tableDestination, outputStream, + Collections.emptyMap(), (List) null, metadataFileWriter, computedCache); } } else { @@ -622,9 +623,9 @@ private static void writeTablesImpl( for (final ParquetTableWriter.IndexWritingInfo info : indexInfoList) { outputStreams.add(info.destOutputStream); } - final Table sourceTable = sources[tableIdx]; - ParquetTableWriter.write(sourceTable, definition, writeInstructions, destinations[tableIdx], - outputStream, Collections.emptyMap(), indexInfoList, metadataFileWriter, computedCache); + final Table source = sources[tableIdx]; + ParquetTableWriter.write(source, definition, writeInstructions, tableDestination, outputStream, + Collections.emptyMap(), indexInfoList, metadataFileWriter, computedCache); } } diff --git a/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/ParquetTableReadWriteTest.java b/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/ParquetTableReadWriteTest.java index 00ef39474a1..e7faf5be88c 100644 --- a/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/ParquetTableReadWriteTest.java +++ b/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/ParquetTableReadWriteTest.java @@ -2623,6 +2623,40 @@ public void testReadingParquetFilesWithDifferentPageSizes() { assertTableEquals(expected, fromDisk); } + @Test + public void testOnWriteCallback() { + // Write a few tables to disk and check the sizes and number of rows in the files + final Table table1 = TableTools.emptyTable(100_000).update( + "someIntColumn = i * 200", + "someLongColumn = ii * 500"); + final File dest1 = new File(rootFile, "table1.parquet"); + final Table table2 = TableTools.emptyTable(2000).update( + "someIntColumn = i", + "someLongColumn = ii"); + final File dest2 = new File(rootFile, "table2.parquet"); + + final List parquetFilesWritten = new ArrayList<>(); + final ParquetInstructions.OnWriteCompleted onWriteCompleted = parquetFilesWritten::add; + final ParquetInstructions writeInstructions = new ParquetInstructions.Builder() + .setOnWriteCompleted(onWriteCompleted) + .build(); + ParquetTools.writeTables(new Table[] {table1, table2}, + new String[] {dest1.getPath(), dest2.getPath()}, writeInstructions); + + assertEquals(2, parquetFilesWritten.size()); + // Check the destination URIs + assertEquals(dest1.toURI(), parquetFilesWritten.get(0).destination()); + assertEquals(dest2.toURI(), parquetFilesWritten.get(1).destination()); + + // Check the number of rows + assertEquals(100_000, parquetFilesWritten.get(0).numRows()); + assertEquals(2000, parquetFilesWritten.get(1).numRows()); + + // Check the size of the files + assertEquals(dest1.length(), parquetFilesWritten.get(0).numBytes()); + assertEquals(dest2.length(), parquetFilesWritten.get(1).numBytes()); + } + // Following is used for testing both writing APIs for parquet tables private interface TestParquetTableWriter { void writeTable(final Table table, final File destFile); diff --git a/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/S3ParquetLocalStackTest.java b/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/S3ParquetLocalStackTest.java index d4c69017d78..69115fee18b 100644 --- a/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/S3ParquetLocalStackTest.java +++ b/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/S3ParquetLocalStackTest.java @@ -7,7 +7,6 @@ import io.deephaven.extensions.s3.testlib.SingletonContainers.LocalStack; import org.junit.Assume; import org.junit.BeforeClass; -import org.junit.Ignore; import software.amazon.awssdk.services.s3.S3AsyncClient; import java.io.IOException; diff --git a/py/server/deephaven/experimental/iceberg.py b/py/server/deephaven/experimental/iceberg.py index a38093befaf..16e9d359256 100644 --- a/py/server/deephaven/experimental/iceberg.py +++ b/py/server/deephaven/experimental/iceberg.py @@ -3,7 +3,7 @@ # """ This module adds Iceberg table support into Deephaven. """ from __future__ import annotations -from typing import Optional, Dict +from typing import Optional, Dict, Union, Sequence import jpy @@ -14,10 +14,14 @@ from deephaven.jcompat import j_hashmap -_JIcebergReadInstructions = jpy.get_type("io.deephaven.iceberg.util.IcebergReadInstructions") _JIcebergUpdateMode = jpy.get_type("io.deephaven.iceberg.util.IcebergUpdateMode") +_JIcebergReadInstructions = jpy.get_type("io.deephaven.iceberg.util.IcebergReadInstructions") +_JIcebergWriteInstructions = jpy.get_type("io.deephaven.iceberg.util.IcebergWriteInstructions") +_JSchemaProvider = jpy.get_type("io.deephaven.iceberg.util.SchemaProvider") +_JTableParquetWriterOptions = jpy.get_type("io.deephaven.iceberg.util.TableParquetWriterOptions") _JIcebergCatalogAdapter = jpy.get_type("io.deephaven.iceberg.util.IcebergCatalogAdapter") _JIcebergTableAdapter = jpy.get_type("io.deephaven.iceberg.util.IcebergTableAdapter") +_JIcebergTableWriter = jpy.get_type("io.deephaven.iceberg.util.IcebergTableWriter") _JIcebergTable = jpy.get_type("io.deephaven.iceberg.util.IcebergTable") _JIcebergTools = jpy.get_type("io.deephaven.iceberg.util.IcebergTools") @@ -34,7 +38,8 @@ class IcebergUpdateMode(JObjectWrapper): """ - This class specifies the update mode for an Iceberg table to be loaded into Deephaven. The modes are: + :class:`.IcebergUpdateMode` specifies the update mode for an Iceberg table to be loaded into Deephaven. The modes + are: - :py:func:`static() `: The table is loaded once and does not change - :py:func:`manual_refresh() `: The table can be manually refreshed by the user. @@ -62,7 +67,7 @@ def manual_refresh(cls) -> IcebergUpdateMode: return IcebergUpdateMode(_JIcebergUpdateMode.manualRefreshingMode()) @classmethod - def auto_refresh(cls, auto_refresh_ms:Optional[int] = None) -> IcebergUpdateMode: + def auto_refresh(cls, auto_refresh_ms: Optional[int] = None) -> IcebergUpdateMode: """ Creates an IcebergUpdateMode with auto-refreshing enabled. @@ -81,8 +86,9 @@ def j_object(self) -> jpy.JType: class IcebergReadInstructions(JObjectWrapper): """ - This class specifies the instructions for reading an Iceberg table into Deephaven. These include column rename - instructions and table definitions, as well as special data instructions for loading data files from the cloud. + :class:`.IcebergReadInstructions` specifies the instructions for reading an Iceberg table into Deephaven. These + include column rename instructions and table definitions, as well as special data instructions for loading data + files from the cloud. """ j_object_type = _JIcebergReadInstructions @@ -140,17 +146,226 @@ def j_object(self) -> jpy.JType: return self._j_object +class IcebergWriteInstructions(JObjectWrapper): + """ + :class:`.IcebergWriteInstructions` provides instructions intended for writing deephaven tables as partitions to Iceberg + tables. + """ + + j_object_type = _JIcebergWriteInstructions + + def __init__(self, + tables: Union[Table, Sequence[Table]], + partition_paths: Optional[Union[str, Sequence[str]]] = None): + """ + Initializes the instructions using the provided parameters. + + Args: + tables (Union[Table, Sequence[Table]]): The deephaven tables to write. + partition_paths (Optional[Union[str, Sequence[str]]]): The partition paths where each table will be written. + For example, if the iceberg table is partitioned by "year" and "month", a partition path could be + "year=2021/month=01". + If writing to a partitioned iceberg table, users must provide partition path for each table in tables + argument in the same order. + Else when writing to a non-partitioned table, users should not provide any partition paths. + Defaults to `None`, which means the deephaven tables will be written to the root data directory of the + iceberg table. + + Raises: + DHError: If unable to build the instructions object. + """ + + try: + builder = self.j_object_type.builder() + + if isinstance(tables, Table): + builder.addTables(tables.j_table) + elif isinstance(tables, Sequence): + for table in tables: + builder.addTables(table.j_table) + + if partition_paths: + if isinstance(partition_paths, str): + builder.addPartitionPaths(partition_paths) + elif isinstance(partition_paths, Sequence): + for partition_path in partition_paths: + builder.addPartitionPaths(partition_path) + + self._j_object = builder.build() + + except Exception as e: + raise DHError(e, "Failed to build Iceberg write instructions") from e + + @property + def j_object(self) -> jpy.JType: + return self._j_object + + +class SchemaProvider(JObjectWrapper): + """ + :class:`.SchemaProvider` is used to extract the schema from an Iceberg table. Users can specify multiple ways to do + so, for example, by schema ID, snapshot ID, current schema, etc. This can be useful for passing a schema when + writing to an Iceberg table. + """ + + j_object_type = _JSchemaProvider + + def __init__(self, _j_object: jpy.JType): + """ + Initializes the :class:`.SchemaProvider` object. + + Args: + _j_object (SchemaProvider): the Java :class:`.SchemaProvider` object. + """ + self._j_object = _j_object + + @property + def j_object(self) -> jpy.JType: + return self._j_object + + @classmethod + def from_current(cls) -> 'SchemaProvider': + """ + Used for extracting the current schema from the table. + + Returns: + the SchemaProvider object. + """ + return cls(_JSchemaProvider.fromCurrent()) + + @classmethod + def from_schema_id(cls, schema_id: int) -> 'SchemaProvider': + """ + Used for extracting the schema from the table using the specified schema id. + + Args: + schema_id (int): the schema id to use. + + Returns: + the :class:`.SchemaProvider` object. + """ + return cls(_JSchemaProvider.fromSchemaId(schema_id)) + + @classmethod + def from_snapshot_id(cls, snapshot_id: int) -> 'SchemaProvider': + """ + Used for extracting the schema from the table using the specified snapshot id. + + Args: + snapshot_id (int): the snapshot id to use. + + Returns: + the :class:`.SchemaProvider` object. + """ + return cls(_JSchemaProvider.fromSnapshotId(snapshot_id)) + + @classmethod + def from_current_snapshot(cls) -> 'SchemaProvider': + """ + Used for extracting the schema from the table using the current snapshot. + + Returns: + the SchemaProvider object. + """ + return cls(_JSchemaProvider.fromCurrentSnapshot()) + + +class TableParquetWriterOptions(JObjectWrapper): + """ + :class:`.TableParquetWriterOptions` provides specialized instructions for configuring :class:`.IcebergTableWriter` + instances. + """ + + j_object_type = _JTableParquetWriterOptions + + def __init__(self, + table_definition: TableDefinitionLike, + schema_provider: Optional[SchemaProvider] = None, + field_id_to_column_name: Optional[Dict[int, str]] = None, + compression_codec_name: Optional[str] = None, + maximum_dictionary_keys: Optional[int] = None, + maximum_dictionary_size: Optional[int] = None, + target_page_size: Optional[int] = None, + data_instructions: Optional[s3.S3Instructions] = None): + """ + Initializes the instructions using the provided parameters. + + Args: + table_definition: TableDefinitionLike: The table definition to use when writing Iceberg data files using + this writer instance. This definition can be used to skip some columns or add additional columns with + null values. The provided definition should have at least one column. + schema_provider: Optional[SchemaProvider]: Used to extract a Schema from a iceberg table. This schema will + be used in conjunction with the field_id_to_column_name to map Deephaven columns from table_definition + to Iceberg columns. + Users can specify how to extract the schema in multiple ways (by ID, snapshot ID, initial schema, etc.). + Defaults to `None`, which means use the current schema from the table. + field_id_to_column_name: Optional[Dict[int, str]]: A one-to-one map from Iceberg field IDs from the + schema_spec to Deephaven column names from the table_definition. + Defaults to `None`, which means map Iceberg columns to Deephaven columns using column names. + compression_codec_name (Optional[str]): The compression codec to use for writing the parquet file. Allowed + values include "UNCOMPRESSED", "SNAPPY", "GZIP", "LZO", "LZ4", "LZ4_RAW", "ZSTD", etc. Defaults to + `None`, which means use "SNAPPY". + maximum_dictionary_keys (Optional[int]): the maximum number of unique keys the Parquet writer should add to + a dictionary page before switching to non-dictionary encoding, never used for non-String columns. + Defaults to `None`, which means use 2^20 (1,048,576) + maximum_dictionary_size (Optional[int]): the maximum number of bytes the Parquet writer should add to the + dictionary before switching to non-dictionary encoding, never used for non-String columns. Defaults to + `None`, which means use 2^20 (1,048,576) + target_page_size (Optional[int]): the target Parquet file page size in bytes, if not specified. Defaults to + `None`, which means use 2^20 bytes (1 MiB) + + Raises: + DHError: If unable to build the object. + """ + + try: + builder = self.j_object_type.builder() + + builder.tableDefinition(TableDefinition(table_definition).j_table_definition) + + if schema_provider: + builder.schemaProvider(schema_provider.j_object) + + if field_id_to_column_name: + for field_id, column_name in field_id_to_column_name.items(): + builder.putFieldIdToColumnName(field_id, column_name) + + if compression_codec_name: + builder.compressionCodecName(compression_codec_name) + + if maximum_dictionary_keys: + builder.maximumDictionaryKeys(maximum_dictionary_keys) + + if maximum_dictionary_size: + builder.maximumDictionarySize(maximum_dictionary_size) + + if target_page_size: + builder.targetPageSize(target_page_size) + + if data_instructions: + builder.dataInstructions(data_instructions.j_object) + + self._j_object = builder.build() + + except Exception as e: + raise DHError(e, "Failed to build Iceberg write instructions") from e + + @property + def j_object(self) -> jpy.JType: + return self._j_object + + class IcebergTable(Table): """ - IcebergTable is a subclass of Table that allows users to dynamically update the table with new snapshots from - the Iceberg catalog. + :class:`.IcebergTable` is a subclass of Table that allows users to dynamically update the table with new snapshots + from the Iceberg catalog. """ j_object_type = _JIcebergTable def __init__(self, j_table: jpy.JType): super().__init__(j_table) - def update(self, snapshot_id:Optional[int] = None): + def update(self, snapshot_id: Optional[int] = None): """ Updates the table to match the contents of the specified snapshot. This may result in row removes and additions that will be propagated asynchronously via this IcebergTable's UpdateGraph. If no snapshot is provided, the @@ -167,7 +382,7 @@ def update(self, snapshot_id:Optional[int] = None): """ try: - if snapshot_id is not None: + if snapshot_id: self.j_object.update(snapshot_id) return self.j_object.update() @@ -179,10 +394,40 @@ def j_object(self) -> jpy.JType: return self.j_table +class IcebergTableWriter(JObjectWrapper): + """ + :class:`.IcebergTableWriter` is responsible for writing Deephaven tables to an Iceberg table. Each + :class:`.IcebergTableWriter` instance associated with a single :class:`.IcebergTableAdapter` and can be used to + write multiple Deephaven tables to this Iceberg table. + """ + j_object_type = _JIcebergTableWriter or type(None) + + def __init__(self, j_object: _JIcebergTableWriter): + self.j_table_writer = j_object + + def append(self, instructions: IcebergWriteInstructions): + """ + Append the provided Deephaven tables as new partitions to the existing Iceberg table in a single snapshot. + Users can provide the tables using the :attr:`.IcebergWriteInstructions.tables` parameter and optionally provide the + partition paths where each table will be written using the :attr:`.IcebergWriteInstructions.partition_paths` + parameter. + This method will not perform any compatibility checks between the existing schema and the provided Deephaven + tables. All such checks happen at the time of creation of the :class:`.IcebergTableWriter` instance. + + Args: + instructions (IcebergWriteInstructions): the customization instructions for write. + """ + self.j_object.append(instructions.j_object) + + @property + def j_object(self) -> jpy.JType: + return self.j_table_writer + + class IcebergTableAdapter(JObjectWrapper): """ - This class provides an interface for interacting with Iceberg tables. It allows the user to list snapshots, - retrieve table definitions and reading Iceberg tables into Deephaven tables. + :class:`.IcebergTableAdapter` provides an interface for interacting with Iceberg tables. It allows the user to list + snapshots, retrieve table definitions and reading Iceberg tables into Deephaven tables. """ j_object_type = _JIcebergTableAdapter or type(None) @@ -217,7 +462,7 @@ def definition(self, instructions: Optional[IcebergReadInstructions] = None) -> a table containing the table definition. """ - if instructions is not None: + if instructions: return Table(self.j_object.definitionTable(instructions.j_object)) return Table(self.j_object.definitionTable()) @@ -233,13 +478,27 @@ def table(self, instructions: Optional[IcebergReadInstructions] = None) -> Icebe instructions. Returns: - Table: the table read from the catalog. + the table read from the catalog. """ - if instructions is not None: + if instructions: return IcebergTable(self.j_object.table(instructions.j_object)) return IcebergTable(self.j_object.table()) + def table_writer(self, writer_options: TableParquetWriterOptions) -> IcebergTableWriter: + """ + Create a new :class:`.IcebergTableWriter` for this Iceberg table using the provided writer options. + This method will perform schema validation to ensure that the provided table definition from the writer options + is compatible with the Iceberg table schema. All further writes performed by the returned writer will not be + validated against the table's schema, and thus will be faster. + + Args: + writer_options: The options to configure the table writer. + + Returns: + the table writer object + """ + return IcebergTableWriter(self.j_object.tableWriter(writer_options.j_object)) @property def j_object(self) -> jpy.JType: @@ -248,8 +507,8 @@ def j_object(self) -> jpy.JType: class IcebergCatalogAdapter(JObjectWrapper): """ - This class provides an interface for interacting with Iceberg catalogs. It allows listing namespaces, tables and - snapshots, as well as reading Iceberg tables into Deephaven tables. + :class:`.IcebergCatalogAdapter` provides an interface for interacting with Iceberg catalogs. It allows listing + namespaces, tables and snapshots, as well as reading Iceberg tables into Deephaven tables. """ j_object_type = _JIcebergCatalogAdapter or type(None) @@ -269,7 +528,7 @@ def namespaces(self, namespace: Optional[str] = None) -> Table: a table containing the namespaces. """ - if namespace is not None: + if namespace: return Table(self.j_object.namespaces(namespace)) return Table(self.j_object.namespaces()) @@ -299,6 +558,22 @@ def load_table(self, table_identifier: str) -> IcebergTableAdapter: return IcebergTableAdapter(self.j_object.loadTable(table_identifier)) + def create_table(self, table_identifier: str, table_definition: TableDefinitionLike) -> IcebergTableAdapter: + """ + Create a new Iceberg table in the catalog with the given table identifier and definition. + All columns of partitioning type will be used to create the partition spec for the table. + + Args: + table_identifier (str): the identifier of the new table. + table_definition (TableDefinitionLike): the table definition of the new table. + + Returns: + :class:`.IcebergTableAdapter`: the table adapter for the new Iceberg table. + """ + + return IcebergTableAdapter(self.j_object.createTable(table_identifier, + TableDefinition(table_definition).j_table_definition)) + @property def j_object(self) -> jpy.JType: return self.j_catalog_adapter @@ -333,7 +608,7 @@ def adapter_s3_rest( need to set this; it is most useful when connecting to non-AWS, S3-compatible APIs. Returns: - IcebergCatalogAdapter: the catalog adapter for the provided S3 REST catalog. + :class:`.IcebergCatalogAdapter`: the catalog adapter for the provided S3 REST catalog. Raises: DHError: If unable to build the catalog adapter. @@ -371,7 +646,7 @@ def adapter_aws_glue( catalog URI. Returns: - IcebergCatalogAdapter: the catalog adapter for the provided AWS Glue catalog. + :class:`.IcebergCatalogAdapter`: the catalog adapter for the provided AWS Glue catalog. Raises: DHError: If unable to build the catalog adapter. @@ -467,7 +742,7 @@ def adapter( hadoop_config (Optional[Dict[str, str]]): hadoop configuration properties for the catalog to load s3_instructions (Optional[s3.S3Instructions]): the S3 instructions if applicable Returns: - IcebergCatalogAdapter: the catalog adapter created from the provided properties + :class:`.IcebergCatalogAdapter`: the catalog adapter created from the provided properties Raises: DHError: If unable to build the catalog adapter @@ -481,8 +756,8 @@ def adapter( return IcebergCatalogAdapter( _JIcebergToolsS3.createAdapter( name, - j_hashmap(properties if properties is not None else {}), - j_hashmap(hadoop_config if hadoop_config is not None else {}), + j_hashmap(properties if properties else {}), + j_hashmap(hadoop_config if hadoop_config else {}), s3_instructions.j_object, ) ) @@ -493,8 +768,8 @@ def adapter( return IcebergCatalogAdapter( _JIcebergTools.createAdapter( name, - j_hashmap(properties if properties is not None else {}), - j_hashmap(hadoop_config if hadoop_config is not None else {}), + j_hashmap(properties if properties else {}), + j_hashmap(hadoop_config if hadoop_config else {}), ) ) except Exception as e: diff --git a/py/server/deephaven/experimental/s3.py b/py/server/deephaven/experimental/s3.py index 47426533335..afe2b463e88 100644 --- a/py/server/deephaven/experimental/s3.py +++ b/py/server/deephaven/experimental/s3.py @@ -25,6 +25,7 @@ module will fail to find the java types. """ + class Credentials(JObjectWrapper): """ Credentials object for authenticating with an S3 server. diff --git a/py/server/tests/test_iceberg.py b/py/server/tests/test_iceberg.py index a3dcb72ac2f..28a2758c0d0 100644 --- a/py/server/tests/test_iceberg.py +++ b/py/server/tests/test_iceberg.py @@ -13,6 +13,7 @@ _JTableDefinition = jpy.get_type("io.deephaven.engine.table.TableDefinition") + class IcebergTestCase(BaseTestCase): """ Test cases for the deephaven.iceberg module (performed locally) """ @@ -46,7 +47,7 @@ def test_instruction_create_with_col_renames(self): self.assertTrue(col_rename_dict["old_name_c"] == "new_name_c") def test_instruction_create_with_table_definition_dict(self): - table_def={ + table_def = { "x": dtypes.int32, "y": dtypes.double, "z": dtypes.double, @@ -59,7 +60,7 @@ def test_instruction_create_with_table_definition_dict(self): self.assertTrue(col_names[2] == "z") def test_instruction_create_with_table_definition_list(self): - table_def=[ + table_def = [ col_def("Partition", dtypes.int32, column_type=ColumnType.PARTITIONING), col_def("x", dtypes.int32), col_def("y", dtypes.double), @@ -76,3 +77,65 @@ def test_instruction_create_with_table_definition_list(self): def test_instruction_create_with_snapshot_id(self): iceberg_read_instructions = iceberg.IcebergReadInstructions(snapshot_id=12345) self.assertTrue(iceberg_read_instructions.j_object.snapshotId().getAsLong() == 12345) + + def test_writer_options_create_default(self): + writer_options = iceberg.TableParquetWriterOptions(table_definition={"x": dtypes.int32}) + self.assertEqual(writer_options.j_object.compressionCodecName(), "SNAPPY") + self.assertEqual(writer_options.j_object.maximumDictionaryKeys(), 1048576) + self.assertEqual(writer_options.j_object.maximumDictionarySize(), 1048576) + self.assertEqual(writer_options.j_object.targetPageSize(), 65536) + + def test_writer_options_create_with_s3_instructions(self): + s3_instructions = s3.S3Instructions(region_name="us-east-1", + access_key_id="some_access_key_id", + secret_access_key="some_secret_access_key" + ) + writer_options = iceberg.TableParquetWriterOptions(table_definition={"x": dtypes.int32}, + data_instructions=s3_instructions) + + def test_writer_options_create_with_table_definition_dict(self): + table_def = { + "x": dtypes.int32, + "y": dtypes.double, + "z": dtypes.double, + } + writer_options = iceberg.TableParquetWriterOptions(table_def) + col_names = j_list_to_list(writer_options.j_object.tableDefinition().getColumnNames()) + self.assertTrue(col_names[0] == "x") + self.assertTrue(col_names[1] == "y") + self.assertTrue(col_names[2] == "z") + + def test_writer_options_create_with_table_definition_list(self): + table_def = [ + col_def("Partition", dtypes.int32, column_type=ColumnType.PARTITIONING), + col_def("x", dtypes.int32), + col_def("y", dtypes.double), + col_def("z", dtypes.double), + ] + + writer_options = iceberg.TableParquetWriterOptions(table_def) + col_names = j_list_to_list(writer_options.j_object.tableDefinition().getColumnNames()) + self.assertTrue(col_names[0] == "Partition") + self.assertTrue(col_names[1] == "x") + self.assertTrue(col_names[2] == "y") + self.assertTrue(col_names[3] == "z") + + def test_writer_options_create_with_compression_codec(self): + writer_options = iceberg.TableParquetWriterOptions(table_definition={"x": dtypes.int32}, + compression_codec_name="GZIP") + self.assertEqual(writer_options.j_object.compressionCodecName(), "GZIP") + + def test_writer_options_create_with_max_dictionary_keys(self): + writer_options = iceberg.TableParquetWriterOptions(table_definition={"x": dtypes.int32}, + maximum_dictionary_keys=1024) + self.assertEqual(writer_options.j_object.maximumDictionaryKeys(), 1024) + + def test_writer_options_create_with_max_dictionary_size(self): + writer_options = iceberg.TableParquetWriterOptions(table_definition={"x": dtypes.int32}, + maximum_dictionary_size=8192) + self.assertEqual(writer_options.j_object.maximumDictionarySize(), 8192) + + def test_writer_options_create_with_target_page_size(self): + writer_options = iceberg.TableParquetWriterOptions(table_definition={"x": dtypes.int32}, + target_page_size=4096) + self.assertEqual(writer_options.j_object.targetPageSize(), 4096)