From 8c81883a0bb1a51cb0797a6e791f98716e195b93 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Tue, 27 Aug 2024 11:59:58 -0500 Subject: [PATCH 01/28] Initial commit --- .../iceberg/util/IcebergToolsS3.java | 26 +- .../iceberg/util/IcebergToolsTest.java | 135 ++++-- .../iceberg/util/IcebergCatalogAdapter.java | 409 +++++++++++++++++- .../iceberg/util/IcebergInstructions.java | 16 +- .../util/IcebergParquetWriteInstructions.java | 119 +++++ .../parquet/base/ParquetFileWriter.java | 4 +- .../parquet/table/ParquetInstructions.java | 175 ++++---- .../parquet/table/ParquetTableWriter.java | 22 +- .../deephaven/parquet/table/ParquetTools.java | 36 +- .../table/transfer/TransferObject.java | 2 +- .../s3/S3CompletableOutputStream.java | 4 +- 11 files changed, 771 insertions(+), 177 deletions(-) create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.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 166b47e5d28..1586e4286f0 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 @@ -9,6 +9,7 @@ import org.apache.iceberg.aws.AwsClientProperties; import org.apache.iceberg.aws.glue.GlueCatalog; import org.apache.iceberg.aws.s3.S3FileIOProperties; +import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.rest.RESTCatalog; import org.jetbrains.annotations.NotNull; @@ -50,11 +51,7 @@ 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); + final Catalog catalog = new RESTCatalog(); // Configure the properties map from the Iceberg instructions. if (!Strings.isNullOrEmpty(accessKeyId) && !Strings.isNullOrEmpty(secretAccessKey)) { @@ -67,13 +64,7 @@ public static IcebergCatalogAdapter createS3Rest( if (!Strings.isNullOrEmpty(endpointOverride)) { properties.put(S3FileIOProperties.ENDPOINT, endpointOverride); } - - final FileIO fileIO = CatalogUtil.loadFileIO(S3_FILE_IO_CLASS, properties, null); - - final String catalogName = name != null ? name : "IcebergCatalog-" + catalogURI; - catalog.initialize(catalogName, properties); - - return new IcebergCatalogAdapter(catalog, fileIO); + return createAdapterCommon(name, catalogURI, warehouseLocation, catalog, properties); } /** @@ -95,13 +86,22 @@ public static IcebergCatalogAdapter createGlue( // Set up the properties map for the Iceberg catalog final Map properties = new HashMap<>(); - final GlueCatalog catalog = new GlueCatalog(); + final Catalog catalog = new GlueCatalog(); + 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); final FileIO fileIO = CatalogUtil.loadFileIO(S3_FILE_IO_CLASS, properties, null); + properties.put(CatalogProperties.FILE_IO_IMPL, fileIO.getClass().getName()); final String catalogName = name != null ? name : "IcebergCatalog-" + catalogURI; catalog.initialize(catalogName, 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 44a17942cdf..9e31711c24e 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 @@ -12,11 +12,13 @@ import io.deephaven.extensions.s3.S3Instructions; import io.deephaven.iceberg.TestCatalog.IcebergTestCatalog; import io.deephaven.iceberg.TestCatalog.IcebergTestFileIO; +import junit.framework.TestCase; import org.apache.iceberg.Snapshot; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.types.Type; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -30,6 +32,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; @@ -186,7 +189,7 @@ public void tearDown() throws ExecutionException, InterruptedException { } @Test - public void testListNamespaces() { + void testListNamespaces() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); final Collection namespaces = adapter.listNamespaces(); @@ -203,7 +206,7 @@ public void testListNamespaces() { } @Test - public void testListTables() { + void testListTables() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); final Namespace ns = Namespace.of("sales"); @@ -227,7 +230,7 @@ public void testListTables() { } @Test - public void testListSnapshots() { + void testListSnapshots() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); final TLongArrayList snapshotIds = new TLongArrayList(); @@ -253,7 +256,7 @@ public void testListSnapshots() { } @Test - public void testOpenTableA() throws ExecutionException, InterruptedException, TimeoutException { + void testOpenTableA() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); final IcebergCatalogAdapter adapter = @@ -276,7 +279,7 @@ public void testOpenTableA() throws ExecutionException, InterruptedException, Ti } @Test - public void testOpenTableB() throws ExecutionException, InterruptedException, TimeoutException { + void testOpenTableB() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesMulti(); final IcebergCatalogAdapter adapter = @@ -298,7 +301,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 = @@ -321,7 +324,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, resourceFileIO); @@ -336,7 +339,7 @@ public void testOpenTableS3Only() throws ExecutionException, InterruptedExceptio } @Test - public void testOpenTableDefinition() throws ExecutionException, InterruptedException, TimeoutException { + void testOpenTableDefinition() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); final IcebergInstructions localInstructions = IcebergInstructions.builder() @@ -357,7 +360,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(), @@ -392,7 +395,7 @@ public void testOpenTablePartitionTypeException() { } @Test - public void testOpenTableDefinitionRename() throws ExecutionException, InterruptedException, TimeoutException { + void testOpenTableDefinitionRename() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); final TableDefinition renamed = TableDefinition.of( @@ -429,7 +432,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( @@ -459,7 +462,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( @@ -489,7 +492,7 @@ public void testReorderedPartitioningColumn() throws ExecutionException, Interru } @Test - public void testZeroPartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { + void testZeroPartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); final IcebergInstructions localInstructions = IcebergInstructions.builder() @@ -510,7 +513,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(), @@ -546,7 +549,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 @@ -581,7 +584,7 @@ public void testMissingPartitioningColumns() { } @Test - public void testOpenTableColumnRename() throws ExecutionException, InterruptedException, TimeoutException { + void testOpenTableColumnRename() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); final IcebergInstructions localInstructions = IcebergInstructions.builder() @@ -602,7 +605,7 @@ public void testOpenTableColumnRename() throws ExecutionException, InterruptedEx } @Test - public void testOpenTableColumnLegalization() throws ExecutionException, InterruptedException, TimeoutException { + void testOpenTableColumnLegalization() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesRenamed(); final IcebergInstructions localInstructions = IcebergInstructions.builder() @@ -622,7 +625,7 @@ public void testOpenTableColumnLegalization() throws ExecutionException, Interru } @Test - public void testOpenTableColumnLegalizationRename() + void testOpenTableColumnLegalizationRename() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesRenamed(); @@ -652,7 +655,7 @@ public void testOpenTableColumnLegalizationRename() } @Test - public void testOpenTableColumnLegalizationPartitionException() { + void testOpenTableColumnLegalizationPartitionException() { final TableDefinition tableDef = TableDefinition.of( ColumnDefinition.ofInt("Year").withPartitioning(), ColumnDefinition.ofInt("Month").withPartitioning()); @@ -684,7 +687,7 @@ public void testOpenTableColumnLegalizationPartitionException() { } @Test - public void testOpenTableColumnRenamePartitioningColumns() + void testOpenTableColumnRenamePartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); @@ -717,7 +720,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, resourceFileIO); @@ -749,7 +752,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, resourceFileIO); @@ -796,7 +799,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, resourceFileIO); @@ -811,7 +814,7 @@ public void testOpenAllTypesTable() throws ExecutionException, InterruptedExcept } @Test - public void testTableDefinition() { + void testTableDefinition() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); final Namespace ns = Namespace.of("sales"); @@ -836,7 +839,7 @@ public void testTableDefinition() { } @Test - public void testTableDefinitionTable() { + void testTableDefinitionTable() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); final Namespace ns = Namespace.of("sales"); @@ -869,7 +872,7 @@ public void testTableDefinitionTable() { } @Test - public void testTableDefinitionWithInstructions() { + void testTableDefinitionWithInstructions() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); IcebergInstructions localInstructions = IcebergInstructions.builder() @@ -908,4 +911,84 @@ public void testTableDefinitionWithInstructions() { tableDef = adapter.getTableDefinition("sales.sales_multi", localInstructions); Assert.equals(tableDef, "tableDef", userTableDef); } + + // TODO Add this back when we have a simulator for Iceberg table + // @Test + // void testAddSnapshot() throws ExecutionException, InterruptedException, TimeoutException { + // uploadSalesMulti(); + // + // final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + // + // final Namespace ns = Namespace.of("sales"); + // final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); + // final List snapshots = adapter.listSnapshots(tableId); + // Assert.eq(snapshots.size(), "snapshots.size()", 4, "4 snapshots for sales/sales_multi"); + // final io.deephaven.engine.table.Table dhTable = adapter.readTable(tableId, icebergInstructions); + // Assert.eq(dhTable.size(), "dhTable.size()", 100_000, "100_000 rows in the table"); + // + // // Read the first snapshot + // final io.deephaven.engine.table.Table dhTableFirstSnapshot = + // adapter.readTable(tableId, snapshots.get(0).snapshotId(), icebergInstructions); + // Assert.eq(dhTableFirstSnapshot.size(), "table0.size()", 18266, "18266 rows in the table"); + // Assert.equals(dhTableFirstSnapshot.getDefinition(), "table0.getDefinition()", SALES_MULTI_DEFINITION); + // + // // Append these rows back to the iceberg table + // final ParquetInstructions parquetInstructions = ParquetInstructions.builder() + // .setSpecialInstructions(s3Instructions) + // .build(); + // adapter.appendPartition(tableId, dhTableFirstSnapshot, parquetInstructions); + // + // // Verify we have added a new snapshot + // final List newSnapshots = adapter.listSnapshots(tableId); + // Assert.eq(newSnapshots.size(), "newSnapshots.size()", 5, "5 snapshots for sales/sales_multi"); + // + // // Verify we have added the right number of rows + // final io.deephaven.engine.table.Table dhTableNewSnapshot = + // adapter.readTable(tableId, newSnapshots.get(4).snapshotId(), icebergInstructions); + // Assert.eq(dhTableNewSnapshot.size(), "table0.size()", 18266, "18266 rows in the table"); + // + // // Verify the overall size of the table + // final io.deephaven.engine.table.Table dhTableUpdated = adapter.readTable(tableId, icebergInstructions); + // Assert.eq(dhTableUpdated.size(), "dhTableUpdated.size()", 100_000 + 18266, "100_000 + 18266 rows in the table"); + // } + + @Test + void testAddSnapshotWrongSchema() throws ExecutionException, InterruptedException, TimeoutException { + uploadSalesMulti(); + + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + + final Namespace ns = Namespace.of("sales"); + final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); + final io.deephaven.engine.table.Table dhTable = adapter.readTable(tableId, instructions); + Assert.eq(dhTable.size(), "dhTable.size()", 100_000, "100_000 rows in the table"); + + // Drop one column and append to original table + final io.deephaven.engine.table.Table updatedDhTable = dhTable.dropColumns("Region"); + try { + adapter.appendTable(tableId, updatedDhTable, instructions); + TestCase.fail(); + } catch (final IllegalArgumentException e) { + Assert.eqTrue(e.getMessage().contains("Schema mismatch"), "Exception message"); + } + } + + @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, BigDecimal.class, byte[].class + }; + + for (final Class javaType : javaTypes) { + // Java type -> Iceberg type + final Type icebergType = IcebergCatalogAdapter.convertToIcebergType(javaType); + + // Iceberg type -> Deephaven type + final io.deephaven.qst.type.Type deephavenType = IcebergCatalogAdapter.convertToDHType(icebergType); + + // Deephaven type == Java type + Assert.eq(javaType, javaType.getName(), deephavenType.clazz(), deephavenType.clazz().getName()); + } + } } 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 4955ca9223b..753d458db97 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 @@ -3,6 +3,7 @@ // package io.deephaven.iceberg.util; +import io.deephaven.UncheckedDeephavenException; import io.deephaven.api.util.NameValidator; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.*; @@ -22,8 +23,13 @@ import io.deephaven.iceberg.layout.IcebergKeyValuePartitionedLayout; import io.deephaven.iceberg.location.IcebergTableLocationFactory; import io.deephaven.iceberg.location.IcebergTableLocationKey; +import io.deephaven.parquet.table.ParquetInstructions; +import io.deephaven.parquet.table.ParquetTools; import io.deephaven.time.DateTimeUtils; import io.deephaven.util.annotations.VisibleForTesting; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -32,14 +38,18 @@ 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.apache.iceberg.io.FileIO; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.math.BigDecimal; import java.time.Instant; import java.time.LocalDateTime; +import java.time.LocalDate; +import java.time.LocalTime; import java.util.*; import java.util.stream.Collectors; @@ -112,7 +122,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(); @@ -155,7 +165,7 @@ private static TableDefinition fromSchema( * @param icebergType The Iceberg data type to be converted. * @return The converted Deephaven type. */ - static io.deephaven.qst.type.Type convertPrimitiveType(@NotNull final Type icebergType) { + static io.deephaven.qst.type.Type convertToDHType(@NotNull final Type icebergType) { final Type.TypeID typeId = icebergType.typeId(); switch (typeId) { case BOOLEAN: @@ -176,23 +186,57 @@ static io.deephaven.qst.type.Type convertPrimitiveType(@NotNull final Type ic ? 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); + return io.deephaven.qst.type.Type.find(LocalDate.class); case TIME: - return io.deephaven.qst.type.Type.find(java.time.LocalTime.class); + return io.deephaven.qst.type.Type.find(LocalTime.class); case DECIMAL: - return io.deephaven.qst.type.Type.find(java.math.BigDecimal.class); + // TODO We know the precision and scale of the decimal, but we're not using it here. + 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 LIST: // Fall through // TODO Add support for lists case MAP: // Fall through default: throw new TableDataException("Unsupported iceberg column type " + typeId.name()); } } + @VisibleForTesting + static Type convertToIcebergType(final Class columnType) { + if (columnType == Boolean.class) { + return Types.BooleanType.get(); + } else if (columnType == double.class) { + return Types.DoubleType.get(); + } else if (columnType == float.class) { + return Types.FloatType.get(); + } else if (columnType == int.class) { + return Types.IntegerType.get(); + } else if (columnType == long.class) { + return Types.LongType.get(); + } else if (columnType == String.class) { + return Types.StringType.get(); + } else if (columnType == Instant.class) { + return Types.TimestampType.withZone(); + } else if (columnType == LocalDateTime.class) { + return Types.TimestampType.withoutZone(); + } else if (columnType == LocalDate.class) { + return Types.DateType.get(); + } else if (columnType == LocalTime.class) { + return Types.TimeType.get(); + } else if (columnType == BigDecimal.class) { + // TODO Compute precision and scale from the table and use that for parquet writing and appending + return Types.DecimalType.of(38, 18); + } else if (columnType == byte[].class) { + return Types.BinaryType.get(); + } else { + throw new TableDataException("Unsupported deephaven column type " + columnType.getName()); + } + // TODO Add support for writing lists, for reading too + } + /** * 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)}. @@ -627,7 +671,6 @@ private TableDefinition getTableDefinitionInternal( * @param instructions The instructions for customizations while reading * @return The loaded table */ - @SuppressWarnings("unused") public Table readTable( @NotNull final TableIdentifier tableIdentifier, @Nullable final IcebergInstructions instructions) { @@ -641,7 +684,6 @@ public Table readTable( * @param instructions The instructions for customizations while reading * @return The loaded table */ - @SuppressWarnings("unused") public Table readTable( @NotNull final String tableIdentifier, @Nullable final IcebergInstructions instructions) { @@ -656,7 +698,6 @@ public Table readTable( * @param instructions The instructions for customizations while reading * @return The loaded table */ - @SuppressWarnings("unused") public Table readTable( @NotNull final TableIdentifier tableIdentifier, final long tableSnapshotId, @@ -677,7 +718,6 @@ public Table readTable( * @param instructions The instructions for customizations while reading * @return The loaded table */ - @SuppressWarnings("unused") public Table readTable( @NotNull final String tableIdentifier, final long tableSnapshotId, @@ -693,7 +733,6 @@ public Table readTable( * @param instructions The instructions for customizations while reading * @return The loaded table */ - @SuppressWarnings("unused") public Table readTable( @NotNull final TableIdentifier tableIdentifier, @NotNull final Snapshot tableSnapshot, @@ -780,4 +819,352 @@ public Catalog catalog() { public FileIO fileIO() { return fileIO; } + + /** + * Write a Deephaven table to a new Iceberg table in the catalog. + * + * @param dhTable The Deephaven table to write + * @param namespace The namespace for the new table, will be created if it does not exist + * @param tableName The name of the new table + */ + public void writeTable( + @NotNull final Table dhTable, + @NotNull final String namespace, + @NotNull final String tableName) { + writeTable(dhTable, namespace, tableName, IcebergParquetWriteInstructions.DEFAULT); + } + + /** + * Write a Deephaven table to a new Iceberg table in the catalog. + * + * @param dhTable The Deephaven table to write + * @param namespace The namespace for the new table, will be created if it does not exist + * @param tableName The name of the new table + * @param instructions The instructions for customizations while writing + */ + public void writeTable( + @NotNull final Table dhTable, + @NotNull final String namespace, + @NotNull final String tableName, + @NotNull final IcebergInstructions instructions) { + verifyWriteInstructions(instructions); + final Namespace ns = createNamespaceIfDoesntExist(namespace); + final TableIdentifier tableIdentifier = TableIdentifier.of(ns, tableName); + final TableDefinition useDefinition = instructions.tableDefinition().orElse(dhTable.getDefinition()); + final SpecAndSchema specAndSchema = fromTableDefinition(useDefinition, instructions); + final org.apache.iceberg.Table icebergTable = createNewIcebergTable(tableIdentifier, specAndSchema); + try { + writeParquet(icebergTable, dhTable, instructions); + } catch (final Exception writeException) { + // If we fail to write the table, we should delete the table to avoid leaving a partial table in the catalog + try { + catalog.dropTable(tableIdentifier, true); + } catch (final Exception dropException) { + writeException.addSuppressed(dropException); + } + throw writeException; + } + + } + + /** + * Append provided deephaven table as a new partition to the existing iceberg table in a single snapshot. + * + * @param icebergTableIdentifier The identifier for the iceberg table to append to + * @param dhTable The deephaven table to append + */ + public void appendTable( + @NotNull final String icebergTableIdentifier, + @NotNull final Table dhTable) { + appendTable(TableIdentifier.parse(icebergTableIdentifier), dhTable, IcebergParquetWriteInstructions.DEFAULT); + } + + /** + * Append provided deephaven table as a new partition to the existing iceberg table in a single snapshot. + * + * @param icebergTableIdentifier The identifier for the iceberg table to append to + * @param dhTable The deephaven table to append + * @param instructions The instructions for customizations while writing + */ + public void appendTable( + @NotNull final String icebergTableIdentifier, + @NotNull final Table dhTable, + @Nullable final IcebergInstructions instructions) { + appendTable(TableIdentifier.parse(icebergTableIdentifier), dhTable, instructions); + } + + /** + * Append provided deephaven table as a new partition to the existing iceberg table in a single snapshot. + * + * @param icebergTableIdentifier The identifier for the iceberg table to append to + * @param dhTable The deephaven table to append + */ + public void appendTable( + @NotNull final TableIdentifier icebergTableIdentifier, + @NotNull final Table dhTable) { + appendTable(icebergTableIdentifier, dhTable, IcebergParquetWriteInstructions.DEFAULT); + } + + /** + * Append provided deephaven table as a new partition to the existing iceberg table in a single snapshot. + * + * @param icebergTableIdentifier The identifier for the iceberg table to append to + * @param dhTable The deephaven table to append + * @param instructions The instructions for customizations while writing + */ + public void appendTable( + @NotNull final TableIdentifier icebergTableIdentifier, + @NotNull final Table dhTable, + @NotNull final IcebergInstructions instructions) { + verifyWriteInstructions(instructions); + // Load the table from the catalog. + final org.apache.iceberg.Table icebergTable = catalog.loadTable(icebergTableIdentifier); + if (icebergTable == null) { + throw new IllegalArgumentException("Table not found: " + icebergTableIdentifier); + } + + // Make sure spec and schema for the iceberg and deephaven table is identical + final TableDefinition useDefinition = instructions.tableDefinition().orElse(dhTable.getDefinition()); + final SpecAndSchema specAndSchema = fromTableDefinition(useDefinition, instructions); + if (!icebergTable.schema().sameSchema(specAndSchema.schema)) { + throw new IllegalArgumentException("Schema mismatch, iceberg table schema " + + icebergTable.schema() + ", schema derived from deephaven table: " + specAndSchema.schema); + } + if (!icebergTable.spec().compatibleWith(specAndSchema.partitionSpec)) { + throw new IllegalArgumentException("Partition spec mismatch, iceberg table partition spec " + + icebergTable.spec() + ", partition spec derived from deephaven table: " + + specAndSchema.partitionSpec); + } + writeParquet(icebergTable, dhTable, instructions); + } + + public void writePartitionedTable( + @NotNull final Table dhTable, + @NotNull final String namespace, + @NotNull final String tableName, + @NotNull final IcebergInstructions instructions) { + verifyWriteInstructions(instructions); + final Namespace ns = createNamespaceIfDoesntExist(namespace); + final TableIdentifier tableIdentifier = TableIdentifier.of(ns, tableName); + final TableDefinition useDefinition = instructions.tableDefinition().orElse(dhTable.getDefinition()); + if (useDefinition.getPartitioningColumns().isEmpty()) { + throw new IllegalArgumentException("Table must have partitioning columns to write partitioned data"); + } + final SpecAndSchema specAndSchema = fromTableDefinition(useDefinition, instructions); + writePartitionedTableImpl(dhTable, tableIdentifier, specAndSchema, instructions); + } + + public void writePartitionedTable( + @NotNull final PartitionedTable dhTable, + @NotNull final String namespace, + @NotNull final String tableName) { + writePartitionedTable(dhTable, namespace, tableName, IcebergParquetWriteInstructions.DEFAULT); + } + + public void writePartitionedTable( + @NotNull final PartitionedTable dhTable, + @NotNull final String namespace, + @NotNull final String tableName, + @NotNull final IcebergInstructions instructions) { + verifyWriteInstructions(instructions); + final Namespace ns = createNamespaceIfDoesntExist(namespace); + final TableIdentifier tableIdentifier = TableIdentifier.of(ns, tableName); + final SpecAndSchema specAndSchema; + if (instructions.tableDefinition().isPresent()) { + specAndSchema = fromTableDefinition(instructions.tableDefinition().get(), instructions); + } else { + specAndSchema = forPartitionedTable(dhTable, instructions); + } + writePartitionedTableImpl(dhTable, tableIdentifier, specAndSchema, instructions); + } + + private static void verifyWriteInstructions(@NotNull final IcebergInstructions instructions) { + // We ony support writing to Parquet files + if (!(instructions instanceof IcebergParquetWriteInstructions)) { + throw new IllegalArgumentException("Unsupported instructions of class " + instructions.getClass() + " for" + + " writing Iceberg table, expected: " + IcebergParquetWriteInstructions.class); + } + } + + private Namespace createNamespaceIfDoesntExist(@NotNull final String namespace) { + final Namespace ns = Namespace.of(namespace); + if (catalog instanceof SupportsNamespaces) { + final SupportsNamespaces nsCatalog = (SupportsNamespaces) catalog; + if (!nsCatalog.namespaceExists(ns)) { + nsCatalog.createNamespace(ns); + } + } + return ns; + } + + private static class SpecAndSchema { + private final PartitionSpec partitionSpec; + private final Schema schema; + + private SpecAndSchema(final PartitionSpec partitionSpec, final Schema schema) { + this.partitionSpec = partitionSpec; + this.schema = schema; + } + } + + /** + * Create {@link PartitionSpec} and {@link Schema} from a {@link TableDefinition}. + */ + private static SpecAndSchema fromTableDefinition( + @NotNull final TableDefinition tableDefinition, + @NotNull final IcebergInstructions instructions) { + final Collection partitioningColumnNames = new ArrayList<>(); + final List fields = new ArrayList<>(); + int fieldID = 1; // Iceberg field IDs start from 1 + // Create the schema + for (final ColumnDefinition columnDefinition : tableDefinition.getColumns()) { + final String dhColumnName = columnDefinition.getName(); + // TODO Check with others how column name renames should work for writing + final String icebergColName = instructions.columnRenames().getOrDefault(dhColumnName, dhColumnName); + final Type icebergType = convertToIcebergType(columnDefinition.getDataType()); + fields.add(Types.NestedField.optional(fieldID, icebergColName, icebergType)); + if (columnDefinition.isPartitioning()) { + partitioningColumnNames.add(icebergColName); + } + fieldID++; + } + final Schema schema = new Schema(fields); + final PartitionSpec partitionSpec = createPartitionSpec(schema, partitioningColumnNames); + return new SpecAndSchema(partitionSpec, schema); + } + + private static SpecAndSchema forPartitionedTable( + @NotNull final PartitionedTable partitionedTable, + @NotNull final IcebergInstructions instructions) { + // TODO Look at the duplication + final List fields = new ArrayList<>(); + int fieldID = 1; // Iceberg field IDs start from 1 + // Create the schema + final TableDefinition partitionedTableDefinition = partitionedTable.table().getDefinition(); + final Set keyColumnNames = partitionedTable.keyColumnNames(); + for (final String keyColumnName : keyColumnNames) { + final ColumnDefinition keyColumnDefinition = partitionedTableDefinition.getColumn(keyColumnName); + final String icebergColName = instructions.columnRenames().getOrDefault(keyColumnName, keyColumnName); + final Type icebergType = convertToIcebergType(keyColumnDefinition.getDataType()); + fields.add(Types.NestedField.optional(fieldID, icebergColName, icebergType)); + fieldID++; + } + final TableDefinition constituentDefinition = partitionedTable.constituentDefinition(); + for (final ColumnDefinition leafColumnDefinition : constituentDefinition.getColumns()) { + final String dhColumnName = leafColumnDefinition.getName(); + if (keyColumnNames.contains(dhColumnName)) { + continue; + } + final String icebergColName = instructions.columnRenames().getOrDefault(dhColumnName, dhColumnName); + final Type icebergType = convertToIcebergType(leafColumnDefinition.getDataType()); + fields.add(Types.NestedField.optional(fieldID, icebergColName, icebergType)); + fieldID++; + } + final Schema schema = new Schema(fields); + final PartitionSpec partitionSpec = createPartitionSpec(schema, keyColumnNames); + return new SpecAndSchema(partitionSpec, schema); + } + + private org.apache.iceberg.Table createNewIcebergTable( + @NotNull final TableIdentifier tableIdentifier, + @NotNull final SpecAndSchema specAndSchema) { + try { + return catalog.createTable(tableIdentifier, specAndSchema.schema, specAndSchema.partitionSpec); + } catch (final AlreadyExistsException e) { + throw new IllegalArgumentException("Table already exists: " + tableIdentifier); + } + } + + private static void writeParquet( + @NotNull final org.apache.iceberg.Table icebergTable, + @NotNull final Table dhTable, + @NotNull final IcebergInstructions instructions) { + if (dhTable.isEmpty()) { + return; + } + // Generate a unique path for the new partition and write the data to it + final String newDataLocation = icebergTable.locationProvider().newDataLocation(UUID.randomUUID() + ".parquet"); + final List parquetFilesWritten = new ArrayList<>(1); + final ParquetInstructions parquetInstructions = ((IcebergParquetWriteInstructions) instructions) + .toParquetInstructions(parquetFilesWritten); + ParquetTools.writeTable(dhTable, newDataLocation, parquetInstructions); + createSnapshot(icebergTable, parquetFilesWritten, false); + } + + private static void createSnapshot( + @NotNull final org.apache.iceberg.Table icebergTable, + @NotNull final Collection parquetFilesWritten, + final boolean isPartitioned) { + if (parquetFilesWritten.isEmpty()) { + throw new UncheckedDeephavenException("Failed to create a snapshot because no parquet files were written"); + } + // Append new data files to the table + final AppendFiles append = icebergTable.newAppend(); + for (final ParquetInstructions.CompletedWrite parquetFileWritten : parquetFilesWritten) { + final String filePath = parquetFileWritten.destination().toString(); + final DataFiles.Builder dfBuilder = DataFiles.builder(icebergTable.spec()) + .withPath(filePath) + .withFormat(FileFormat.PARQUET) + .withRecordCount(parquetFileWritten.numRows()) + .withFileSizeInBytes(parquetFileWritten.numBytes()); + if (isPartitioned) { + // TODO Find the partition path properly + final String tableDataLocation = icebergTable.location() + "/data/"; + final String partitionPath = filePath.substring(tableDataLocation.length(), filePath.lastIndexOf('/')); + dfBuilder.withPartitionPath(partitionPath); + } + append.appendFile(dfBuilder.build()); + } + // Commit the changes to create a new snapshot + append.commit(); + } + + private void writePartitionedTableImpl( + @NotNull final TABLE dhTable, + @NotNull final TableIdentifier tableIdentifier, + @NotNull final SpecAndSchema specAndSchema, + @NotNull final IcebergInstructions instructions) { + final org.apache.iceberg.Table icebergTable = createNewIcebergTable(tableIdentifier, specAndSchema); + try { + final List parquetFilesWritten = new ArrayList<>(); + final ParquetInstructions parquetInstructions = ((IcebergParquetWriteInstructions) instructions) + .toParquetInstructions(parquetFilesWritten); + // TODO Find data location properly + final String destinationDir = icebergTable.location() + "/data"; + if (dhTable instanceof PartitionedTable) { + // TODO This duplicated code doesn't look good, do something about it + final PartitionedTable partitionedTable = (PartitionedTable) dhTable; + if (partitionedTable.table().isEmpty()) { + return; + } + ParquetTools.writeKeyValuePartitionedTable(partitionedTable, destinationDir, parquetInstructions); + } else { + final Table table = (Table) dhTable; + if (table.isEmpty()) { + return; + } + ParquetTools.writeKeyValuePartitionedTable(table, destinationDir, parquetInstructions); + } + createSnapshot(icebergTable, parquetFilesWritten, true); + } catch (final Exception writeException) { + // If we fail to write the table, we should delete the table to avoid leaving a partial table in the catalog + try { + catalog.dropTable(tableIdentifier, true); + } catch (final Exception dropException) { + writeException.addSuppressed(dropException); + } + throw writeException; + } + } + + private static PartitionSpec createPartitionSpec( + @NotNull final Schema schema, + @NotNull final Iterable partitionColumnNames) { + // Create the partition spec + final PartitionSpec.Builder partitionSpecBuilder = PartitionSpec.builderFor(schema); + for (final String partitioningColumnName : partitionColumnNames) { + partitionSpecBuilder.identity(partitioningColumnName); + } + return partitionSpecBuilder.build(); + } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java index b595b4cfd14..33b7393eaf8 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java @@ -11,15 +11,15 @@ import java.util.Optional; /** - * This class provides instructions intended for reading Iceberg catalogs and tables. The default values documented in - * this class may change in the future. As such, callers may wish to explicitly set the values. + * This class provides instructions intended for reading/writing Iceberg catalogs and tables. The default values + * documented in this class may change in the future. As such, callers may wish to explicitly set the values. */ @Immutable @BuildableStyle public abstract class IcebergInstructions { /** - * The default {@link IcebergInstructions} to use when reading Iceberg data files. Providing this will use system - * defaults for cloud provider-specific parameters + * The default {@link IcebergInstructions} to use when reading/writing Iceberg data files. Providing this will use + * system defaults for cloud provider-specific parameters. */ @SuppressWarnings("unused") public static final IcebergInstructions DEFAULT = builder().build(); @@ -29,19 +29,19 @@ public static Builder builder() { } /** - * The {@link TableDefinition} to use when reading Iceberg data files. + * The {@link TableDefinition} to use when reading/writing Iceberg data files. */ public abstract Optional tableDefinition(); /** - * The data instructions to use for reading the Iceberg data files (might be S3Instructions or other cloud + * 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(); /** - * A {@link Map map} of rename instructions from Iceberg to Deephaven column names to use when reading the Iceberg - * data files. + * A {@link Map map} of rename instructions from Iceberg to Deephaven column names to use when reading/writing the + * Iceberg data files. */ public abstract Map columnRenames(); diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java new file mode 100644 index 00000000000..fbf2048e0c2 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java @@ -0,0 +1,119 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.util; + +import io.deephaven.annotations.BuildableStyle; +import io.deephaven.parquet.table.ParquetInstructions; +import org.immutables.value.Value.Default; +import org.immutables.value.Value.Immutable; +import org.jetbrains.annotations.NotNull; + +import java.util.List; +import java.util.Map; + +/** + * This class provides instructions intended for writing Iceberg tables as Parquet data files. + */ +// TODO Check with Devin +// I am getting a warning here +// (immutables:subtype) Should not inherit io.deephaven.iceberg.util.IcebergInstructions which is a value type +// itself. Avoid extending from another abstract value type. Better to share common abstract class or interface which +// are not carrying @Immutable annotation. If still extending from immutable abstract type be ready to face some +// incoherences in generated types. +@Immutable +@BuildableStyle +public abstract class IcebergParquetWriteInstructions extends IcebergInstructions { + /** + * The default {@link IcebergInstructions} to use when reading/writing Iceberg tables as Parquet data files. + */ + @SuppressWarnings("unused") + public static final IcebergParquetWriteInstructions DEFAULT = builder().build(); + + public static Builder builder() { + return ImmutableIcebergParquetWriteInstructions.builder(); + } + + /** + * The name of the compression codec to use when writing Parquet files; defaults to + * {@value ParquetInstructions#DEFAULT_COMPRESSION_CODEC_NAME}. + */ + @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. + */ + @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. + */ + @Default + public int maximumDictionarySize() { + return ParquetInstructions.DEFAULT_MAXIMUM_DICTIONARY_SIZE; + } + + /** + * The target page size for writing the parquet files; defaults to + * {@value ParquetInstructions#DEFAULT_TARGET_PAGE_SIZE}. + */ + @Default + public int getTargetPageSize() { + return ParquetInstructions.DEFAULT_TARGET_PAGE_SIZE; + } + + /** + * Convert this {@link IcebergParquetWriteInstructions} to a {@link ParquetInstructions}. + * + * @param completedWrites List of completed writes to be set in the {@link ParquetInstructions} + */ + public ParquetInstructions toParquetInstructions( + @NotNull final List completedWrites) { + final ParquetInstructions.Builder builder = new ParquetInstructions.Builder(); + + tableDefinition().ifPresent(builder::setTableDefinition); + dataInstructions().ifPresent(builder::setSpecialInstructions); + + // Add any column rename mappings. + if (!columnRenames().isEmpty()) { + for (final Map.Entry entry : columnRenames().entrySet()) { + builder.addColumnNameMapping(entry.getKey(), entry.getValue()); + } + } + + // Add parquet writing specific instructions. + builder.setCompressionCodecName(compressionCodecName()); + builder.setMaximumDictionaryKeys(maximumDictionaryKeys()); + builder.setMaximumDictionarySize(maximumDictionarySize()); + builder.setTargetPageSize(getTargetPageSize()); + builder.setCompletedWrites(completedWrites); + + return builder.build(); + } + + public interface Builder extends IcebergInstructions.Builder { + @SuppressWarnings("unused") + Builder compressionCodecName(String compressionCodecName); + + @SuppressWarnings("unused") + Builder maximumDictionaryKeys(int maximumDictionaryKeys); + + @SuppressWarnings("unused") + Builder maximumDictionarySize(int maximumDictionarySize); + + @SuppressWarnings("unused") + Builder targetPageSize(int targetPageSize); + + IcebergParquetWriteInstructions build(); + } +} 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..d6e02910241 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 @@ -45,7 +45,7 @@ public final class ParquetFileWriter implements AutoCloseable { public ParquetFileWriter( final URI dest, - final OutputStream destOutputStream, + final CountingOutputStream countingOutput, final int targetPageSize, final ByteBufferAllocator allocator, final MessageType type, @@ -55,7 +55,7 @@ public ParquetFileWriter( this.targetPageSize = targetPageSize; this.allocator = allocator; this.extraMetaData = new HashMap<>(extraMetaData); - this.countingOutput = new CountingOutputStream(destOutputStream); + this.countingOutput = countingOutput; countingOutput.write(MAGIC); this.type = type; this.compressorAdapter = DeephavenCompressorAdapterFactory.getInstance().getByName(codecName); 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 069ddd4dac6..e992c875f39 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 @@ -11,10 +11,10 @@ import io.deephaven.hash.KeyedObjectKey; import io.deephaven.parquet.base.ParquetUtils; import io.deephaven.util.annotations.VisibleForTesting; -import org.apache.parquet.hadoop.metadata.CompressionCodecName; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.net.URI; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -33,95 +33,18 @@ */ public abstract class ParquetInstructions implements ColumnToCodecMappings { - private static volatile String defaultCompressionCodecName = CompressionCodecName.SNAPPY.toString(); - - /** - * Set the default for {@link #getCompressionCodecName()}. - * - * @deprecated Use {@link Builder#setCompressionCodecName(String)} instead. - * @param name The new default - */ - @Deprecated - public static void setDefaultCompressionCodecName(final String name) { - defaultCompressionCodecName = name; - } - - /** - * @return The default for {@link #getCompressionCodecName()} - */ - public static String getDefaultCompressionCodecName() { - return defaultCompressionCodecName; - } - - private static volatile int defaultMaximumDictionaryKeys = 1 << 20; - - /** - * Set the default for {@link #getMaximumDictionaryKeys()}. - * - * @param maximumDictionaryKeys The new default - * @see Builder#setMaximumDictionaryKeys(int) - */ - public static void setDefaultMaximumDictionaryKeys(final int maximumDictionaryKeys) { - defaultMaximumDictionaryKeys = Require.geqZero(maximumDictionaryKeys, "maximumDictionaryKeys"); - } - - /** - * @return The default for {@link #getMaximumDictionaryKeys()} - */ - public static int getDefaultMaximumDictionaryKeys() { - return defaultMaximumDictionaryKeys; - } - - private static volatile int defaultMaximumDictionarySize = 1 << 20; - - /** - * Set the default for {@link #getMaximumDictionarySize()}. - * - * @param maximumDictionarySize The new default - * @see Builder#setMaximumDictionarySize(int) - */ - public static void setDefaultMaximumDictionarySize(final int maximumDictionarySize) { - defaultMaximumDictionarySize = Require.geqZero(maximumDictionarySize, "maximumDictionarySize"); - } - - /** - * @return The default for {@link #getMaximumDictionarySize()} - */ - public static int getDefaultMaximumDictionarySize() { - return defaultMaximumDictionarySize; - } - - public static final int MIN_TARGET_PAGE_SIZE = - Configuration.getInstance().getIntegerWithDefault("Parquet.minTargetPageSize", 1 << 11); // 2KB - private static final int DEFAULT_TARGET_PAGE_SIZE = - Configuration.getInstance().getIntegerWithDefault("Parquet.defaultTargetPageSize", 1 << 16); // 64KB - private static volatile int defaultTargetPageSize = DEFAULT_TARGET_PAGE_SIZE; + public static final String DEFAULT_COMPRESSION_CODEC_NAME = "SNAPPY"; + public static final int DEFAULT_MAXIMUM_DICTIONARY_KEYS = 1 << 20; + public static final int DEFAULT_MAXIMUM_DICTIONARY_SIZE = 1 << 20; + public static final int MIN_TARGET_PAGE_SIZE = 1 << 11; // 2KB + private static final int minTargetPageSize = Configuration.getInstance().getIntegerWithDefault( + "Parquet.minTargetPageSize", MIN_TARGET_PAGE_SIZE); + public static final int DEFAULT_TARGET_PAGE_SIZE = 1 << 16; // 64KB + private static final int defaultTargetPageSize = Configuration.getInstance().getIntegerWithDefault( + "Parquet.defaultTargetPageSize", DEFAULT_TARGET_PAGE_SIZE); private static final boolean DEFAULT_IS_REFRESHING = false; - /** - * Set the default target page size (in bytes) used to section rows of data into pages during column writing. This - * number should be no smaller than {@link #MIN_TARGET_PAGE_SIZE}. - * - * @param newDefaultSizeBytes the new default target page size. - */ - public static void setDefaultTargetPageSize(final int newDefaultSizeBytes) { - if (newDefaultSizeBytes < MIN_TARGET_PAGE_SIZE) { - throw new IllegalArgumentException( - "Default target page size should be larger than " + MIN_TARGET_PAGE_SIZE + " bytes"); - } - defaultTargetPageSize = newDefaultSizeBytes; - } - - /** - * Get the current default target page size in bytes. - * - * @return the current default target page size in bytes. - */ - public static int getDefaultTargetPageSize() { - return defaultTargetPageSize; - } - public enum ParquetFileLayout { /** * A single parquet file. @@ -251,6 +174,36 @@ public abstract ParquetInstructions withTableDefinitionAndLayout(final TableDefi */ public abstract String baseNameForPartitionedParquetData(); + public static class CompletedWrite { + private final URI destination; + private final long numRows; + private final long numBytes; + + CompletedWrite(final URI destination, final long numRows, final long numBytes) { + this.destination = destination; + this.numRows = numRows; + this.numBytes = numBytes; + } + + public URI destination() { + return destination; + } + + public long numRows() { + return numRows; + } + + public long numBytes() { + return numBytes; + } + } + + /** + * @return A list to be populated with information about all the parquet files written, ignoring the index and + * metadata files. + */ + public abstract Optional> completedWrites(); + @VisibleForTesting public static boolean sameColumnNamesAndCodecMappings(final ParquetInstructions i1, final ParquetInstructions i2) { if (i1 == EMPTY) { @@ -302,17 +255,17 @@ public Object getSpecialInstructions() { @Override public String getCompressionCodecName() { - return defaultCompressionCodecName; + return DEFAULT_COMPRESSION_CODEC_NAME; } @Override public int getMaximumDictionaryKeys() { - return defaultMaximumDictionaryKeys; + return DEFAULT_MAXIMUM_DICTIONARY_KEYS; } @Override public int getMaximumDictionarySize() { - return defaultMaximumDictionarySize; + return DEFAULT_MAXIMUM_DICTIONARY_SIZE; } @Override @@ -372,7 +325,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 @@ -380,7 +333,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> completedWrites() { + return Optional.empty(); } }; @@ -455,6 +413,7 @@ private static final class ReadOnly extends ParquetInstructions { private final ParquetFileLayout fileLayout; private final TableDefinition tableDefinition; private final Collection> indexColumns; + private final List completedWrites; private ReadOnly( final KeyedObjectHashMap columnNameToInstructions, @@ -470,7 +429,8 @@ private ReadOnly( final String baseNameForPartitionedParquetData, final ParquetFileLayout fileLayout, final TableDefinition tableDefinition, - final Collection> indexColumns) { + final Collection> indexColumns, + final List completedWrites) { this.columnNameToInstructions = columnNameToInstructions; this.parquetColumnNameToInstructions = parquetColumnNameToColumnName; this.compressionCodecName = compressionCodecName; @@ -488,6 +448,7 @@ private ReadOnly( : indexColumns.stream() .map(List::copyOf) .collect(Collectors.toUnmodifiableList()); + this.completedWrites = completedWrites; } private String getOrDefault(final String columnName, final String defaultValue, @@ -625,7 +586,7 @@ public ParquetInstructions withTableDefinitionAndLayout( getCompressionCodecName(), getMaximumDictionaryKeys(), getMaximumDictionarySize(), isLegacyParquet(), getTargetPageSize(), isRefreshing(), getSpecialInstructions(), generateMetadataFiles(), baseNameForPartitionedParquetData(), useLayout, useDefinition, - indexColumns); + indexColumns, completedWrites); } @Override @@ -634,7 +595,12 @@ ParquetInstructions withIndexColumns(final Collection> useIndexColu getCompressionCodecName(), getMaximumDictionaryKeys(), getMaximumDictionarySize(), isLegacyParquet(), getTargetPageSize(), isRefreshing(), getSpecialInstructions(), generateMetadataFiles(), baseNameForPartitionedParquetData(), fileLayout, - tableDefinition, useIndexColumns); + tableDefinition, useIndexColumns, completedWrites); + } + + @Override + public Optional> completedWrites() { + return Optional.ofNullable(completedWrites); } KeyedObjectHashMap copyColumnNameToInstructions() { @@ -681,9 +647,9 @@ public static class Builder { // We only store entries in parquetColumnNameToInstructions when the parquetColumnName is // different than the columnName (ie, the column name mapping is not the default mapping) private KeyedObjectHashMap parquetColumnNameToInstructions; - private String compressionCodecName = defaultCompressionCodecName; - private int maximumDictionaryKeys = defaultMaximumDictionaryKeys; - private int maximumDictionarySize = defaultMaximumDictionarySize; + private String compressionCodecName = DEFAULT_COMPRESSION_CODEC_NAME; + private int maximumDictionaryKeys = DEFAULT_MAXIMUM_DICTIONARY_KEYS; + private int maximumDictionarySize = DEFAULT_MAXIMUM_DICTIONARY_SIZE; private boolean isLegacyParquet; private int targetPageSize = defaultTargetPageSize; private boolean isRefreshing = DEFAULT_IS_REFRESHING; @@ -693,6 +659,7 @@ public static class Builder { private ParquetFileLayout fileLayout; private TableDefinition tableDefinition; private Collection> indexColumns; + private List completedWrites; /** * For each additional field added, make sure to update the copy constructor builder @@ -720,6 +687,7 @@ public Builder(final ParquetInstructions parquetInstructions) { fileLayout = readOnlyParquetInstructions.getFileLayout().orElse(null); tableDefinition = readOnlyParquetInstructions.getTableDefinition().orElse(null); indexColumns = readOnlyParquetInstructions.getIndexColumns().orElse(null); + completedWrites = readOnlyParquetInstructions.completedWrites().orElse(null); } private void newColumnNameToInstructionsMap() { @@ -869,8 +837,8 @@ public Builder setIsLegacyParquet(final boolean isLegacyParquet) { } public Builder setTargetPageSize(final int targetPageSize) { - if (targetPageSize < MIN_TARGET_PAGE_SIZE) { - throw new IllegalArgumentException("Target page size should be >= " + MIN_TARGET_PAGE_SIZE); + if (targetPageSize < minTargetPageSize) { + throw new IllegalArgumentException("Target page size should be >= " + minTargetPageSize); } this.targetPageSize = targetPageSize; return this; @@ -987,6 +955,11 @@ public Builder addAllIndexColumns(final Iterable> indexColumns) { return this; } + public Builder setCompletedWrites(final List completedWrites) { + this.completedWrites = completedWrites; + return this; + } + public ParquetInstructions build() { final KeyedObjectHashMap columnNameToInstructionsOut = columnNameToInstructions; columnNameToInstructions = null; @@ -996,7 +969,7 @@ public ParquetInstructions build() { return new ReadOnly(columnNameToInstructionsOut, parquetColumnNameToColumnNameOut, compressionCodecName, maximumDictionaryKeys, maximumDictionarySize, isLegacyParquet, targetPageSize, isRefreshing, specialInstructions, generateMetadataFiles, baseNameForPartitionedParquetData, fileLayout, - tableDefinition, indexColumns); + tableDefinition, indexColumns, completedWrites); } } 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..40200b48358 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 @@ -3,6 +3,7 @@ // package io.deephaven.parquet.table; +import com.google.common.io.CountingOutputStream; import io.deephaven.api.SortColumn; import io.deephaven.engine.liveness.LivenessScopeStack; import io.deephaven.engine.rowset.RowSet; @@ -108,10 +109,11 @@ static class IndexWritingInfo { * impacting both schema and written data, we store results in computedCache to avoid having to calculate * twice. An example is the necessary precision and scale for a BigDecimal column written as a decimal * logical type. + * @return The number of bytes written for the table (excluding indexes) * * @throws IOException For file writing related errors */ - static void write( + static long write( @NotNull final Table t, @NotNull final TableDefinition definition, @NotNull final ParquetInstructions writeInstructions, @@ -173,8 +175,8 @@ static void write( if (!sortedColumns.isEmpty()) { tableInfoBuilder.addSortingColumns(SortColumnInfo.of(sortedColumns.get(0))); } - write(t, definition, writeInstructions, dest, destOutputStream, incomingMeta, - tableInfoBuilder, metadataFileWriter, computedCache); + return write(t, definition, writeInstructions, dest, destOutputStream, incomingMeta, tableInfoBuilder, + metadataFileWriter, computedCache); } /** @@ -191,9 +193,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 +211,15 @@ private static void write( final Table t = pretransformTable(table, definition); final TrackingRowSet tableRowSet = t.getRowSet(); final Map> columnSourceMap = t.getColumnSourceMap(); + final CountingOutputStream countingOutput = new CountingOutputStream(destOutputStream); try (final ParquetFileWriter parquetFileWriter = getParquetFileWriter(computedCache, definition, - tableRowSet, columnSourceMap, dest, destOutputStream, writeInstructions, tableMeta, + tableRowSet, columnSourceMap, dest, countingOutput, writeInstructions, tableMeta, tableInfoBuilder, metadataFileWriter)) { // Given the transformation, do not use the original table's "definition" for writing write(t, writeInstructions, parquetFileWriter, computedCache); } destOutputStream.done(); + return countingOutput.getCount(); } } @@ -317,7 +323,7 @@ private static Table pretransformTable(@NotNull final Table table, @NotNull fina * @param tableRowSet The row set being written * @param columnSourceMap The columns of the table * @param dest The destination URI to write to - * @param destOutputStream The output stream to write to dest + * @param countingOutput The output stream to write to dest * @param writeInstructions Write instructions for the file * @param tableMeta Metadata to include in the parquet metadata * @param tableInfoBuilder Builder for accumulating per-column information to construct the deephaven metadata @@ -332,7 +338,7 @@ private static ParquetFileWriter getParquetFileWriter( @NotNull final RowSet tableRowSet, @NotNull final Map> columnSourceMap, @NotNull final URI dest, - @NotNull final OutputStream destOutputStream, + @NotNull final CountingOutputStream countingOutput, @NotNull final ParquetInstructions writeInstructions, @NotNull final Map tableMeta, @NotNull final TableInfo.Builder tableInfoBuilder, @@ -378,7 +384,7 @@ private static ParquetFileWriter getParquetFileWriter( final Map extraMetaData = new HashMap<>(tableMeta); extraMetaData.put(METADATA_KEY, tableInfoBuilder.build().serializeToJSON()); - return new ParquetFileWriter(dest, destOutputStream, writeInstructions.getTargetPageSize(), + return new ParquetFileWriter(dest, countingOutput, writeInstructions.getTargetPageSize(), new HeapByteBufferAllocator(), mappedSchema.getParquetSchema(), writeInstructions.getCompressionCodecName(), extraMetaData, metadataFileWriter); } 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 ab035bdd0c8..790a0c299a2 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 @@ -587,6 +587,14 @@ private static void writeTablesImpl( metadataFileWriter = NullParquetMetadataFileWriter.INSTANCE; } + final boolean collectCompletedWrites = writeInstructions.completedWrites().isPresent(); + final List completedWrites; + if (collectCompletedWrites) { + completedWrites = new ArrayList<>(destinations.length); + } else { + completedWrites = null; + } + // List of output streams created, to rollback in case of exceptions final List outputStreams = new ArrayList<>(destinations.length); try (final SafeCloseable ignored = () -> SafeCloseable.closeAll(outputStreams.stream())) { @@ -598,9 +606,13 @@ private static void writeTablesImpl( final CompletableOutputStream outputStream = channelsProvider.getOutputStream( destinations[tableIdx], PARQUET_OUTPUT_BUFFER_SIZE); outputStreams.add(outputStream); - ParquetTableWriter.write(source, definition, writeInstructions, destinations[tableIdx], - outputStream, Collections.emptyMap(), (List) null, - metadataFileWriter, computedCache); + final long numBytes = ParquetTableWriter.write(source, definition, writeInstructions, + destinations[tableIdx], outputStream, Collections.emptyMap(), + (List) null, metadataFileWriter, computedCache); + if (collectCompletedWrites) { + completedWrites.add(new ParquetInstructions.CompletedWrite(destinations[tableIdx], + source.size(), numBytes)); + } } } else { // Shared parquet column names across all tables @@ -622,8 +634,13 @@ private static void writeTablesImpl( outputStreams.add(info.destOutputStream); } final Table sourceTable = sources[tableIdx]; - ParquetTableWriter.write(sourceTable, definition, writeInstructions, destinations[tableIdx], - outputStream, Collections.emptyMap(), indexInfoList, metadataFileWriter, computedCache); + final long numBytes = ParquetTableWriter.write(sourceTable, definition, writeInstructions, + tableDestination, outputStream, Collections.emptyMap(), indexInfoList, + metadataFileWriter, computedCache); + if (collectCompletedWrites) { + completedWrites.add(new ParquetInstructions.CompletedWrite(tableDestination, + sourceTable.size(), numBytes)); + } } } @@ -652,9 +669,18 @@ private static void writeTablesImpl( log.error().append("Error in rolling back output stream ").append(e1).endl(); } } + if (completedWrites != null) { + completedWrites.clear(); + } throw new UncheckedDeephavenException("Error writing parquet tables", e); } } + if (collectCompletedWrites && !completedWrites.isEmpty()) { + // Add the completed writes to the list provided in the instructions + final List userCompletedWritesList = + writeInstructions.completedWrites().get(); + userCompletedWritesList.addAll(completedWrites); + } } /** diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/transfer/TransferObject.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/transfer/TransferObject.java index e87eff5aa85..95403c9b007 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/transfer/TransferObject.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/transfer/TransferObject.java @@ -39,7 +39,7 @@ static TransferObject create( @NotNull final Map> computedCache, @NotNull final String columnName, @NotNull final ColumnSource columnSource) { - Class columnType = columnSource.getType(); + final Class columnType = columnSource.getType(); if (columnType == int.class) { return IntTransfer.create(columnSource, tableRowSet, instructions.getTargetPageSize()); } diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3CompletableOutputStream.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3CompletableOutputStream.java index 43004a6ba70..1db1a3c2957 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3CompletableOutputStream.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3CompletableOutputStream.java @@ -344,8 +344,8 @@ private void completeMultipartUpload() throws IOException { */ private void abortMultipartUpload() throws IOException { if (uploadId == null) { - throw new IllegalStateException("Cannot abort multipart upload for uri " + uri + " because upload ID " + - "is null"); + // Upload not started, or already completed/aborted + return; } final AbortMultipartUploadRequest abortRequest = AbortMultipartUploadRequest.builder() .bucket(uri.bucket().orElseThrow()) From 758c1f350eb5bfe08abfe3e43786c6ca135a2c9c Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Wed, 28 Aug 2024 14:16:44 -0500 Subject: [PATCH 02/28] Added type info map and modified instructions class hierarchy --- .../iceberg/util/IcebergToolsTest.java | 78 +++-------- .../iceberg/layout/IcebergBaseLayout.java | 6 +- .../iceberg/layout/IcebergFlatLayout.java | 4 +- .../IcebergKeyValuePartitionedLayout.java | 4 +- .../iceberg/util/IcebergBaseInstructions.java | 45 ++++++ .../iceberg/util/IcebergCatalogAdapter.java | 131 ++++++++++++------ .../iceberg/util/IcebergInstructions.java | 48 ++----- .../util/IcebergParquetWriteInstructions.java | 17 +-- .../parquet/base/ParquetFileWriter.java | 1 - 9 files changed, 177 insertions(+), 157 deletions(-) create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergBaseInstructions.java 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 9e31711c24e..4e0bb1f6a9d 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 @@ -98,7 +98,8 @@ public abstract class IcebergToolsTest { ColumnDefinition.ofString("ColumnType"), ColumnDefinition.ofBoolean("IsPartitioning")); - IcebergInstructions instructions; + IcebergBaseInstructions instructions; + IcebergBaseInstructions writeInstructions; public abstract S3AsyncClient s3AsyncClient(); @@ -130,6 +131,9 @@ void setUp() throws ExecutionException, InterruptedException { instructions = IcebergInstructions.builder() .dataInstructions(s3Instructions) .build(); + writeInstructions = IcebergParquetWriteInstructions.builder() + .dataInstructions(s3Instructions) + .build(); } private void uploadParquetFiles(final File root, final String prefixToRemove) @@ -342,7 +346,7 @@ void testOpenTableS3Only() throws ExecutionException, InterruptedException, Time void testOpenTableDefinition() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(SALES_PARTITIONED_DEFINITION) .dataInstructions(instructions.dataInstructions().get()) .build(); @@ -370,7 +374,7 @@ void testOpenTablePartitionTypeException() { ColumnDefinition.ofLong("Unit_Price"), ColumnDefinition.ofTime("Order_Date")); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) .dataInstructions(instructions.dataInstructions().get()) .build(); @@ -407,7 +411,7 @@ void testOpenTableDefinitionRename() throws ExecutionException, InterruptedExcep ColumnDefinition.ofDouble("UnitPrice"), ColumnDefinition.ofTime("OrderDate")); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(renamed) .dataInstructions(instructions.dataInstructions().get()) .putColumnRenames("Region", "RegionName") @@ -444,7 +448,7 @@ void testSkippedPartitioningColumn() throws ExecutionException, InterruptedExcep ColumnDefinition.ofDouble("Unit_Price"), ColumnDefinition.ofTime("Order_Date")); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) .dataInstructions(instructions.dataInstructions().get()) .build(); @@ -474,7 +478,7 @@ void testReorderedPartitioningColumn() throws ExecutionException, InterruptedExc ColumnDefinition.ofDouble("Unit_Price"), ColumnDefinition.ofTime("Order_Date")); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) .dataInstructions(instructions.dataInstructions().get()) .build(); @@ -495,7 +499,7 @@ void testReorderedPartitioningColumn() throws ExecutionException, InterruptedExc void testZeroPartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(SALES_MULTI_DEFINITION) .dataInstructions(instructions.dataInstructions().get()) .build(); @@ -523,7 +527,7 @@ void testIncorrectPartitioningColumns() throws ExecutionException, InterruptedEx ColumnDefinition.ofDouble("Unit_Price"), ColumnDefinition.ofTime("Order_Date")); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) .dataInstructions(instructions.dataInstructions().get()) .build(); @@ -559,7 +563,7 @@ void testMissingPartitioningColumns() { ColumnDefinition.ofLong("Unit_Price"), ColumnDefinition.ofTime("Order_Date")); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) .dataInstructions(instructions.dataInstructions().get()) .build(); @@ -587,7 +591,7 @@ void testMissingPartitioningColumns() { void testOpenTableColumnRename() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() .dataInstructions(instructions.dataInstructions().get()) .putColumnRenames("Region", "RegionName") .putColumnRenames("Item_Type", "ItemType") @@ -608,7 +612,7 @@ void testOpenTableColumnRename() throws ExecutionException, InterruptedException void testOpenTableColumnLegalization() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesRenamed(); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() .dataInstructions(instructions.dataInstructions().get()) .build(); @@ -629,7 +633,7 @@ void testOpenTableColumnLegalizationRename() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesRenamed(); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() .dataInstructions(instructions.dataInstructions().get()) .putColumnRenames("Item&Type", "Item_Type") .putColumnRenames("Units/Sold", "Units_Sold") @@ -660,7 +664,7 @@ void testOpenTableColumnLegalizationPartitionException() { ColumnDefinition.ofInt("Year").withPartitioning(), ColumnDefinition.ofInt("Month").withPartitioning()); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) .putColumnRenames("Year", "Current Year") .putColumnRenames("Month", "Current Month") @@ -691,7 +695,7 @@ void testOpenTableColumnRenamePartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() .dataInstructions(instructions.dataInstructions().get()) .putColumnRenames("VendorID", "vendor_id") .putColumnRenames("month", "__month") @@ -875,7 +879,7 @@ void testTableDefinitionTable() { void testTableDefinitionWithInstructions() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); - IcebergInstructions localInstructions = IcebergInstructions.builder() + IcebergBaseInstructions localInstructions = IcebergInstructions.builder() .dataInstructions(instructions.dataInstructions().get()) .putColumnRenames("Region", "Area") .putColumnRenames("Item_Type", "ItemType") @@ -912,46 +916,6 @@ void testTableDefinitionWithInstructions() { Assert.equals(tableDef, "tableDef", userTableDef); } - // TODO Add this back when we have a simulator for Iceberg table - // @Test - // void testAddSnapshot() throws ExecutionException, InterruptedException, TimeoutException { - // uploadSalesMulti(); - // - // final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); - // - // final Namespace ns = Namespace.of("sales"); - // final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); - // final List snapshots = adapter.listSnapshots(tableId); - // Assert.eq(snapshots.size(), "snapshots.size()", 4, "4 snapshots for sales/sales_multi"); - // final io.deephaven.engine.table.Table dhTable = adapter.readTable(tableId, icebergInstructions); - // Assert.eq(dhTable.size(), "dhTable.size()", 100_000, "100_000 rows in the table"); - // - // // Read the first snapshot - // final io.deephaven.engine.table.Table dhTableFirstSnapshot = - // adapter.readTable(tableId, snapshots.get(0).snapshotId(), icebergInstructions); - // Assert.eq(dhTableFirstSnapshot.size(), "table0.size()", 18266, "18266 rows in the table"); - // Assert.equals(dhTableFirstSnapshot.getDefinition(), "table0.getDefinition()", SALES_MULTI_DEFINITION); - // - // // Append these rows back to the iceberg table - // final ParquetInstructions parquetInstructions = ParquetInstructions.builder() - // .setSpecialInstructions(s3Instructions) - // .build(); - // adapter.appendPartition(tableId, dhTableFirstSnapshot, parquetInstructions); - // - // // Verify we have added a new snapshot - // final List newSnapshots = adapter.listSnapshots(tableId); - // Assert.eq(newSnapshots.size(), "newSnapshots.size()", 5, "5 snapshots for sales/sales_multi"); - // - // // Verify we have added the right number of rows - // final io.deephaven.engine.table.Table dhTableNewSnapshot = - // adapter.readTable(tableId, newSnapshots.get(4).snapshotId(), icebergInstructions); - // Assert.eq(dhTableNewSnapshot.size(), "table0.size()", 18266, "18266 rows in the table"); - // - // // Verify the overall size of the table - // final io.deephaven.engine.table.Table dhTableUpdated = adapter.readTable(tableId, icebergInstructions); - // Assert.eq(dhTableUpdated.size(), "dhTableUpdated.size()", 100_000 + 18266, "100_000 + 18266 rows in the table"); - // } - @Test void testAddSnapshotWrongSchema() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesMulti(); @@ -960,13 +924,13 @@ void testAddSnapshotWrongSchema() throws ExecutionException, InterruptedExceptio final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); - final io.deephaven.engine.table.Table dhTable = adapter.readTable(tableId, instructions); + final io.deephaven.engine.table.Table dhTable = adapter.readTable(tableId, writeInstructions); Assert.eq(dhTable.size(), "dhTable.size()", 100_000, "100_000 rows in the table"); // Drop one column and append to original table final io.deephaven.engine.table.Table updatedDhTable = dhTable.dropColumns("Region"); try { - adapter.appendTable(tableId, updatedDhTable, instructions); + adapter.appendTable(tableId, updatedDhTable, writeInstructions); TestCase.fail(); } catch (final IllegalArgumentException e) { Assert.eqTrue(e.getMessage().contains("Schema mismatch"), "Exception message"); 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 f5334cf866c..5c00f60dfe3 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 @@ -9,7 +9,7 @@ import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; import io.deephaven.iceberg.location.IcebergTableLocationKey; import io.deephaven.iceberg.location.IcebergTableParquetLocationKey; -import io.deephaven.iceberg.util.IcebergInstructions; +import io.deephaven.iceberg.util.IcebergBaseInstructions; import io.deephaven.parquet.table.ParquetInstructions; import org.apache.iceberg.*; import org.apache.iceberg.io.FileIO; @@ -46,7 +46,7 @@ public abstract class IcebergBaseLayout implements TableLocationKeyFinder tableDefinition(); + + /** + * The data instructions to use for reading/writing the Iceberg data files (might be S3Instructions or other cloud + * provider-specific instructions). + */ + Optional dataInstructions(); + + /** + * A {@link Map map} of rename instructions from Iceberg to Deephaven column names to use when reading/writing the + * Iceberg data files. + */ + Map columnRenames(); + + interface Builder { + @SuppressWarnings("unused") + Builder tableDefinition(TableDefinition tableDefinition); + + @SuppressWarnings("unused") + Builder dataInstructions(Object s3Instructions); + + @SuppressWarnings("unused") + Builder putColumnRenames(String key, String value); + + @SuppressWarnings("unused") + Builder putAllColumnRenames(Map entries); + + IcebergBaseInstructions build(); + } +} 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 753d458db97..76f97f3bae4 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 @@ -3,6 +3,10 @@ // package io.deephaven.iceberg.util; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.ObjectNode; import io.deephaven.UncheckedDeephavenException; import io.deephaven.api.util.NameValidator; import io.deephaven.engine.rowset.RowSetFactory; @@ -27,6 +31,7 @@ import io.deephaven.parquet.table.ParquetTools; import io.deephaven.time.DateTimeUtils; import io.deephaven.util.annotations.VisibleForTesting; +import org.apache.hadoop.util.StringUtils; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFiles; import org.apache.iceberg.FileFormat; @@ -34,11 +39,11 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +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.apache.iceberg.io.FileIO; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; @@ -74,6 +79,8 @@ public class IcebergCatalogAdapter { ColumnDefinition.fromGenericType("Summary", Map.class), ColumnDefinition.fromGenericType("SnapshotObject", Snapshot.class)); + private static final String DEFAULT_GENERATED_FILE_FORMAT = "parquet"; + private final Catalog catalog; private final FileIO fileIO; @@ -462,7 +469,7 @@ private Snapshot getSnapshot(@NotNull final TableIdentifier tableIdentifier, fin private Map getRenameColumnMap( @NotNull final org.apache.iceberg.Table table, @NotNull final Schema schema, - @NotNull final IcebergInstructions instructions) { + @NotNull final IcebergBaseInstructions instructions) { final Set takenNames = new HashSet<>(); @@ -507,7 +514,7 @@ private Map getRenameColumnMap( */ public TableDefinition getTableDefinition( @NotNull final String tableIdentifier, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergBaseInstructions instructions) { final TableIdentifier tableId = TableIdentifier.parse(tableIdentifier); // Load the table from the catalog. return getTableDefinition(tableId, instructions); @@ -523,7 +530,7 @@ public TableDefinition getTableDefinition( */ public TableDefinition getTableDefinition( @NotNull final TableIdentifier tableIdentifier, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergBaseInstructions instructions) { // Load the table from the catalog. return getTableDefinitionInternal(tableIdentifier, null, instructions); } @@ -580,7 +587,7 @@ public TableDefinition getTableDefinition( */ public Table getTableDefinitionTable( @NotNull final String tableIdentifier, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergBaseInstructions instructions) { final TableIdentifier tableId = TableIdentifier.parse(tableIdentifier); return getTableDefinitionTable(tableId, instructions); } @@ -595,7 +602,7 @@ public Table getTableDefinitionTable( */ public Table getTableDefinitionTable( @NotNull final TableIdentifier tableIdentifier, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergBaseInstructions instructions) { final TableDefinition definition = getTableDefinition(tableIdentifier, instructions); return TableTools.metaTable(definition); } @@ -647,7 +654,7 @@ public Table getTableDefinitionTable( private TableDefinition getTableDefinitionInternal( @NotNull final TableIdentifier tableIdentifier, @Nullable final Snapshot tableSnapshot, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergBaseInstructions instructions) { final org.apache.iceberg.Table table = catalog.loadTable(tableIdentifier); if (table == null) { throw new IllegalArgumentException("Table not found: " + tableIdentifier); @@ -656,7 +663,8 @@ private TableDefinition getTableDefinitionInternal( final Snapshot snapshot = tableSnapshot != null ? tableSnapshot : table.currentSnapshot(); final Schema schema = snapshot != null ? table.schemas().get(snapshot.schemaId()) : table.schema(); - final IcebergInstructions userInstructions = instructions == null ? IcebergInstructions.DEFAULT : instructions; + final IcebergBaseInstructions userInstructions = + instructions == null ? IcebergInstructions.DEFAULT : instructions; return fromSchema(schema, table.spec(), @@ -673,7 +681,7 @@ private TableDefinition getTableDefinitionInternal( */ public Table readTable( @NotNull final TableIdentifier tableIdentifier, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergBaseInstructions instructions) { return readTableInternal(tableIdentifier, null, instructions); } @@ -686,7 +694,7 @@ public Table readTable( */ public Table readTable( @NotNull final String tableIdentifier, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergBaseInstructions instructions) { return readTable(TableIdentifier.parse(tableIdentifier), instructions); } @@ -701,7 +709,7 @@ public Table readTable( public Table readTable( @NotNull final TableIdentifier tableIdentifier, final long tableSnapshotId, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergBaseInstructions instructions) { // Find the snapshot with the given snapshot id final Snapshot tableSnapshot = getSnapshot(tableIdentifier, tableSnapshotId); if (tableSnapshot == null) { @@ -721,7 +729,7 @@ public Table readTable( public Table readTable( @NotNull final String tableIdentifier, final long tableSnapshotId, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergBaseInstructions instructions) { return readTable(TableIdentifier.parse(tableIdentifier), tableSnapshotId, instructions); } @@ -736,14 +744,14 @@ public Table readTable( public Table readTable( @NotNull final TableIdentifier tableIdentifier, @NotNull final Snapshot tableSnapshot, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergBaseInstructions instructions) { return readTableInternal(tableIdentifier, tableSnapshot, instructions); } private Table readTableInternal( @NotNull final TableIdentifier tableIdentifier, @Nullable final Snapshot tableSnapshot, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergBaseInstructions instructions) { // Load the table from the catalog. final org.apache.iceberg.Table table = catalog.loadTable(tableIdentifier); if (table == null) { @@ -758,7 +766,8 @@ private Table readTableInternal( final org.apache.iceberg.PartitionSpec partitionSpec = table.spec(); // Get default instructions if none are provided - final IcebergInstructions userInstructions = instructions == null ? IcebergInstructions.DEFAULT : instructions; + final IcebergBaseInstructions userInstructions = + instructions == null ? IcebergInstructions.DEFAULT : instructions; // Get the user supplied table definition. final TableDefinition userTableDef = userInstructions.tableDefinition().orElse(null); @@ -846,15 +855,16 @@ public void writeTable( @NotNull final Table dhTable, @NotNull final String namespace, @NotNull final String tableName, - @NotNull final IcebergInstructions instructions) { - verifyWriteInstructions(instructions); + @NotNull final IcebergBaseInstructions instructions) { + final IcebergParquetWriteInstructions writeInstructions = verifyWriteInstructions(instructions); final Namespace ns = createNamespaceIfDoesntExist(namespace); final TableIdentifier tableIdentifier = TableIdentifier.of(ns, tableName); final TableDefinition useDefinition = instructions.tableDefinition().orElse(dhTable.getDefinition()); final SpecAndSchema specAndSchema = fromTableDefinition(useDefinition, instructions); - final org.apache.iceberg.Table icebergTable = createNewIcebergTable(tableIdentifier, specAndSchema); + final org.apache.iceberg.Table icebergTable = + createNewIcebergTable(tableIdentifier, specAndSchema, writeInstructions); try { - writeParquet(icebergTable, dhTable, instructions); + writeParquet(icebergTable, dhTable, writeInstructions); } catch (final Exception writeException) { // If we fail to write the table, we should delete the table to avoid leaving a partial table in the catalog try { @@ -889,7 +899,7 @@ public void appendTable( public void appendTable( @NotNull final String icebergTableIdentifier, @NotNull final Table dhTable, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergBaseInstructions instructions) { appendTable(TableIdentifier.parse(icebergTableIdentifier), dhTable, instructions); } @@ -915,8 +925,8 @@ public void appendTable( public void appendTable( @NotNull final TableIdentifier icebergTableIdentifier, @NotNull final Table dhTable, - @NotNull final IcebergInstructions instructions) { - verifyWriteInstructions(instructions); + @NotNull final IcebergBaseInstructions instructions) { + final IcebergParquetWriteInstructions writeInstructions = verifyWriteInstructions(instructions); // Load the table from the catalog. final org.apache.iceberg.Table icebergTable = catalog.loadTable(icebergTableIdentifier); if (icebergTable == null) { @@ -935,15 +945,15 @@ public void appendTable( + icebergTable.spec() + ", partition spec derived from deephaven table: " + specAndSchema.partitionSpec); } - writeParquet(icebergTable, dhTable, instructions); + writeParquet(icebergTable, dhTable, writeInstructions); } public void writePartitionedTable( @NotNull final Table dhTable, @NotNull final String namespace, @NotNull final String tableName, - @NotNull final IcebergInstructions instructions) { - verifyWriteInstructions(instructions); + @NotNull final IcebergBaseInstructions instructions) { + final IcebergParquetWriteInstructions writeInstructions = verifyWriteInstructions(instructions); final Namespace ns = createNamespaceIfDoesntExist(namespace); final TableIdentifier tableIdentifier = TableIdentifier.of(ns, tableName); final TableDefinition useDefinition = instructions.tableDefinition().orElse(dhTable.getDefinition()); @@ -951,7 +961,7 @@ public void writePartitionedTable( throw new IllegalArgumentException("Table must have partitioning columns to write partitioned data"); } final SpecAndSchema specAndSchema = fromTableDefinition(useDefinition, instructions); - writePartitionedTableImpl(dhTable, tableIdentifier, specAndSchema, instructions); + writePartitionedTableImpl(dhTable, tableIdentifier, specAndSchema, writeInstructions); } public void writePartitionedTable( @@ -965,8 +975,8 @@ public void writePartitionedTable( @NotNull final PartitionedTable dhTable, @NotNull final String namespace, @NotNull final String tableName, - @NotNull final IcebergInstructions instructions) { - verifyWriteInstructions(instructions); + @NotNull final IcebergBaseInstructions instructions) { + final IcebergParquetWriteInstructions writeInstructions = verifyWriteInstructions(instructions); final Namespace ns = createNamespaceIfDoesntExist(namespace); final TableIdentifier tableIdentifier = TableIdentifier.of(ns, tableName); final SpecAndSchema specAndSchema; @@ -975,15 +985,17 @@ public void writePartitionedTable( } else { specAndSchema = forPartitionedTable(dhTable, instructions); } - writePartitionedTableImpl(dhTable, tableIdentifier, specAndSchema, instructions); + writePartitionedTableImpl(dhTable, tableIdentifier, specAndSchema, writeInstructions); } - private static void verifyWriteInstructions(@NotNull final IcebergInstructions instructions) { + private static IcebergParquetWriteInstructions verifyWriteInstructions( + @NotNull final IcebergBaseInstructions instructions) { // We ony support writing to Parquet files if (!(instructions instanceof IcebergParquetWriteInstructions)) { throw new IllegalArgumentException("Unsupported instructions of class " + instructions.getClass() + " for" + " writing Iceberg table, expected: " + IcebergParquetWriteInstructions.class); } + return (IcebergParquetWriteInstructions) instructions; } private Namespace createNamespaceIfDoesntExist(@NotNull final String namespace) { @@ -1012,7 +1024,7 @@ private SpecAndSchema(final PartitionSpec partitionSpec, final Schema schema) { */ private static SpecAndSchema fromTableDefinition( @NotNull final TableDefinition tableDefinition, - @NotNull final IcebergInstructions instructions) { + @NotNull final IcebergBaseInstructions instructions) { final Collection partitioningColumnNames = new ArrayList<>(); final List fields = new ArrayList<>(); int fieldID = 1; // Iceberg field IDs start from 1 @@ -1035,7 +1047,7 @@ private static SpecAndSchema fromTableDefinition( private static SpecAndSchema forPartitionedTable( @NotNull final PartitionedTable partitionedTable, - @NotNull final IcebergInstructions instructions) { + @NotNull final IcebergBaseInstructions instructions) { // TODO Look at the duplication final List fields = new ArrayList<>(); int fieldID = 1; // Iceberg field IDs start from 1 @@ -1065,28 +1077,60 @@ private static SpecAndSchema forPartitionedTable( return new SpecAndSchema(partitionSpec, schema); } + /** + * Convert a map of column IDs to names to a JSON string. For example, the map {1 -> "A", 2 -> "B"} would be + * converted to: [{"field-id":1,"names":["A"]},{"field-id":2,"names":["B"]}] + */ + // TODO Check with others if there is a better way to do this + private static String convertIdToNameMapToJson(final Map idToNameMap) { + final ObjectMapper mapper = new ObjectMapper(); + final ArrayNode parentNode = mapper.createArrayNode(); + for (final Map.Entry columnIndo : idToNameMap.entrySet()) { + final ObjectNode columnNode = mapper.createObjectNode(); + columnNode.put("field-id", columnIndo.getKey()); + final ArrayNode namesArray = mapper.createArrayNode(); + namesArray.add(columnIndo.getValue()); + columnNode.set("names", namesArray); + parentNode.add(columnNode); + } + try { + return mapper.writeValueAsString(parentNode); + } catch (final JsonProcessingException e) { + throw new UncheckedDeephavenException("Failed to convert id to name map to JSON, map=" + idToNameMap, e); + } + } + private org.apache.iceberg.Table createNewIcebergTable( @NotNull final TableIdentifier tableIdentifier, - @NotNull final SpecAndSchema specAndSchema) { - try { - return catalog.createTable(tableIdentifier, specAndSchema.schema, specAndSchema.partitionSpec); - } catch (final AlreadyExistsException e) { + @NotNull final SpecAndSchema specAndSchema, + @NotNull final IcebergParquetWriteInstructions writeInstructions) { + // Check if a table with the same name already exists + if (catalog.tableExists(tableIdentifier)) { throw new IllegalArgumentException("Table already exists: " + tableIdentifier); } + // It is required to either set the column name mapping in the table properties or to provide the field IDs in + // the parquet file, to map the column names from the parquet file to the iceberg table schema. We are using the + // former approach here. + // TODO Check with larry if we are looking at this correctly while reading + final String columnNameMappingJson = convertIdToNameMapToJson(specAndSchema.schema.idToName()); + return catalog.createTable(tableIdentifier, specAndSchema.schema, specAndSchema.partitionSpec, null, + Map.of(TableProperties.DEFAULT_FILE_FORMAT, DEFAULT_GENERATED_FILE_FORMAT, + TableProperties.PARQUET_COMPRESSION, + StringUtils.toLowerCase(writeInstructions.compressionCodecName()), + TableProperties.DEFAULT_NAME_MAPPING, columnNameMappingJson)); } private static void writeParquet( @NotNull final org.apache.iceberg.Table icebergTable, @NotNull final Table dhTable, - @NotNull final IcebergInstructions instructions) { + @NotNull final IcebergParquetWriteInstructions writeInstructions) { if (dhTable.isEmpty()) { return; } // Generate a unique path for the new partition and write the data to it final String newDataLocation = icebergTable.locationProvider().newDataLocation(UUID.randomUUID() + ".parquet"); final List parquetFilesWritten = new ArrayList<>(1); - final ParquetInstructions parquetInstructions = ((IcebergParquetWriteInstructions) instructions) - .toParquetInstructions(parquetFilesWritten); + final ParquetInstructions parquetInstructions = writeInstructions.toParquetInstructions(parquetFilesWritten); ParquetTools.writeTable(dhTable, newDataLocation, parquetInstructions); createSnapshot(icebergTable, parquetFilesWritten, false); } @@ -1123,12 +1167,13 @@ private
void writePartitionedTableImpl( @NotNull final TABLE dhTable, @NotNull final TableIdentifier tableIdentifier, @NotNull final SpecAndSchema specAndSchema, - @NotNull final IcebergInstructions instructions) { - final org.apache.iceberg.Table icebergTable = createNewIcebergTable(tableIdentifier, specAndSchema); + @NotNull final IcebergParquetWriteInstructions writeInstructions) { + final org.apache.iceberg.Table icebergTable = + createNewIcebergTable(tableIdentifier, specAndSchema, writeInstructions); try { final List parquetFilesWritten = new ArrayList<>(); - final ParquetInstructions parquetInstructions = ((IcebergParquetWriteInstructions) instructions) - .toParquetInstructions(parquetFilesWritten); + final ParquetInstructions parquetInstructions = + writeInstructions.toParquetInstructions(parquetFilesWritten); // TODO Find data location properly final String destinationDir = icebergTable.location() + "/data"; if (dhTable instanceof PartitionedTable) { diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java index 33b7393eaf8..7b09667b8e5 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java @@ -4,22 +4,21 @@ package io.deephaven.iceberg.util; import io.deephaven.annotations.BuildableStyle; -import io.deephaven.engine.table.TableDefinition; import org.immutables.value.Value.Immutable; -import java.util.Map; -import java.util.Optional; - /** - * This class provides instructions intended for reading/writing Iceberg catalogs and tables. The default values - * documented in this class may change in the future. As such, callers may wish to explicitly set the values. + * This class provides instructions intended for reading Iceberg catalogs and tables. The default values documented in + * this class may change in the future. As such, callers may wish to explicitly set the values. */ @Immutable @BuildableStyle -public abstract class IcebergInstructions { +// TODO I propose renaming, but this will be breaking change: +// IcebergInstructions -> IcebergReadInstructions +// IcebergBaseInstructions -> IcebergInstructions +public abstract class IcebergInstructions implements IcebergBaseInstructions { /** - * The default {@link IcebergInstructions} to use when reading/writing Iceberg data files. Providing this will use - * system defaults for cloud provider-specific parameters. + * The default {@link IcebergInstructions} to use when reading Iceberg data files. Providing this will use system + * defaults for cloud provider-specific parameters. */ @SuppressWarnings("unused") public static final IcebergInstructions DEFAULT = builder().build(); @@ -28,36 +27,7 @@ public static Builder builder() { return ImmutableIcebergInstructions.builder(); } - /** - * The {@link TableDefinition} to use when reading/writing Iceberg data files. - */ - public abstract Optional 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(); - - /** - * A {@link Map map} of rename instructions from Iceberg to Deephaven column names to use when reading/writing the - * Iceberg data files. - */ - public abstract Map columnRenames(); - - public interface Builder { - @SuppressWarnings("unused") - Builder tableDefinition(TableDefinition tableDefinition); - - @SuppressWarnings("unused") - Builder dataInstructions(Object s3Instructions); - - @SuppressWarnings("unused") - Builder putColumnRenames(String key, String value); - - @SuppressWarnings("unused") - Builder putAllColumnRenames(Map entries); - + public interface Builder extends IcebergBaseInstructions.Builder { IcebergInstructions build(); } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java index fbf2048e0c2..fda0aac88f4 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java @@ -13,19 +13,16 @@ import java.util.Map; /** - * This class provides instructions intended for writing Iceberg tables as Parquet data files. + * This class provides instructions intended for writing Iceberg tables as Parquet data files as well as reading for + * reading Iceberg catalogs and tables. The default values documented in this class may change in the future. As such, + * callers may wish to explicitly set the values. */ -// TODO Check with Devin -// I am getting a warning here -// (immutables:subtype) Should not inherit io.deephaven.iceberg.util.IcebergInstructions which is a value type -// itself. Avoid extending from another abstract value type. Better to share common abstract class or interface which -// are not carrying @Immutable annotation. If still extending from immutable abstract type be ready to face some -// incoherences in generated types. @Immutable @BuildableStyle -public abstract class IcebergParquetWriteInstructions extends IcebergInstructions { +public abstract class IcebergParquetWriteInstructions implements IcebergBaseInstructions { /** - * The default {@link IcebergInstructions} to use when reading/writing Iceberg tables as Parquet data files. + * The default {@link IcebergParquetWriteInstructions} to use when reading/writing Iceberg tables as Parquet data + * files. */ @SuppressWarnings("unused") public static final IcebergParquetWriteInstructions DEFAULT = builder().build(); @@ -101,7 +98,7 @@ public ParquetInstructions toParquetInstructions( return builder.build(); } - public interface Builder extends IcebergInstructions.Builder { + public interface Builder extends IcebergBaseInstructions.Builder { @SuppressWarnings("unused") Builder compressionCodecName(String compressionCodecName); 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 d6e02910241..7a44b097137 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 @@ -18,7 +18,6 @@ import org.jetbrains.annotations.NotNull; import java.io.IOException; -import java.io.OutputStream; import java.net.URI; import java.util.ArrayList; import java.util.HashMap; From 48cb8d88c6ffbdd79f144ebba67d6b50a27a924d Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Wed, 28 Aug 2024 16:38:53 -0500 Subject: [PATCH 03/28] Minor tweaks to the instructions class hierarchy --- .../deephaven/iceberg/util/IcebergToolsTest.java | 4 ++-- .../iceberg/util/IcebergBaseInstructions.java | 16 +++++----------- .../iceberg/util/IcebergInstructions.java | 2 +- .../util/IcebergParquetWriteInstructions.java | 2 +- 4 files changed, 9 insertions(+), 15 deletions(-) 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 4e0bb1f6a9d..cc139c23523 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 @@ -98,8 +98,8 @@ public abstract class IcebergToolsTest { ColumnDefinition.ofString("ColumnType"), ColumnDefinition.ofBoolean("IsPartitioning")); - IcebergBaseInstructions instructions; - IcebergBaseInstructions writeInstructions; + private IcebergInstructions instructions; + private IcebergParquetWriteInstructions writeInstructions; public abstract S3AsyncClient s3AsyncClient(); diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergBaseInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergBaseInstructions.java index 932b055e64e..5ca4bb90320 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergBaseInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergBaseInstructions.java @@ -27,19 +27,13 @@ public interface IcebergBaseInstructions { */ Map columnRenames(); - interface Builder { - @SuppressWarnings("unused") - Builder tableDefinition(TableDefinition tableDefinition); + interface Builder { + INSTRUCTIONS_BUILDER tableDefinition(TableDefinition tableDefinition); - @SuppressWarnings("unused") - Builder dataInstructions(Object s3Instructions); + INSTRUCTIONS_BUILDER dataInstructions(Object s3Instructions); - @SuppressWarnings("unused") - Builder putColumnRenames(String key, String value); + INSTRUCTIONS_BUILDER putColumnRenames(String key, String value); - @SuppressWarnings("unused") - Builder putAllColumnRenames(Map entries); - - IcebergBaseInstructions build(); + INSTRUCTIONS_BUILDER putAllColumnRenames(Map entries); } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java index 7b09667b8e5..724e8987176 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java @@ -27,7 +27,7 @@ public static Builder builder() { return ImmutableIcebergInstructions.builder(); } - public interface Builder extends IcebergBaseInstructions.Builder { + public interface Builder extends IcebergBaseInstructions.Builder { IcebergInstructions build(); } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java index fda0aac88f4..7125f51e790 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java @@ -98,7 +98,7 @@ public ParquetInstructions toParquetInstructions( return builder.build(); } - public interface Builder extends IcebergBaseInstructions.Builder { + public interface Builder extends IcebergBaseInstructions.Builder { @SuppressWarnings("unused") Builder compressionCodecName(String compressionCodecName); From 244bc991f80f476cb9d585b8399fd529aa72b1a0 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Thu, 29 Aug 2024 10:47:03 -0500 Subject: [PATCH 04/28] Merged writeTable and appendTable into addPartition --- .../iceberg/util/IcebergToolsTest.java | 3 +- .../iceberg/util/IcebergCatalogAdapter.java | 181 ++++++++---------- .../util/IcebergParquetWriteInstructions.java | 24 +++ 3 files changed, 111 insertions(+), 97 deletions(-) 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 cc139c23523..0920d784896 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 @@ -133,6 +133,7 @@ void setUp() throws ExecutionException, InterruptedException { .build(); writeInstructions = IcebergParquetWriteInstructions.builder() .dataInstructions(s3Instructions) + .verifySchema(true) .build(); } @@ -930,7 +931,7 @@ void testAddSnapshotWrongSchema() throws ExecutionException, InterruptedExceptio // Drop one column and append to original table final io.deephaven.engine.table.Table updatedDhTable = dhTable.dropColumns("Region"); try { - adapter.appendTable(tableId, updatedDhTable, writeInstructions); + adapter.addPartition(tableId, updatedDhTable, writeInstructions); TestCase.fail(); } catch (final IllegalArgumentException e) { Assert.eqTrue(e.getMessage().contains("Schema mismatch"), "Exception message"); 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 76f97f3bae4..1a635966b04 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 @@ -830,154 +830,136 @@ public FileIO fileIO() { } /** - * Write a Deephaven table to a new Iceberg table in the catalog. - * - * @param dhTable The Deephaven table to write - * @param namespace The namespace for the new table, will be created if it does not exist - * @param tableName The name of the new table - */ - public void writeTable( - @NotNull final Table dhTable, - @NotNull final String namespace, - @NotNull final String tableName) { - writeTable(dhTable, namespace, tableName, IcebergParquetWriteInstructions.DEFAULT); - } - - /** - * Write a Deephaven table to a new Iceberg table in the catalog. - * - * @param dhTable The Deephaven table to write - * @param namespace The namespace for the new table, will be created if it does not exist - * @param tableName The name of the new table - * @param instructions The instructions for customizations while writing - */ - public void writeTable( - @NotNull final Table dhTable, - @NotNull final String namespace, - @NotNull final String tableName, - @NotNull final IcebergBaseInstructions instructions) { - final IcebergParquetWriteInstructions writeInstructions = verifyWriteInstructions(instructions); - final Namespace ns = createNamespaceIfDoesntExist(namespace); - final TableIdentifier tableIdentifier = TableIdentifier.of(ns, tableName); - final TableDefinition useDefinition = instructions.tableDefinition().orElse(dhTable.getDefinition()); - final SpecAndSchema specAndSchema = fromTableDefinition(useDefinition, instructions); - final org.apache.iceberg.Table icebergTable = - createNewIcebergTable(tableIdentifier, specAndSchema, writeInstructions); - try { - writeParquet(icebergTable, dhTable, writeInstructions); - } catch (final Exception writeException) { - // If we fail to write the table, we should delete the table to avoid leaving a partial table in the catalog - try { - catalog.dropTable(tableIdentifier, true); - } catch (final Exception dropException) { - writeException.addSuppressed(dropException); - } - throw writeException; - } - - } - - /** - * Append provided deephaven table as a new partition to the existing iceberg table in a single snapshot. + * Add the provided deephaven table as a new partition to the existing iceberg table in a single snapshot. * * @param icebergTableIdentifier The identifier for the iceberg table to append to * @param dhTable The deephaven table to append */ - public void appendTable( + public void addPartition( @NotNull final String icebergTableIdentifier, @NotNull final Table dhTable) { - appendTable(TableIdentifier.parse(icebergTableIdentifier), dhTable, IcebergParquetWriteInstructions.DEFAULT); + addPartition(TableIdentifier.parse(icebergTableIdentifier), dhTable, IcebergParquetWriteInstructions.DEFAULT); } /** - * Append provided deephaven table as a new partition to the existing iceberg table in a single snapshot. + * Add the provided deephaven table as a new partition to the existing iceberg table in a single snapshot. * * @param icebergTableIdentifier The identifier for the iceberg table to append to * @param dhTable The deephaven table to append * @param instructions The instructions for customizations while writing */ - public void appendTable( + public void addPartition( @NotNull final String icebergTableIdentifier, @NotNull final Table dhTable, - @Nullable final IcebergBaseInstructions instructions) { - appendTable(TableIdentifier.parse(icebergTableIdentifier), dhTable, instructions); + @NotNull final IcebergBaseInstructions instructions) { + addPartition(TableIdentifier.parse(icebergTableIdentifier), dhTable, instructions); } /** - * Append provided deephaven table as a new partition to the existing iceberg table in a single snapshot. + * Add the provided deephaven table as a new partition to the existing iceberg table in a single snapshot. * * @param icebergTableIdentifier The identifier for the iceberg table to append to * @param dhTable The deephaven table to append */ - public void appendTable( + public void addPartition( @NotNull final TableIdentifier icebergTableIdentifier, @NotNull final Table dhTable) { - appendTable(icebergTableIdentifier, dhTable, IcebergParquetWriteInstructions.DEFAULT); + addPartition(icebergTableIdentifier, dhTable, IcebergParquetWriteInstructions.DEFAULT); } /** - * Append provided deephaven table as a new partition to the existing iceberg table in a single snapshot. + * Add the provided deephaven table as a new partition to the existing iceberg table in a single snapshot. * - * @param icebergTableIdentifier The identifier for the iceberg table to append to + * @param tableIdentifier The identifier for the iceberg table to append to * @param dhTable The deephaven table to append * @param instructions The instructions for customizations while writing */ - public void appendTable( - @NotNull final TableIdentifier icebergTableIdentifier, + public void addPartition( + @NotNull final TableIdentifier tableIdentifier, @NotNull final Table dhTable, @NotNull final IcebergBaseInstructions instructions) { + final TableDefinition useDefinition = instructions.tableDefinition().orElse(dhTable.getDefinition()); + final SpecAndSchema specAndSchema = fromTableDefinition(useDefinition, instructions); final IcebergParquetWriteInstructions writeInstructions = verifyWriteInstructions(instructions); - // Load the table from the catalog. - final org.apache.iceberg.Table icebergTable = catalog.loadTable(icebergTableIdentifier); - if (icebergTable == null) { - throw new IllegalArgumentException("Table not found: " + icebergTableIdentifier); + boolean tableCreated = false; + + // Try loading the table from the catalog, or create it if it required + final org.apache.iceberg.Table icebergTable; + if (catalog.tableExists(tableIdentifier)) { + icebergTable = catalog.loadTable(tableIdentifier); + } else if (!writeInstructions.createTableIfNotExist()) { + throw new IllegalArgumentException("Table not found: " + tableIdentifier + ", update the " + + "instructions to create the table if it does not exist and try again."); + } else { + createNamespaceIfNotExists(tableIdentifier.namespace()); + icebergTable = createNewIcebergTable(tableIdentifier, specAndSchema, writeInstructions); + tableCreated = true; } - // Make sure spec and schema for the iceberg and deephaven table is identical - final TableDefinition useDefinition = instructions.tableDefinition().orElse(dhTable.getDefinition()); - final SpecAndSchema specAndSchema = fromTableDefinition(useDefinition, instructions); - if (!icebergTable.schema().sameSchema(specAndSchema.schema)) { - throw new IllegalArgumentException("Schema mismatch, iceberg table schema " - + icebergTable.schema() + ", schema derived from deephaven table: " + specAndSchema.schema); + if (writeInstructions.verifySchema()) { + // Make sure spec and schema for the iceberg and deephaven table is identical + if (!icebergTable.schema().sameSchema(specAndSchema.schema)) { + throw new IllegalArgumentException("Schema mismatch, iceberg table schema " + + icebergTable.schema() + ", schema derived from deephaven table: " + specAndSchema.schema); + } + if (!icebergTable.spec().compatibleWith(specAndSchema.partitionSpec)) { + throw new IllegalArgumentException("Partition spec mismatch, iceberg table partition spec " + + icebergTable.spec() + ", partition spec derived from deephaven table: " + + specAndSchema.partitionSpec); + } + } + + if (dhTable.isEmpty()) { + return; } - if (!icebergTable.spec().compatibleWith(specAndSchema.partitionSpec)) { - throw new IllegalArgumentException("Partition spec mismatch, iceberg table partition spec " - + icebergTable.spec() + ", partition spec derived from deephaven table: " - + specAndSchema.partitionSpec); + + try { + final List parquetFileinfo = + writeParquet(icebergTable, dhTable, writeInstructions); + createSnapshot(icebergTable, parquetFileinfo, false); + } catch (final Exception writeException) { + // If we fail to write the table, we should delete the table to avoid leaving a partial table in the catalog + if (tableCreated) { + try { + catalog.dropTable(tableIdentifier, true); + } catch (final Exception dropException) { + writeException.addSuppressed(dropException); + } + } + throw writeException; } - writeParquet(icebergTable, dhTable, writeInstructions); } - public void writePartitionedTable( + public void writePartitioned( @NotNull final Table dhTable, @NotNull final String namespace, @NotNull final String tableName, @NotNull final IcebergBaseInstructions instructions) { final IcebergParquetWriteInstructions writeInstructions = verifyWriteInstructions(instructions); - final Namespace ns = createNamespaceIfDoesntExist(namespace); + final Namespace ns = createNamespaceIfNotExists(namespace); final TableIdentifier tableIdentifier = TableIdentifier.of(ns, tableName); final TableDefinition useDefinition = instructions.tableDefinition().orElse(dhTable.getDefinition()); if (useDefinition.getPartitioningColumns().isEmpty()) { throw new IllegalArgumentException("Table must have partitioning columns to write partitioned data"); } final SpecAndSchema specAndSchema = fromTableDefinition(useDefinition, instructions); - writePartitionedTableImpl(dhTable, tableIdentifier, specAndSchema, writeInstructions); + writePartitionedImpl(dhTable, tableIdentifier, specAndSchema, writeInstructions); } - public void writePartitionedTable( + public void writePartitioned( @NotNull final PartitionedTable dhTable, @NotNull final String namespace, @NotNull final String tableName) { - writePartitionedTable(dhTable, namespace, tableName, IcebergParquetWriteInstructions.DEFAULT); + writePartitioned(dhTable, namespace, tableName, IcebergParquetWriteInstructions.DEFAULT); } - public void writePartitionedTable( + public void writePartitioned( @NotNull final PartitionedTable dhTable, @NotNull final String namespace, @NotNull final String tableName, @NotNull final IcebergBaseInstructions instructions) { final IcebergParquetWriteInstructions writeInstructions = verifyWriteInstructions(instructions); - final Namespace ns = createNamespaceIfDoesntExist(namespace); + final Namespace ns = createNamespaceIfNotExists(namespace); final TableIdentifier tableIdentifier = TableIdentifier.of(ns, tableName); final SpecAndSchema specAndSchema; if (instructions.tableDefinition().isPresent()) { @@ -985,7 +967,7 @@ public void writePartitionedTable( } else { specAndSchema = forPartitionedTable(dhTable, instructions); } - writePartitionedTableImpl(dhTable, tableIdentifier, specAndSchema, writeInstructions); + writePartitionedImpl(dhTable, tableIdentifier, specAndSchema, writeInstructions); } private static IcebergParquetWriteInstructions verifyWriteInstructions( @@ -998,15 +980,19 @@ private static IcebergParquetWriteInstructions verifyWriteInstructions( return (IcebergParquetWriteInstructions) instructions; } - private Namespace createNamespaceIfDoesntExist(@NotNull final String namespace) { - final Namespace ns = Namespace.of(namespace); + // TODO look at if required + private Namespace createNamespaceIfNotExists(@NotNull final String namespace) { + return createNamespaceIfNotExists(Namespace.of(namespace)); + } + + private Namespace createNamespaceIfNotExists(@NotNull final Namespace namespace) { if (catalog instanceof SupportsNamespaces) { final SupportsNamespaces nsCatalog = (SupportsNamespaces) catalog; - if (!nsCatalog.namespaceExists(ns)) { - nsCatalog.createNamespace(ns); + if (!nsCatalog.namespaceExists(namespace)) { + nsCatalog.createNamespace(namespace); } } - return ns; + return namespace; } private static class SpecAndSchema { @@ -1120,19 +1106,17 @@ private org.apache.iceberg.Table createNewIcebergTable( TableProperties.DEFAULT_NAME_MAPPING, columnNameMappingJson)); } - private static void writeParquet( + @NotNull + private static List writeParquet( @NotNull final org.apache.iceberg.Table icebergTable, @NotNull final Table dhTable, @NotNull final IcebergParquetWriteInstructions writeInstructions) { - if (dhTable.isEmpty()) { - return; - } // Generate a unique path for the new partition and write the data to it final String newDataLocation = icebergTable.locationProvider().newDataLocation(UUID.randomUUID() + ".parquet"); final List parquetFilesWritten = new ArrayList<>(1); final ParquetInstructions parquetInstructions = writeInstructions.toParquetInstructions(parquetFilesWritten); ParquetTools.writeTable(dhTable, newDataLocation, parquetInstructions); - createSnapshot(icebergTable, parquetFilesWritten, false); + return parquetFilesWritten; } private static void createSnapshot( @@ -1163,11 +1147,16 @@ private static void createSnapshot( append.commit(); } - private
void writePartitionedTableImpl( + private
void writePartitionedImpl( @NotNull final TABLE dhTable, @NotNull final TableIdentifier tableIdentifier, @NotNull final SpecAndSchema specAndSchema, @NotNull final IcebergParquetWriteInstructions writeInstructions) { + if (catalog.tableExists(tableIdentifier)) { + throw new IllegalArgumentException("Table already exists: " + tableIdentifier + ", we do not support " + + "adding a deephaven table with partitioning columns to an existing iceberg table."); + } + createNamespaceIfNotExists(tableIdentifier.namespace()); final org.apache.iceberg.Table icebergTable = createNewIcebergTable(tableIdentifier, specAndSchema, writeInstructions); try { diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java index 7125f51e790..374d3e034e9 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java @@ -31,6 +31,24 @@ public static Builder builder() { return ImmutableIcebergParquetWriteInstructions.builder(); } + /** + * While appending a partition to an iceberg table, whether to create the iceberg table if it does not exist; + * defaults to {@code false}. + */ + @Default + public boolean createTableIfNotExist() { + return false; + } + + /** + * While appending a partition to an iceberg table, whether to verify that the schema of the table being appended is + * consistent with the iceberg table; defaults to {@code false}. + */ + @Default + public boolean verifySchema() { + return false; + } + /** * The name of the compression codec to use when writing Parquet files; defaults to * {@value ParquetInstructions#DEFAULT_COMPRESSION_CODEC_NAME}. @@ -99,6 +117,12 @@ public ParquetInstructions toParquetInstructions( } public interface Builder extends IcebergBaseInstructions.Builder { + @SuppressWarnings("unused") + Builder createTableIfNotExist(boolean createTableIfNotExist); + + @SuppressWarnings("unused") + Builder verifySchema(boolean verifySchema); + @SuppressWarnings("unused") Builder compressionCodecName(String compressionCodecName); From 09340c2ea11090de431d9682dbfe308a30b0e726 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Fri, 30 Aug 2024 11:09:44 -0500 Subject: [PATCH 05/28] Split IcebergParquetWriteInstructions into WriteInstr and ParquetWriteInstr --- .../IcebergParquetWriteInstructionsTest.java | 128 ++++++++++++++++++ .../iceberg/util/IcebergBaseInstructions.java | 1 + .../util/IcebergParquetWriteInstructions.java | 54 ++++---- .../util/IcebergWriteInstructions.java | 39 ++++++ 4 files changed, 199 insertions(+), 23 deletions(-) create mode 100644 extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteInstructions.java diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java new file mode 100644 index 00000000000..aea8d42ec18 --- /dev/null +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java @@ -0,0 +1,128 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.util; + +import io.deephaven.parquet.table.ParquetInstructions; +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +class IcebergParquetWriteInstructionsTest { + + @Test + void defaults() { + final IcebergParquetWriteInstructions instructions = IcebergParquetWriteInstructions.builder().build(); + assertThat(instructions.tableDefinition().isEmpty()).isTrue(); + assertThat(instructions.dataInstructions().isEmpty()).isTrue(); + assertThat(instructions.columnRenames().isEmpty()).isTrue(); + assertThat(instructions.createTableIfNotExist()).isFalse(); + assertThat(instructions.verifySchema()).isFalse(); + assertThat(instructions.compressionCodecName()).isEqualTo("SNAPPY"); + assertThat(instructions.maximumDictionaryKeys()).isEqualTo(1048576); + assertThat(instructions.maximumDictionarySize()).isEqualTo(1048576); + } + + @Test + void testSetCreateTableIfNotExist() { + assertThat(IcebergParquetWriteInstructions.builder() + .createTableIfNotExist(true) + .build() + .createTableIfNotExist()) + .isTrue(); + } + + @Test + void testSetVerifySchema() { + assertThat(IcebergParquetWriteInstructions.builder() + .verifySchema(true) + .build() + .verifySchema()) + .isTrue(); + } + + @Test + void testSetCompressionCodecName() { + assertThat(IcebergParquetWriteInstructions.builder() + .compressionCodecName("GZIP") + .build() + .compressionCodecName()) + .isEqualTo("GZIP"); + } + + @Test + void testSetMaximumDictionaryKeys() { + assertThat(IcebergParquetWriteInstructions.builder() + .maximumDictionaryKeys(100) + .build() + .maximumDictionaryKeys()) + .isEqualTo(100); + } + + @Test + void testSetMaximumDictionarySize() { + assertThat(IcebergParquetWriteInstructions.builder() + .maximumDictionarySize(100) + .build() + .maximumDictionarySize()) + .isEqualTo(100); + } + + @Test + void testSetTargetPageSize() { + assertThat(IcebergParquetWriteInstructions.builder() + .targetPageSize(1024 * 1024) + .build() + .targetPageSize()) + .isEqualTo(1024 * 1024); + } + + @Test + void testMinMaximumDictionaryKeys() { + try { + IcebergParquetWriteInstructions.builder() + .maximumDictionaryKeys(-1) + .build(); + } catch (IllegalArgumentException e) { + assertThat(e).hasMessageContaining("maximumDictionaryKeys"); + } + } + + @Test + void testMinMaximumDictionarySize() { + try { + IcebergParquetWriteInstructions.builder() + .maximumDictionarySize(-1) + .build(); + } catch (IllegalArgumentException e) { + assertThat(e).hasMessageContaining("maximumDictionarySize"); + } + } + + @Test + void testMinTargetPageSize() { + try { + IcebergParquetWriteInstructions.builder() + .targetPageSize(1024) + .build(); + } catch (IllegalArgumentException e) { + assertThat(e).hasMessageContaining("targetPageSize"); + } + } + + @Test + void toParquetInstructionTest() { + final IcebergParquetWriteInstructions icebergInstructions = IcebergParquetWriteInstructions.builder() + .compressionCodecName("GZIP") + .maximumDictionaryKeys(100) + .maximumDictionarySize(200) + .targetPageSize(1024 * 1024) + .build(); + final ParquetInstructions parquetInstructions = icebergInstructions.toParquetInstructions(null); + assertThat(parquetInstructions.getCompressionCodecName()).isEqualTo("GZIP"); + assertThat(parquetInstructions.getMaximumDictionaryKeys()).isEqualTo(100); + assertThat(parquetInstructions.getMaximumDictionarySize()).isEqualTo(200); + assertThat(parquetInstructions.getTargetPageSize()).isEqualTo(1024 * 1024); + + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergBaseInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergBaseInstructions.java index 5ca4bb90320..643fe4d3bd0 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergBaseInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergBaseInstructions.java @@ -34,6 +34,7 @@ interface Builder { INSTRUCTIONS_BUILDER putColumnRenames(String key, String value); + @SuppressWarnings("unused") INSTRUCTIONS_BUILDER putAllColumnRenames(Map entries); } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java index 374d3e034e9..105b7768d19 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java @@ -7,11 +7,14 @@ import io.deephaven.parquet.table.ParquetInstructions; import org.immutables.value.Value.Default; import org.immutables.value.Value.Immutable; +import org.immutables.value.Value.Check; import org.jetbrains.annotations.NotNull; import java.util.List; import java.util.Map; +import static io.deephaven.parquet.table.ParquetInstructions.MIN_TARGET_PAGE_SIZE; + /** * This class provides instructions intended for writing Iceberg tables as Parquet data files as well as reading for * reading Iceberg catalogs and tables. The default values documented in this class may change in the future. As such, @@ -19,7 +22,7 @@ */ @Immutable @BuildableStyle -public abstract class IcebergParquetWriteInstructions implements IcebergBaseInstructions { +public abstract class IcebergParquetWriteInstructions extends IcebergWriteInstructions { /** * The default {@link IcebergParquetWriteInstructions} to use when reading/writing Iceberg tables as Parquet data * files. @@ -31,24 +34,6 @@ public static Builder builder() { return ImmutableIcebergParquetWriteInstructions.builder(); } - /** - * While appending a partition to an iceberg table, whether to create the iceberg table if it does not exist; - * defaults to {@code false}. - */ - @Default - public boolean createTableIfNotExist() { - return false; - } - - /** - * While appending a partition to an iceberg table, whether to verify that the schema of the table being appended is - * consistent with the iceberg table; defaults to {@code false}. - */ - @Default - public boolean verifySchema() { - return false; - } - /** * The name of the compression codec to use when writing Parquet files; defaults to * {@value ParquetInstructions#DEFAULT_COMPRESSION_CODEC_NAME}. @@ -80,10 +65,11 @@ public int maximumDictionarySize() { /** * The target page size for writing the parquet files; defaults to - * {@value ParquetInstructions#DEFAULT_TARGET_PAGE_SIZE}. + * {@value ParquetInstructions#DEFAULT_TARGET_PAGE_SIZE}, should be greater than or equal to + * {@value ParquetInstructions#MIN_TARGET_PAGE_SIZE}. */ @Default - public int getTargetPageSize() { + public int targetPageSize() { return ParquetInstructions.DEFAULT_TARGET_PAGE_SIZE; } @@ -92,7 +78,7 @@ public int getTargetPageSize() { * * @param completedWrites List of completed writes to be set in the {@link ParquetInstructions} */ - public ParquetInstructions toParquetInstructions( + ParquetInstructions toParquetInstructions( @NotNull final List completedWrites) { final ParquetInstructions.Builder builder = new ParquetInstructions.Builder(); @@ -110,7 +96,7 @@ public ParquetInstructions toParquetInstructions( builder.setCompressionCodecName(compressionCodecName()); builder.setMaximumDictionaryKeys(maximumDictionaryKeys()); builder.setMaximumDictionarySize(maximumDictionarySize()); - builder.setTargetPageSize(getTargetPageSize()); + builder.setTargetPageSize(targetPageSize()); builder.setCompletedWrites(completedWrites); return builder.build(); @@ -137,4 +123,26 @@ public interface Builder extends IcebergBaseInstructions.Builder { IcebergParquetWriteInstructions build(); } + + @Check + final void boundsCheckMaxDictionaryKeys() { + if (maximumDictionaryKeys() < 0) { + throw new IllegalArgumentException("maximumDictionaryKeys(=" + maximumDictionaryKeys() + ") must be >= 0"); + } + } + + @Check + final void boundsCheckMaxDictionarySize() { + if (maximumDictionarySize() < 0) { + throw new IllegalArgumentException("maximumDictionarySize(=" + maximumDictionarySize() + ") must be >= 0"); + } + } + + @Check + final void boundsCheckMinTargetPageSize() { + 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/IcebergWriteInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteInstructions.java new file mode 100644 index 00000000000..30551101c1c --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteInstructions.java @@ -0,0 +1,39 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.util; + +import org.immutables.value.Value.Default; + +/** + * This class provides instructions intended for writing Iceberg tables as well as for reading Iceberg catalogs and + * tables. The default values documented in this class may change in the future. As such, callers may wish to explicitly + * set the values. + */ +abstract class IcebergWriteInstructions implements IcebergBaseInstructions { + /** + * While appending a partition to an iceberg table, whether to create the iceberg table if it does not exist; + * defaults to {@code false}. + */ + @Default + public boolean createTableIfNotExist() { + return false; + } + + /** + * While appending a partition to an iceberg table, whether to verify that the schema of the table being appended is + * consistent with the iceberg table; defaults to {@code false}. + */ + @Default + public boolean verifySchema() { + return false; + } + + public interface Builder extends IcebergBaseInstructions.Builder { + @SuppressWarnings("unused") + INSTRUCTIONS_BUILDER createTableIfNotExist(boolean createTableIfNotExist); + + @SuppressWarnings("unused") + INSTRUCTIONS_BUILDER verifySchema(boolean verifySchema); + } +} From c70b50e6ab755fa7706879bfb4b823e603af463f Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Wed, 25 Sep 2024 13:36:27 -0500 Subject: [PATCH 06/28] Resolving more conflicts --- .../iceberg/util/IcebergToolsTest.java | 46 +++++++++---------- .../iceberg/util/IcebergCatalogAdapter.java | 8 ++-- 2 files changed, 27 insertions(+), 27 deletions(-) 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 c9733663dee..232c2ba2b80 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 @@ -20,10 +20,10 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.types.Type; -import org.junit.jupiter.api.Test; import org.junit.After; import org.junit.Before; import org.junit.Rule; +import org.junit.Test; import org.junit.experimental.categories.Category; import software.amazon.awssdk.core.async.AsyncRequestBody; import software.amazon.awssdk.services.s3.S3AsyncClient; @@ -266,7 +266,7 @@ public void testListSnapshots() { } @Test - void testOpenTableA() throws ExecutionException, InterruptedException, TimeoutException { + public void testOpenTableA() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); @@ -288,7 +288,7 @@ void testOpenTableA() throws ExecutionException, InterruptedException, TimeoutEx } @Test - void testOpenTableB() throws ExecutionException, InterruptedException, TimeoutException { + public void testOpenTableB() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesMulti(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); @@ -309,7 +309,7 @@ void testOpenTableB() throws ExecutionException, InterruptedException, TimeoutEx } @Test - void testOpenTableC() throws ExecutionException, InterruptedException, TimeoutException { + public void testOpenTableC() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesSingle(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); @@ -331,7 +331,7 @@ void testOpenTableC() throws ExecutionException, InterruptedException, TimeoutEx } @Test - void testOpenTableS3Only() throws ExecutionException, InterruptedException, TimeoutException { + public void testOpenTableS3Only() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); @@ -346,7 +346,7 @@ void testOpenTableS3Only() throws ExecutionException, InterruptedException, Time } @Test - void testOpenTableDefinition() throws ExecutionException, InterruptedException, TimeoutException { + public void testOpenTableDefinition() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() @@ -366,7 +366,7 @@ void testOpenTableDefinition() throws ExecutionException, InterruptedException, } @Test - void testOpenTablePartitionTypeException() { + public void testOpenTablePartitionTypeException() { final TableDefinition tableDef = TableDefinition.of( ColumnDefinition.ofLong("year").withPartitioning(), ColumnDefinition.ofInt("month").withPartitioning(), @@ -400,7 +400,7 @@ void testOpenTablePartitionTypeException() { } @Test - void testOpenTableDefinitionRename() throws ExecutionException, InterruptedException, TimeoutException { + public void testOpenTableDefinitionRename() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); final TableDefinition renamed = TableDefinition.of( @@ -436,7 +436,7 @@ void testOpenTableDefinitionRename() throws ExecutionException, InterruptedExcep } @Test - void testSkippedPartitioningColumn() throws ExecutionException, InterruptedException, TimeoutException { + public void testSkippedPartitioningColumn() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); final TableDefinition tableDef = TableDefinition.of( @@ -465,7 +465,7 @@ void testSkippedPartitioningColumn() throws ExecutionException, InterruptedExcep } @Test - void testReorderedPartitioningColumn() throws ExecutionException, InterruptedException, TimeoutException { + public void testReorderedPartitioningColumn() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); final TableDefinition tableDef = TableDefinition.of( @@ -494,7 +494,7 @@ void testReorderedPartitioningColumn() throws ExecutionException, InterruptedExc } @Test - void testZeroPartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { + public void testZeroPartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() @@ -514,7 +514,7 @@ void testZeroPartitioningColumns() throws ExecutionException, InterruptedExcepti } @Test - void testIncorrectPartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { + public void testIncorrectPartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { final TableDefinition tableDef = TableDefinition.of( ColumnDefinition.ofInt("month").withPartitioning(), ColumnDefinition.ofInt("year").withPartitioning(), @@ -549,7 +549,7 @@ void testIncorrectPartitioningColumns() throws ExecutionException, InterruptedEx } @Test - void testMissingPartitioningColumns() { + public void testMissingPartitioningColumns() { final TableDefinition tableDef = TableDefinition.of( ColumnDefinition.ofInt("__year").withPartitioning(), // Incorrect name ColumnDefinition.ofInt("__month").withPartitioning(), // Incorrect name @@ -583,7 +583,7 @@ void testMissingPartitioningColumns() { } @Test - void testOpenTableColumnRename() throws ExecutionException, InterruptedException, TimeoutException { + public void testOpenTableColumnRename() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() @@ -603,7 +603,7 @@ void testOpenTableColumnRename() throws ExecutionException, InterruptedException } @Test - void testOpenTableColumnLegalization() throws ExecutionException, InterruptedException, TimeoutException { + public void testOpenTableColumnLegalization() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesRenamed(); final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() @@ -622,7 +622,7 @@ void testOpenTableColumnLegalization() throws ExecutionException, InterruptedExc } @Test - void testOpenTableColumnLegalizationRename() + public void testOpenTableColumnLegalizationRename() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesRenamed(); @@ -651,7 +651,7 @@ void testOpenTableColumnLegalizationRename() } @Test - void testOpenTableColumnLegalizationPartitionException() { + public void testOpenTableColumnLegalizationPartitionException() { final TableDefinition tableDef = TableDefinition.of( ColumnDefinition.ofInt("Year").withPartitioning(), ColumnDefinition.ofInt("Month").withPartitioning()); @@ -682,7 +682,7 @@ void testOpenTableColumnLegalizationPartitionException() { } @Test - void testOpenTableColumnRenamePartitioningColumns() + public void testOpenTableColumnRenamePartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); @@ -714,7 +714,7 @@ void testOpenTableColumnRenamePartitioningColumns() } @Test - void testOpenTableSnapshot() throws ExecutionException, InterruptedException, TimeoutException { + public void testOpenTableSnapshot() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesMulti(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); @@ -746,7 +746,7 @@ void testOpenTableSnapshot() throws ExecutionException, InterruptedException, Ti } @Test - void testOpenTableSnapshotByID() throws ExecutionException, InterruptedException, TimeoutException { + public void testOpenTableSnapshotByID() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesMulti(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); @@ -793,7 +793,7 @@ void testOpenTableSnapshotByID() throws ExecutionException, InterruptedException } @Test - void testOpenAllTypesTable() throws ExecutionException, InterruptedException, TimeoutException { + public void testOpenAllTypesTable() throws ExecutionException, InterruptedException, TimeoutException { uploadAllTypes(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); @@ -907,7 +907,7 @@ public void testTableDefinitionWithInstructions() { } @Test - void testAddSnapshotWrongSchema() throws ExecutionException, InterruptedException, TimeoutException { + public void testAddSnapshotWrongSchema() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesMulti(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); @@ -928,7 +928,7 @@ void testAddSnapshotWrongSchema() throws ExecutionException, InterruptedExceptio } @Test - void testConvertToIcebergTypeAndBack() { + public 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, BigDecimal.class, byte[].class 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 dc5a003619a..ac4be79440f 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 @@ -556,7 +556,7 @@ public TableDefinition getTableDefinition( public TableDefinition getTableDefinition( @NotNull final String tableIdentifier, final long snapshotId, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergBaseInstructions instructions) { final TableIdentifier tableId = TableIdentifier.parse(tableIdentifier); // Find the snapshot with the given snapshot id @@ -581,7 +581,7 @@ public TableDefinition getTableDefinition( public TableDefinition getTableDefinition( @NotNull final TableIdentifier tableIdentifier, @Nullable final Snapshot tableSnapshot, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergBaseInstructions instructions) { // Load the table from the catalog. return getTableDefinitionInternal(tableIdentifier, tableSnapshot, instructions); } @@ -628,7 +628,7 @@ public Table getTableDefinitionTable( public Table getTableDefinitionTable( @NotNull final String tableIdentifier, final long snapshotId, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergBaseInstructions instructions) { final TableIdentifier tableId = TableIdentifier.parse(tableIdentifier); // Find the snapshot with the given snapshot id @@ -652,7 +652,7 @@ public Table getTableDefinitionTable( public Table getTableDefinitionTable( @NotNull final TableIdentifier tableIdentifier, @Nullable final Snapshot tableSnapshot, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergBaseInstructions instructions) { final TableDefinition definition = getTableDefinition(tableIdentifier, tableSnapshot, instructions); return TableTools.metaTable(definition); } From 689e8a1a290bab33ff7e383ae9f0dfa577654b13 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Mon, 30 Sep 2024 16:34:17 -0500 Subject: [PATCH 07/28] Added unit tests and moved Iceberg tests to Junit5 --- extensions/iceberg/s3/build.gradle | 21 +- .../iceberg/util/IcebergLocalStackTest.java | 6 +- .../iceberg/util/IcebergMinIOTest.java | 8 +- .../IcebergParquetWriteInstructionsTest.java | 39 +- .../iceberg/util/IcebergToolsTest.java | 102 +-- .../deephaven/iceberg/base/IcebergUtils.java | 71 ++ .../iceberg/layout/IcebergBaseLayout.java | 32 +- .../iceberg/util/IcebergCatalogAdapter.java | 644 +++++++++++------- .../util/IcebergParquetWriteInstructions.java | 13 +- .../util/IcebergWriteInstructions.java | 6 +- .../deephaven/iceberg/CatalogAdapterTest.java | 172 +++++ .../deephaven/parquet/table/MappedSchema.java | 13 +- .../parquet/table/ParquetInstructions.java | 76 ++- .../io/deephaven/parquet/table/TypeInfos.java | 8 +- 14 files changed, 826 insertions(+), 385 deletions(-) create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/base/IcebergUtils.java diff --git a/extensions/iceberg/s3/build.gradle b/extensions/iceberg/s3/build.gradle index dfb53c52388..75732f903b8 100644 --- a/extensions/iceberg/s3/build.gradle +++ b/extensions/iceberg/s3/build.gradle @@ -29,7 +29,6 @@ dependencies { compileOnly libs.autoservice annotationProcessor libs.autoservice.compiler - testImplementation libs.junit4 testImplementation project(':engine-test-utils') testImplementation libs.testcontainers @@ -45,10 +44,20 @@ dependencies { testRuntimeOnly libs.slf4j.simple } -TestTools.addEngineOutOfBandTest(project) +test { + useJUnitPlatform { + excludeTags("testcontainers") + } +} + +tasks.register('testOutOfBand', Test) { + useJUnitPlatform { + includeTags("testcontainers") + } -testOutOfBand.dependsOn Docker.registryTask(project, 'localstack') -testOutOfBand.systemProperty 'testcontainers.localstack.image', Docker.localImageName('localstack') + dependsOn Docker.registryTask(project, 'localstack') + systemProperty 'testcontainers.localstack.image', Docker.localImageName('localstack') -testOutOfBand.dependsOn Docker.registryTask(project, 'minio') -testOutOfBand.systemProperty 'testcontainers.minio.image', Docker.localImageName('minio') \ No newline at end of file + dependsOn Docker.registryTask(project, 'minio') + systemProperty 'testcontainers.minio.image', Docker.localImageName('minio') +} \ No newline at end of file diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergLocalStackTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergLocalStackTest.java index 6683bd42db1..685b89bbf1b 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergLocalStackTest.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergLocalStackTest.java @@ -5,14 +5,16 @@ import io.deephaven.extensions.s3.S3Instructions.Builder; import io.deephaven.extensions.s3.testlib.SingletonContainers; -import org.junit.BeforeClass; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; import software.amazon.awssdk.services.s3.S3AsyncClient; import java.util.Map; +@Tag("testcontainers") public class IcebergLocalStackTest extends IcebergToolsTest { - @BeforeClass + @BeforeAll public static void initContainer() { // ensure container is started so container startup time isn't associated with a specific test SingletonContainers.LocalStack.init(); diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergMinIOTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergMinIOTest.java index 946f3eca90d..d565e43d1a7 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergMinIOTest.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergMinIOTest.java @@ -7,15 +7,17 @@ import io.deephaven.extensions.s3.testlib.SingletonContainers; import io.deephaven.stats.util.OSUtil; import org.junit.jupiter.api.Assumptions; -import org.junit.BeforeClass; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; import software.amazon.awssdk.services.s3.S3AsyncClient; import java.util.Map; +@Tag("testcontainers") public class IcebergMinIOTest extends IcebergToolsTest { - @BeforeClass - public static void initContainer() { + @BeforeAll + static void initContainer() { // TODO(deephaven-core#5116): MinIO testcontainers does not work on OS X Assumptions.assumeFalse(OSUtil.runningMacOS(), "OSUtil.runningMacOS()"); // ensure container is started so container startup time isn't associated with a specific test diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java index aea8d42ec18..41fb92daa74 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java @@ -4,14 +4,19 @@ package io.deephaven.iceberg.util; import io.deephaven.parquet.table.ParquetInstructions; -import org.junit.jupiter.api.Test; +import io.deephaven.test.types.OutOfBandTest; +import org.junit.experimental.categories.Category; +import org.junit.Test; + +import java.util.Map; import static org.assertj.core.api.AssertionsForClassTypes.assertThat; -class IcebergParquetWriteInstructionsTest { +@Category(OutOfBandTest.class) +public class IcebergParquetWriteInstructionsTest { @Test - void defaults() { + public void defaults() { final IcebergParquetWriteInstructions instructions = IcebergParquetWriteInstructions.builder().build(); assertThat(instructions.tableDefinition().isEmpty()).isTrue(); assertThat(instructions.dataInstructions().isEmpty()).isTrue(); @@ -24,7 +29,7 @@ void defaults() { } @Test - void testSetCreateTableIfNotExist() { + public void testSetCreateTableIfNotExist() { assertThat(IcebergParquetWriteInstructions.builder() .createTableIfNotExist(true) .build() @@ -33,7 +38,7 @@ void testSetCreateTableIfNotExist() { } @Test - void testSetVerifySchema() { + public void testSetVerifySchema() { assertThat(IcebergParquetWriteInstructions.builder() .verifySchema(true) .build() @@ -42,7 +47,7 @@ void testSetVerifySchema() { } @Test - void testSetCompressionCodecName() { + public void testSetCompressionCodecName() { assertThat(IcebergParquetWriteInstructions.builder() .compressionCodecName("GZIP") .build() @@ -51,7 +56,7 @@ void testSetCompressionCodecName() { } @Test - void testSetMaximumDictionaryKeys() { + public void testSetMaximumDictionaryKeys() { assertThat(IcebergParquetWriteInstructions.builder() .maximumDictionaryKeys(100) .build() @@ -60,7 +65,7 @@ void testSetMaximumDictionaryKeys() { } @Test - void testSetMaximumDictionarySize() { + public void testSetMaximumDictionarySize() { assertThat(IcebergParquetWriteInstructions.builder() .maximumDictionarySize(100) .build() @@ -69,7 +74,7 @@ void testSetMaximumDictionarySize() { } @Test - void testSetTargetPageSize() { + public void testSetTargetPageSize() { assertThat(IcebergParquetWriteInstructions.builder() .targetPageSize(1024 * 1024) .build() @@ -78,7 +83,7 @@ void testSetTargetPageSize() { } @Test - void testMinMaximumDictionaryKeys() { + public void testMinMaximumDictionaryKeys() { try { IcebergParquetWriteInstructions.builder() .maximumDictionaryKeys(-1) @@ -89,7 +94,7 @@ void testMinMaximumDictionaryKeys() { } @Test - void testMinMaximumDictionarySize() { + public void testMinMaximumDictionarySize() { try { IcebergParquetWriteInstructions.builder() .maximumDictionarySize(-1) @@ -100,7 +105,7 @@ void testMinMaximumDictionarySize() { } @Test - void testMinTargetPageSize() { + public void testMinTargetPageSize() { try { IcebergParquetWriteInstructions.builder() .targetPageSize(1024) @@ -111,18 +116,22 @@ void testMinTargetPageSize() { } @Test - void toParquetInstructionTest() { + public void toParquetInstructionTest() { final IcebergParquetWriteInstructions icebergInstructions = IcebergParquetWriteInstructions.builder() .compressionCodecName("GZIP") .maximumDictionaryKeys(100) .maximumDictionarySize(200) .targetPageSize(1024 * 1024) .build(); - final ParquetInstructions parquetInstructions = icebergInstructions.toParquetInstructions(null); + final Map fieldIdToName = Map.of(2, "field2", 3, "field3"); + final ParquetInstructions parquetInstructions = icebergInstructions.toParquetInstructions( + null, fieldIdToName); assertThat(parquetInstructions.getCompressionCodecName()).isEqualTo("GZIP"); assertThat(parquetInstructions.getMaximumDictionaryKeys()).isEqualTo(100); assertThat(parquetInstructions.getMaximumDictionarySize()).isEqualTo(200); assertThat(parquetInstructions.getTargetPageSize()).isEqualTo(1024 * 1024); - + assertThat(parquetInstructions.getFieldId("field1")).isEmpty(); + assertThat(parquetInstructions.getFieldId("field2")).hasValue(2); + assertThat(parquetInstructions.getFieldId("field3")).hasValue(3); } } 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 232c2ba2b80..6b8e846e635 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 @@ -13,18 +13,14 @@ import io.deephaven.extensions.s3.S3Instructions; import io.deephaven.iceberg.TestCatalog.IcebergTestCatalog; import junit.framework.TestCase; -import io.deephaven.test.types.OutOfBandTest; import org.apache.iceberg.Snapshot; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.io.FileIO; import org.apache.iceberg.types.Type; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import software.amazon.awssdk.core.async.AsyncRequestBody; import software.amazon.awssdk.services.s3.S3AsyncClient; import software.amazon.awssdk.services.s3.model.CreateBucketRequest; @@ -50,7 +46,6 @@ import static io.deephaven.iceberg.util.IcebergCatalogAdapter.SNAPSHOT_DEFINITION; import static io.deephaven.iceberg.util.IcebergCatalogAdapter.TABLES_DEFINITION; -@Category(OutOfBandTest.class) public abstract class IcebergToolsTest { private static final TableDefinition SALES_SINGLE_DEFINITION = TableDefinition.of( @@ -100,7 +95,6 @@ public abstract class IcebergToolsTest { ColumnDefinition.ofBoolean("IsPartitioning")); private IcebergInstructions instructions; - private IcebergParquetWriteInstructions writeInstructions; public abstract S3AsyncClient s3AsyncClient(); @@ -115,13 +109,12 @@ public abstract class IcebergToolsTest { private String warehousePath; private Catalog resourceCatalog; - private FileIO resourceFileIO; - @Rule - public final EngineCleanup framework = new EngineCleanup(); + private final EngineCleanup engineCleanup = new EngineCleanup(); - @Before - public void setUp() throws ExecutionException, InterruptedException { + @BeforeEach + void setUp() throws Exception { + engineCleanup.setUp(); bucket = "warehouse"; asyncClient = s3AsyncClient(); asyncClient.createBucket(CreateBucketRequest.builder().bucket(bucket).build()).get(); @@ -136,13 +129,9 @@ public void setUp() throws ExecutionException, InterruptedException { instructions = IcebergInstructions.builder() .dataInstructions(s3Instructions) .build(); - writeInstructions = IcebergParquetWriteInstructions.builder() - .dataInstructions(s3Instructions) - .verifySchema(true) - .build(); } - @After + @AfterEach public void tearDown() throws ExecutionException, InterruptedException { for (String key : keys) { asyncClient.deleteObject(DeleteObjectRequest.builder().bucket(bucket).key(key).build()).get(); @@ -199,7 +188,7 @@ private void uploadSalesRenamed() throws ExecutionException, InterruptedExceptio } @Test - public void testListNamespaces() { + void testListNamespaces() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); final Collection namespaces = adapter.listNamespaces(); @@ -216,7 +205,7 @@ public void testListNamespaces() { } @Test - public void testListTables() { + void testListTables() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); @@ -240,7 +229,7 @@ public void testListTables() { } @Test - public void testListSnapshots() { + void testListSnapshots() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); final TLongArrayList snapshotIds = new TLongArrayList(); @@ -266,7 +255,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); @@ -309,7 +298,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); @@ -331,7 +320,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); @@ -346,7 +335,7 @@ public void testOpenTableS3Only() throws ExecutionException, InterruptedExceptio } @Test - public void testOpenTableDefinition() throws ExecutionException, InterruptedException, TimeoutException { + void testOpenTableDefinition() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() @@ -366,7 +355,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(), @@ -400,7 +389,7 @@ public void testOpenTablePartitionTypeException() { } @Test - public void testOpenTableDefinitionRename() throws ExecutionException, InterruptedException, TimeoutException { + void testOpenTableDefinitionRename() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); final TableDefinition renamed = TableDefinition.of( @@ -436,7 +425,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( @@ -465,7 +454,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( @@ -494,7 +483,7 @@ public void testReorderedPartitioningColumn() throws ExecutionException, Interru } @Test - public void testZeroPartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { + void testZeroPartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() @@ -514,7 +503,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(), @@ -549,7 +538,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 @@ -583,7 +572,7 @@ public void testMissingPartitioningColumns() { } @Test - public void testOpenTableColumnRename() throws ExecutionException, InterruptedException, TimeoutException { + void testOpenTableColumnRename() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() @@ -603,7 +592,7 @@ public void testOpenTableColumnRename() throws ExecutionException, InterruptedEx } @Test - public void testOpenTableColumnLegalization() throws ExecutionException, InterruptedException, TimeoutException { + void testOpenTableColumnLegalization() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesRenamed(); final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() @@ -622,7 +611,7 @@ public void testOpenTableColumnLegalization() throws ExecutionException, Interru } @Test - public void testOpenTableColumnLegalizationRename() + void testOpenTableColumnLegalizationRename() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesRenamed(); @@ -651,7 +640,7 @@ public void testOpenTableColumnLegalizationRename() } @Test - public void testOpenTableColumnLegalizationPartitionException() { + void testOpenTableColumnLegalizationPartitionException() { final TableDefinition tableDef = TableDefinition.of( ColumnDefinition.ofInt("Year").withPartitioning(), ColumnDefinition.ofInt("Month").withPartitioning()); @@ -682,7 +671,7 @@ public void testOpenTableColumnLegalizationPartitionException() { } @Test - public void testOpenTableColumnRenamePartitioningColumns() + void testOpenTableColumnRenamePartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); @@ -714,7 +703,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); @@ -746,7 +735,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); @@ -793,7 +782,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); @@ -808,7 +797,7 @@ public void testOpenAllTypesTable() throws ExecutionException, InterruptedExcept } @Test - public void testTableDefinition() { + void testTableDefinition() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); @@ -833,7 +822,7 @@ public void testTableDefinition() { } @Test - public void testTableDefinitionTable() { + void testTableDefinitionTable() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); @@ -866,7 +855,7 @@ public void testTableDefinitionTable() { } @Test - public void testTableDefinitionWithInstructions() { + void testTableDefinitionWithInstructions() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); IcebergBaseInstructions localInstructions = IcebergInstructions.builder() @@ -907,31 +896,10 @@ public void testTableDefinitionWithInstructions() { } @Test - public void testAddSnapshotWrongSchema() throws ExecutionException, InterruptedException, TimeoutException { - uploadSalesMulti(); - - final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); - - final Namespace ns = Namespace.of("sales"); - final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); - final io.deephaven.engine.table.Table dhTable = adapter.readTable(tableId, writeInstructions); - Assert.eq(dhTable.size(), "dhTable.size()", 100_000, "100_000 rows in the table"); - - // Drop one column and append to original table - final io.deephaven.engine.table.Table updatedDhTable = dhTable.dropColumns("Region"); - try { - adapter.addPartition(tableId, updatedDhTable, writeInstructions); - TestCase.fail(); - } catch (final IllegalArgumentException e) { - Assert.eqTrue(e.getMessage().contains("Schema mismatch"), "Exception message"); - } - } - - @Test - public void testConvertToIcebergTypeAndBack() { + 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, BigDecimal.class, byte[].class + LocalDateTime.class, LocalDate.class, LocalTime.class, byte[].class }; for (final Class javaType : javaTypes) { 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..f8acee66ffc --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/base/IcebergUtils.java @@ -0,0 +1,71 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.base; + +import io.deephaven.engine.table.impl.locations.TableDataException; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.ManifestContent; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.ManifestReader; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.FileIO; +import org.jetbrains.annotations.NotNull; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.List; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; + +public final class IcebergUtils { + + /** + * Get a stream of all {@link DataFile} objects from the given {@link Snapshot} and {@link Table}. + * + * @param table The {@link Table} to retrieve data files for. + * @param snapshot The {@link Snapshot} to retrieve data files from. + * @param fileIO The {@link FileIO} to use for reading manifest data files. + * @return A stream of {@link DataFile} objects. + */ + public static Stream getAllDataFiles( + @NotNull final Table table, + @NotNull final Snapshot snapshot, + @NotNull final FileIO fileIO) { + try { + // Retrieve the manifest files from the snapshot + final List manifestFiles = snapshot.allManifests(fileIO); + return manifestFiles.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())); + } + }) + .flatMap(manifestFile -> { + try { + final ManifestReader reader = ManifestFiles.read(manifestFile, fileIO); + return StreamSupport.stream(reader.spliterator(), false).onClose(() -> { + try { + reader.close(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + }); + } catch (final Exception e) { + throw new TableDataException( + String.format("%s:%d:%s - error reading manifest file", table, + snapshot.snapshotId(), manifestFile), + e); + } + }); + } catch (final Exception e) { + throw new TableDataException( + String.format("%s:%d - error retrieving manifest files", table, snapshot.snapshotId()), e); + } + } +} 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 70a142e5daa..d171bde0e49 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 @@ -20,9 +20,11 @@ import java.net.URI; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.function.Consumer; +import java.util.stream.Stream; + +import static io.deephaven.iceberg.base.IcebergUtils.getAllDataFiles; public abstract class IcebergBaseLayout implements TableLocationKeyFinder { /** @@ -142,27 +144,15 @@ private URI dataFileUri(@NotNull DataFile df) { @Override public synchronized void findKeys(@NotNull final Consumer locationKeyObserver) { - try { - // Retrieve the manifest files from the snapshot - final List manifestFiles = snapshot.allManifests(fileIO); - for (final ManifestFile manifestFile : manifestFiles) { - // Currently only can process manifest files with DATA content type. - if (manifestFile.content() != ManifestContent.DATA) { - throw new TableDataException( - String.format("%s:%d - only DATA manifest files are currently supported, encountered %s", - table, snapshot.snapshotId(), manifestFile.content())); - } - try (final ManifestReader reader = ManifestFiles.read(manifestFile, fileIO)) { - for (DataFile df : reader) { - final URI fileUri = dataFileUri(df); - final IcebergTableLocationKey locationKey = - cache.computeIfAbsent(fileUri, uri -> keyFromDataFile(df, fileUri)); - if (locationKey != null) { - locationKeyObserver.accept(locationKey); - } - } + try (final Stream dataFiles = getAllDataFiles(table, snapshot, fileIO)) { + dataFiles.forEach(df -> { + final URI fileUri = dataFileUri(df); + final IcebergTableLocationKey locationKey = + cache.computeIfAbsent(fileUri, uri -> keyFromDataFile(df, fileUri)); + if (locationKey != null) { + locationKeyObserver.accept(locationKey); } - } + }); } catch (final Exception e) { throw new TableDataException( String.format("%s:%d - error finding Iceberg locations", table, snapshot.snapshotId()), e); diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java index 5e83928fb64..8b9c1e102b5 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 @@ -3,10 +3,6 @@ // package io.deephaven.iceberg.util; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.node.ArrayNode; -import com.fasterxml.jackson.databind.node.ObjectNode; import io.deephaven.UncheckedDeephavenException; import io.deephaven.api.util.NameValidator; import io.deephaven.engine.rowset.RowSetFactory; @@ -34,13 +30,18 @@ import io.deephaven.util.annotations.VisibleForTesting; import org.apache.hadoop.util.StringUtils; import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFiles; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.Transaction; +import org.apache.iceberg.UpdatePartitionSpec; +import org.apache.iceberg.UpdateSchema; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.SupportsNamespaces; @@ -57,6 +58,9 @@ import java.time.LocalTime; import java.util.*; import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static io.deephaven.iceberg.base.IcebergUtils.getAllDataFiles; public class IcebergCatalogAdapter { @@ -206,14 +210,13 @@ static io.deephaven.qst.type.Type convertToDHType(@NotNull final Type iceberg case TIME: return io.deephaven.qst.type.Type.find(LocalTime.class); case DECIMAL: - // TODO We know the precision and scale of the decimal, but we're not using it here. 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 // TODO Add support for lists + case LIST: // Fall through case MAP: // Fall through default: throw new TableDataException("Unsupported iceberg column type " + typeId.name()); @@ -242,15 +245,13 @@ static Type convertToIcebergType(final Class columnType) { return Types.DateType.get(); } else if (columnType == LocalTime.class) { return Types.TimeType.get(); - } else if (columnType == BigDecimal.class) { - // TODO Compute precision and scale from the table and use that for parquet writing and appending - return Types.DecimalType.of(38, 18); } else if (columnType == byte[].class) { return Types.BinaryType.get(); } else { throw new TableDataException("Unsupported deephaven column type " + columnType.getName()); } - // TODO Add support for writing lists, for reading too + // TODO Add support for writing and appending big decimals + // TODO Add support for reading and writing lists } /** @@ -880,99 +881,272 @@ public Catalog catalog() { } /** - * Add the provided deephaven table as a new partition to the existing iceberg table in a single snapshot. + * Append the provided deephaven table as a new partition to the existing iceberg table in a single snapshot. * - * @param icebergTableIdentifier The identifier for the iceberg table to append to + * @param tableIdentifier The identifier string for the iceberg table to append to * @param dhTable The deephaven table to append + * @param instructions The instructions for customizations while writing, or null to use default instructions */ - public void addPartition( - @NotNull final String icebergTableIdentifier, - @NotNull final Table dhTable) { - addPartition(TableIdentifier.parse(icebergTableIdentifier), dhTable, IcebergParquetWriteInstructions.DEFAULT); + public void append( + @NotNull final String tableIdentifier, + @NotNull final Table dhTable, + @Nullable final IcebergWriteInstructions instructions) { + append(tableIdentifier, new Table[] {dhTable}, instructions); } /** - * Add the provided deephaven table as a new partition to the existing iceberg table in a single snapshot. + * Append the provided deephaven table as a new partition to the existing iceberg table in a single snapshot. * - * @param icebergTableIdentifier The identifier for the iceberg table to append to + * @param tableIdentifier The identifier for the iceberg table to append to * @param dhTable The deephaven table to append - * @param instructions The instructions for customizations while writing + * @param instructions The instructions for customizations while writing, or null to use default instructions */ - public void addPartition( - @NotNull final String icebergTableIdentifier, + public void append( + @NotNull final TableIdentifier tableIdentifier, @NotNull final Table dhTable, - @NotNull final IcebergBaseInstructions instructions) { - addPartition(TableIdentifier.parse(icebergTableIdentifier), dhTable, instructions); + @Nullable final IcebergWriteInstructions instructions) { + append(tableIdentifier, new Table[] {dhTable}, instructions); } /** - * Add the provided deephaven table as a new partition to the existing iceberg table in a single snapshot. + * Append the provided deephaven tables as new partitions to the existing iceberg table in a single snapshot. All + * tables should have the same definition, else a table definition should be provided in the instructions. * - * @param icebergTableIdentifier The identifier for the iceberg table to append to - * @param dhTable The deephaven table to append + * @param tableIdentifier The identifier string for the iceberg table to append to + * @param dhTables The deephaven tables to append + * @param instructions The instructions for customizations while writing, or null to use default instructions */ - public void addPartition( - @NotNull final TableIdentifier icebergTableIdentifier, - @NotNull final Table dhTable) { - addPartition(icebergTableIdentifier, dhTable, IcebergParquetWriteInstructions.DEFAULT); + public void append( + @NotNull final String tableIdentifier, + @NotNull final Table[] dhTables, + @Nullable final IcebergWriteInstructions instructions) { + append(TableIdentifier.parse(tableIdentifier), dhTables, instructions); } /** - * Add the provided deephaven table as a new partition to the existing iceberg table in a single snapshot. + * Append the provided deephaven tables as new partitions to the existing iceberg table in a single snapshot. All + * tables should have the same definition, else a table definition should be provided in the instructions. * * @param tableIdentifier The identifier for the iceberg table to append to - * @param dhTable The deephaven table to append - * @param instructions The instructions for customizations while writing + * @param dhTables The deephaven tables to append + * @param instructions The instructions for customizations while writing, or null to use default instructions */ - public void addPartition( + public void append( @NotNull final TableIdentifier tableIdentifier, + @NotNull final Table[] dhTables, + @Nullable final IcebergWriteInstructions instructions) { + writeImpl(tableIdentifier, dhTables, instructions, false, true); + } + + /** + * Overwrite the existing iceberg table with the provided deephaven table in a single snapshot. + * + * @param tableIdentifier The identifier string for the iceberg table to overwrite + * @param dhTable The deephaven table to overwrite with + * @param instructions The instructions for customizations while writing, or null to use default instructions + */ + public void overwrite( + @NotNull final String tableIdentifier, @NotNull final Table dhTable, - @NotNull final IcebergBaseInstructions instructions) { - final TableDefinition useDefinition = instructions.tableDefinition().orElse(dhTable.getDefinition()); - final SpecAndSchema specAndSchema = fromTableDefinition(useDefinition, instructions); - final IcebergParquetWriteInstructions writeInstructions = verifyWriteInstructions(instructions); - boolean tableCreated = false; + @Nullable final IcebergWriteInstructions instructions) { + overwrite(tableIdentifier, new Table[] {dhTable}, instructions); + } - // Try loading the table from the catalog, or create it if it required - final org.apache.iceberg.Table icebergTable; - if (catalog.tableExists(tableIdentifier)) { - icebergTable = catalog.loadTable(tableIdentifier); - } else if (!writeInstructions.createTableIfNotExist()) { - throw new IllegalArgumentException("Table not found: " + tableIdentifier + ", update the " + - "instructions to create the table if it does not exist and try again."); - } else { - createNamespaceIfNotExists(tableIdentifier.namespace()); - icebergTable = createNewIcebergTable(tableIdentifier, specAndSchema, writeInstructions); - tableCreated = true; - } + /** + * Overwrite the existing iceberg table with the provided deephaven table in a single snapshot. + * + * @param tableIdentifier The identifier for the iceberg table to overwrite + * @param dhTable The deephaven table to overwrite with + * @param instructions The instructions for customizations while writing, or null to use default instructions + */ + public void overwrite( + @NotNull final TableIdentifier tableIdentifier, + @NotNull final Table dhTable, + @Nullable final IcebergWriteInstructions instructions) { + overwrite(tableIdentifier, new Table[] {dhTable}, instructions); + } - if (writeInstructions.verifySchema()) { - // Make sure spec and schema for the iceberg and deephaven table is identical - if (!icebergTable.schema().sameSchema(specAndSchema.schema)) { - throw new IllegalArgumentException("Schema mismatch, iceberg table schema " - + icebergTable.schema() + ", schema derived from deephaven table: " + specAndSchema.schema); - } - if (!icebergTable.spec().compatibleWith(specAndSchema.partitionSpec)) { - throw new IllegalArgumentException("Partition spec mismatch, iceberg table partition spec " - + icebergTable.spec() + ", partition spec derived from deephaven table: " - + specAndSchema.partitionSpec); + /** + * Overwrite the existing iceberg table with the provided deephaven tables appended together in a single snapshot. + * All tables should have the same definition, else a table definition should be provided in the instructions. + * + * @param tableIdentifier The identifier string for the iceberg table to overwrite + * @param dhTables The deephaven tables to overwrite with + * @param instructions The instructions for customizations while writing, or null to use default instructions + */ + public void overwrite( + @NotNull final String tableIdentifier, + @NotNull final Table[] dhTables, + @Nullable final IcebergWriteInstructions instructions) { + overwrite(TableIdentifier.parse(tableIdentifier), dhTables, instructions); + } + + /** + * Overwrite the existing iceberg table with the provided deephaven tables appended together in a single snapshot. + * All tables should have the same definition, else a table definition should be provided in the instructions. + * + * @param tableIdentifier The identifier for the iceberg table to overwrite + * @param dhTables The deephaven tables to overwrite with + * @param instructions The instructions for customizations while writing, or null to use default instructions + */ + public void overwrite( + @NotNull final TableIdentifier tableIdentifier, + @NotNull final Table[] dhTables, + @Nullable final IcebergWriteInstructions instructions) { + writeImpl(tableIdentifier, dhTables, instructions, true, true); + } + + /** + * Writes data from a Deephaven table 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. + * + * @param tableIdentifier The identifier string for the Iceberg table to write to. + * @param dhTable The Deephaven table containing the data to be written. + * @param instructions The instructions for customizations while writing, or null to use default instructions. + * + * @return A list of {@link DataFile} objects representing the written data files. + */ + public List write( + @NotNull final String tableIdentifier, + @NotNull final Table dhTable, + @Nullable final IcebergWriteInstructions instructions) { + return write(tableIdentifier, new Table[] {dhTable}, instructions); + } + + /** + * Writes data from a Deephaven table 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. + * + * @param tableIdentifier The identifier for the Iceberg table to write to. + * @param dhTable The Deephaven table containing the data to be written. + * @param instructions The instructions for customizations while writing, or null to use default instructions. + * + * @return A list of {@link DataFile} objects representing the written data files. + */ + public List write( + @NotNull final TableIdentifier tableIdentifier, + @NotNull final Table dhTable, + @Nullable final IcebergWriteInstructions instructions) { + return write(tableIdentifier, new Table[] {dhTable}, instructions); + } + + /** + * Writes data from Deephaven 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. All tables + * should have the same definition, else a table definition should be provided in the instructions. + * + * @param tableIdentifier The identifier string for the Iceberg table to write to. + * @param dhTables The Deephaven tables containing the data to be written. + * @param instructions The instructions for customizations while writing, or null to use default instructions. + * + * @return A list of {@link DataFile} objects representing the written data files. + */ + public List write( + @NotNull final String tableIdentifier, + @NotNull final Table[] dhTables, + @Nullable final IcebergWriteInstructions instructions) { + return write(TableIdentifier.parse(tableIdentifier), dhTables, instructions); + } + + /** + * Writes data from Deephaven 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. All tables + * should have the same definition, else a table definition should be provided in the instructions. + * + * @param tableIdentifier The identifier for the Iceberg table to write to. + * @param dhTables The Deephaven tables containing the data to be written. + * @param instructions The instructions for customizations while writing, or null to use default instructions. + * + * @return A list of {@link DataFile} objects representing the written data files. + */ + public List write( + @NotNull final TableIdentifier tableIdentifier, + @NotNull final Table[] dhTables, + @Nullable final IcebergWriteInstructions instructions) { + return writeImpl(tableIdentifier, dhTables, instructions, false, false); + } + + /** + * Appends or overwrites data in an Iceberg table with the provided Deephaven table. + * + * @param tableIdentifier The identifier for the Iceberg table to append to or overwrite + * @param dhTables The Deephaven tables to write + * @param instructions The instructions for customizations while writing, or null to use default instructions + * @param overwrite If true, the existing data in the Iceberg table will be overwritten; if false, the data will be + * appended + * @param addSnapshot If true, a new snapshot will be created in the Iceberg table with the written data + * + * @return A list of DataFile objects representing the written data files. + */ + private List writeImpl( + @NotNull final TableIdentifier tableIdentifier, + @NotNull final Table[] dhTables, + @Nullable final IcebergWriteInstructions instructions, + final boolean overwrite, + final boolean addSnapshot) { + final IcebergWriteInstructions userInstructions = + instructions == null ? IcebergParquetWriteInstructions.DEFAULT : instructions; + + // Verify that all tables have the same definition + final TableDefinition useDefinition; + if (userInstructions.tableDefinition().isPresent()) { + useDefinition = userInstructions.tableDefinition().get(); + } else { + final TableDefinition firstDefinition = dhTables[0].getDefinition(); + for (int idx = 1; idx < dhTables.length; idx++) { + if (!firstDefinition.equals(dhTables[idx].getDefinition())) { + throw new IllegalArgumentException( + "All Deephacen tables must have the same definition, else table definition shuold be " + + "provided when writing multiple tables with different definitions"); + } } + useDefinition = firstDefinition; } - if (dhTable.isEmpty()) { - return; + final IcebergParquetWriteInstructions writeInstructions = verifyWriteInstructions(userInstructions); + + // Try loading the table from the catalog, or create if required + final org.apache.iceberg.Table icebergTable; + final SpecAndSchema newSpecAndSchema; + final boolean newTableCreated; + if (catalog.tableExists(tableIdentifier)) { + icebergTable = catalog.loadTable(tableIdentifier); + newSpecAndSchema = fromTableDefinition(useDefinition, userInstructions); + newTableCreated = false; + if (writeInstructions.verifySchema()) { + if (overwrite) { + verifyOverwriteCompatibility(icebergTable.schema(), newSpecAndSchema.schema); + verifyOverwriteCompatibility(icebergTable.spec(), newSpecAndSchema.partitionSpec); + } else { + verifyAppendCompatibility(icebergTable.schema(), useDefinition); + verifyAppendCompatibility(icebergTable.spec(), useDefinition); + } + } + } else if (writeInstructions.createTableIfNotExist()) { + createNamespaceIfNotExists(tableIdentifier.namespace()); + newSpecAndSchema = fromTableDefinition(useDefinition, userInstructions); + icebergTable = createNewIcebergTable(tableIdentifier, newSpecAndSchema, writeInstructions); + newTableCreated = true; + } else { + throw new IllegalArgumentException( + "Table does not exist: " + tableIdentifier + ", update the instructions " + + "to create the table if it does not exist and try again."); } try { final List parquetFileinfo = - writeParquet(icebergTable, dhTable, writeInstructions); - createSnapshot(icebergTable, parquetFileinfo, false); - } catch (final Exception writeException) { - // If we fail to write the table, we should delete the table to avoid leaving a partial table in the catalog - if (tableCreated) { + writeParquet(icebergTable, dhTables, writeInstructions); + final List appendFiles = dataFilesFromParquet(parquetFileinfo); + if (addSnapshot) { + commit(icebergTable, newSpecAndSchema, appendFiles, writeInstructions, overwrite && !newTableCreated); + } + return appendFiles; + } catch (final RuntimeException writeException) { + if (newTableCreated) { + // Delete it to avoid leaving a partial table in the catalog try { catalog.dropTable(tableIdentifier, true); - } catch (final Exception dropException) { + } catch (final RuntimeException dropException) { writeException.addSuppressed(dropException); } } @@ -980,48 +1154,83 @@ public void addPartition( } } - public void writePartitioned( - @NotNull final Table dhTable, - @NotNull final String namespace, - @NotNull final String tableName, - @NotNull final IcebergBaseInstructions instructions) { - final IcebergParquetWriteInstructions writeInstructions = verifyWriteInstructions(instructions); - final Namespace ns = createNamespaceIfNotExists(namespace); - final TableIdentifier tableIdentifier = TableIdentifier.of(ns, tableName); - final TableDefinition useDefinition = instructions.tableDefinition().orElse(dhTable.getDefinition()); - if (useDefinition.getPartitioningColumns().isEmpty()) { - throw new IllegalArgumentException("Table must have partitioning columns to write partitioned data"); + /** + * Check if the schema for the iceberg table is compatible for overwriting with a deephaven table with provided + * definition. + */ + private static void verifyOverwriteCompatibility( + final Schema icebergSchema, + final Schema newSchema) { + if (!icebergSchema.sameSchema(newSchema)) { + throw new IllegalArgumentException("Schema mismatch, iceberg table schema: " + icebergSchema + + ", schema derived from the table definition: " + newSchema); } - final SpecAndSchema specAndSchema = fromTableDefinition(useDefinition, instructions); - writePartitionedImpl(dhTable, tableIdentifier, specAndSchema, writeInstructions); } - public void writePartitioned( - @NotNull final PartitionedTable dhTable, - @NotNull final String namespace, - @NotNull final String tableName) { - writePartitioned(dhTable, namespace, tableName, IcebergParquetWriteInstructions.DEFAULT); + /** + * Check if the partition spec for the iceberg table is compatible for overwriting with a deephaven table with + * provided definition. + */ + private static void verifyOverwriteCompatibility( + final PartitionSpec icebergPartitionSpec, + final PartitionSpec newPartitionSpec) { + if (!icebergPartitionSpec.compatibleWith(newPartitionSpec)) { + throw new IllegalArgumentException("Partition spec mismatch, iceberg table partition spec: " + + icebergPartitionSpec + ", partition spec derived from table definition: " + newPartitionSpec); + } } - public void writePartitioned( - @NotNull final PartitionedTable dhTable, - @NotNull final String namespace, - @NotNull final String tableName, - @NotNull final IcebergBaseInstructions instructions) { - final IcebergParquetWriteInstructions writeInstructions = verifyWriteInstructions(instructions); - final Namespace ns = createNamespaceIfNotExists(namespace); - final TableIdentifier tableIdentifier = TableIdentifier.of(ns, tableName); - final SpecAndSchema specAndSchema; - if (instructions.tableDefinition().isPresent()) { - specAndSchema = fromTableDefinition(instructions.tableDefinition().get(), instructions); - } else { - specAndSchema = forPartitionedTable(dhTable, instructions); + /** + * Check if the schema for the iceberg table is compatible for appending a deephaven table with provided definition. + */ + private static void verifyAppendCompatibility( + final Schema icebergSchema, + final TableDefinition tableDefinition) { + // Check that all columns in the table definition are part of the Iceberg schema + for (final ColumnDefinition dhColumn : tableDefinition.getColumns()) { + final Types.NestedField icebergColumn = icebergSchema.findField(dhColumn.getName()); + if (icebergColumn == null || !icebergColumn.type().equals(convertToIcebergType(dhColumn.getDataType()))) { + throw new IllegalArgumentException("Schema mismatch, column " + dhColumn.getName() + " from Deephaven " + + "table definition: " + tableDefinition + " is not found or has a different type in Iceberg " + + "table schema: " + icebergSchema); + } + } + + // Check that all required columns in the Iceberg schema are part of the table definition + for (final Types.NestedField icebergColumn : icebergSchema.columns()) { + if (icebergColumn.isOptional()) { + continue; + } + if (tableDefinition.getColumn(icebergColumn.name()) == null) { + throw new IllegalArgumentException("Partition spec mismatch, required column " + icebergColumn.name() + + " from Iceberg table schema: " + icebergSchema + " not found in Deephaven table definition: " + + tableDefinition); + } + } + } + + /** + * Check if the partition spec for the Iceberg table is compatible for appending deephaven table with provided + * definition. + */ + private static void verifyAppendCompatibility( + final PartitionSpec partitionSpec, + final TableDefinition tableDefinition) { + final Set icebergPartitionColumns = partitionSpec.fields().stream() + .map(PartitionField::name) + .collect(Collectors.toSet()); + final Set dhPartitioningColumns = tableDefinition.getColumns().stream() + .filter(ColumnDefinition::isPartitioning) + .map(ColumnDefinition::getName) + .collect(Collectors.toSet()); + if (!icebergPartitionColumns.equals(dhPartitioningColumns)) { + throw new IllegalArgumentException("Partitioning column mismatch, iceberg table partition spec: " + + partitionSpec + ", deephaven table definition: " + tableDefinition); } - writePartitionedImpl(dhTable, tableIdentifier, specAndSchema, writeInstructions); } private static IcebergParquetWriteInstructions verifyWriteInstructions( - @NotNull final IcebergBaseInstructions instructions) { + @NotNull final IcebergWriteInstructions instructions) { // We ony support writing to Parquet files if (!(instructions instanceof IcebergParquetWriteInstructions)) { throw new IllegalArgumentException("Unsupported instructions of class " + instructions.getClass() + " for" + @@ -1030,11 +1239,6 @@ private static IcebergParquetWriteInstructions verifyWriteInstructions( return (IcebergParquetWriteInstructions) instructions; } - // TODO look at if required - private Namespace createNamespaceIfNotExists(@NotNull final String namespace) { - return createNamespaceIfNotExists(Namespace.of(namespace)); - } - private Namespace createNamespaceIfNotExists(@NotNull final Namespace namespace) { if (catalog instanceof SupportsNamespaces) { final SupportsNamespaces nsCatalog = (SupportsNamespaces) catalog; @@ -1047,7 +1251,7 @@ private Namespace createNamespaceIfNotExists(@NotNull final Namespace namespace) private static class SpecAndSchema { private final PartitionSpec partitionSpec; - private final Schema schema; + private Schema schema; private SpecAndSchema(final PartitionSpec partitionSpec, final Schema schema) { this.partitionSpec = partitionSpec; @@ -1064,10 +1268,10 @@ private static SpecAndSchema fromTableDefinition( final Collection partitioningColumnNames = new ArrayList<>(); final List fields = new ArrayList<>(); int fieldID = 1; // Iceberg field IDs start from 1 - // Create the schema + + // Create the schema first and use it to build the partition spec for (final ColumnDefinition columnDefinition : tableDefinition.getColumns()) { final String dhColumnName = columnDefinition.getName(); - // TODO Check with others how column name renames should work for writing final String icebergColName = instructions.columnRenames().getOrDefault(dhColumnName, dhColumnName); final Type icebergType = convertToIcebergType(columnDefinition.getDataType()); fields.add(Types.NestedField.optional(fieldID, icebergColName, icebergType)); @@ -1077,178 +1281,124 @@ private static SpecAndSchema fromTableDefinition( fieldID++; } final Schema schema = new Schema(fields); - final PartitionSpec partitionSpec = createPartitionSpec(schema, partitioningColumnNames); - return new SpecAndSchema(partitionSpec, schema); - } - private static SpecAndSchema forPartitionedTable( - @NotNull final PartitionedTable partitionedTable, - @NotNull final IcebergBaseInstructions instructions) { - // TODO Look at the duplication - final List fields = new ArrayList<>(); - int fieldID = 1; // Iceberg field IDs start from 1 - // Create the schema - final TableDefinition partitionedTableDefinition = partitionedTable.table().getDefinition(); - final Set keyColumnNames = partitionedTable.keyColumnNames(); - for (final String keyColumnName : keyColumnNames) { - final ColumnDefinition keyColumnDefinition = partitionedTableDefinition.getColumn(keyColumnName); - final String icebergColName = instructions.columnRenames().getOrDefault(keyColumnName, keyColumnName); - final Type icebergType = convertToIcebergType(keyColumnDefinition.getDataType()); - fields.add(Types.NestedField.optional(fieldID, icebergColName, icebergType)); - fieldID++; - } - final TableDefinition constituentDefinition = partitionedTable.constituentDefinition(); - for (final ColumnDefinition leafColumnDefinition : constituentDefinition.getColumns()) { - final String dhColumnName = leafColumnDefinition.getName(); - if (keyColumnNames.contains(dhColumnName)) { - continue; - } - final String icebergColName = instructions.columnRenames().getOrDefault(dhColumnName, dhColumnName); - final Type icebergType = convertToIcebergType(leafColumnDefinition.getDataType()); - fields.add(Types.NestedField.optional(fieldID, icebergColName, icebergType)); - fieldID++; - } - final Schema schema = new Schema(fields); - final PartitionSpec partitionSpec = createPartitionSpec(schema, keyColumnNames); + final PartitionSpec partitionSpec = createPartitionSpec(schema, partitioningColumnNames); return new SpecAndSchema(partitionSpec, schema); } - /** - * Convert a map of column IDs to names to a JSON string. For example, the map {1 -> "A", 2 -> "B"} would be - * converted to: [{"field-id":1,"names":["A"]},{"field-id":2,"names":["B"]}] - */ - // TODO Check with others if there is a better way to do this - private static String convertIdToNameMapToJson(final Map idToNameMap) { - final ObjectMapper mapper = new ObjectMapper(); - final ArrayNode parentNode = mapper.createArrayNode(); - for (final Map.Entry columnIndo : idToNameMap.entrySet()) { - final ObjectNode columnNode = mapper.createObjectNode(); - columnNode.put("field-id", columnIndo.getKey()); - final ArrayNode namesArray = mapper.createArrayNode(); - namesArray.add(columnIndo.getValue()); - columnNode.set("names", namesArray); - parentNode.add(columnNode); - } - try { - return mapper.writeValueAsString(parentNode); - } catch (final JsonProcessingException e) { - throw new UncheckedDeephavenException("Failed to convert id to name map to JSON, map=" + idToNameMap, e); + private 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(); } private org.apache.iceberg.Table createNewIcebergTable( @NotNull final TableIdentifier tableIdentifier, @NotNull final SpecAndSchema specAndSchema, @NotNull final IcebergParquetWriteInstructions writeInstructions) { - // Check if a table with the same name already exists if (catalog.tableExists(tableIdentifier)) { throw new IllegalArgumentException("Table already exists: " + tableIdentifier); } - // It is required to either set the column name mapping in the table properties or to provide the field IDs in - // the parquet file, to map the column names from the parquet file to the iceberg table schema. We are using the - // former approach here. - // TODO Check with larry if we are looking at this correctly while reading - final String columnNameMappingJson = convertIdToNameMapToJson(specAndSchema.schema.idToName()); return catalog.createTable(tableIdentifier, specAndSchema.schema, specAndSchema.partitionSpec, null, Map.of(TableProperties.DEFAULT_FILE_FORMAT, DEFAULT_GENERATED_FILE_FORMAT, TableProperties.PARQUET_COMPRESSION, - StringUtils.toLowerCase(writeInstructions.compressionCodecName()), - TableProperties.DEFAULT_NAME_MAPPING, columnNameMappingJson)); + StringUtils.toLowerCase(writeInstructions.compressionCodecName()))); } @NotNull private static List writeParquet( @NotNull final org.apache.iceberg.Table icebergTable, - @NotNull final Table dhTable, + @NotNull final Table[] dhTables, @NotNull final IcebergParquetWriteInstructions writeInstructions) { - // Generate a unique path for the new partition and write the data to it - final String newDataLocation = icebergTable.locationProvider().newDataLocation(UUID.randomUUID() + ".parquet"); - final List parquetFilesWritten = new ArrayList<>(1); - final ParquetInstructions parquetInstructions = writeInstructions.toParquetInstructions(parquetFilesWritten); - ParquetTools.writeTable(dhTable, newDataLocation, parquetInstructions); + final List parquetFilesWritten = new ArrayList<>(dhTables.length); + final ParquetInstructions parquetInstructions = writeInstructions.toParquetInstructions( + parquetFilesWritten, icebergTable.schema().idToName()); + for (final Table dhTable : dhTables) { + final long epochMicrosNow = Instant.now().getEpochSecond() * 1_000_000 + Instant.now().getNano() / 1_000; + final String filename = new StringBuilder() + .append(epochMicrosNow) // To keep the data ordered by time + .append("-") + .append(UUID.randomUUID()) + .append(".parquet") + .toString(); + final String newDataLocation = + icebergTable.locationProvider().newDataLocation(filename); + ParquetTools.writeTable(dhTable, newDataLocation, parquetInstructions); + } return parquetFilesWritten; } - private static void createSnapshot( + /** + * Commit the changes to the Iceberg table by creating a single snapshot. + */ + private static void commit( @NotNull final org.apache.iceberg.Table icebergTable, - @NotNull final Collection parquetFilesWritten, - final boolean isPartitioned) { - if (parquetFilesWritten.isEmpty()) { - throw new UncheckedDeephavenException("Failed to create a snapshot because no parquet files were written"); - } + @NotNull final SpecAndSchema newSpecAndSchema, + @NotNull final Iterable appendFiles, + @NotNull final IcebergWriteInstructions writeInstructions, + final boolean overwrite) { // Append new data files to the table - final AppendFiles append = icebergTable.newAppend(); - for (final ParquetInstructions.CompletedWrite parquetFileWritten : parquetFilesWritten) { - final String filePath = parquetFileWritten.destination().toString(); - final DataFiles.Builder dfBuilder = DataFiles.builder(icebergTable.spec()) - .withPath(filePath) - .withFormat(FileFormat.PARQUET) - .withRecordCount(parquetFileWritten.numRows()) - .withFileSizeInBytes(parquetFileWritten.numBytes()); - if (isPartitioned) { - // TODO Find the partition path properly - final String tableDataLocation = icebergTable.location() + "/data/"; - final String partitionPath = filePath.substring(tableDataLocation.length(), filePath.lastIndexOf('/')); - dfBuilder.withPartitionPath(partitionPath); + final Transaction icebergTransaction = icebergTable.newTransaction(); + + if (overwrite) { + // Delete all the existing data files in the table + final DeleteFiles deletes = icebergTransaction.newDelete(); + try (final Stream dataFiles = + getAllDataFiles(icebergTable, icebergTable.currentSnapshot(), icebergTable.io())) { + dataFiles.forEach(deletes::deleteFile); } - append.appendFile(dfBuilder.build()); - } - // Commit the changes to create a new snapshot - append.commit(); - } - - private
void writePartitionedImpl( - @NotNull final TABLE dhTable, - @NotNull final TableIdentifier tableIdentifier, - @NotNull final SpecAndSchema specAndSchema, - @NotNull final IcebergParquetWriteInstructions writeInstructions) { - if (catalog.tableExists(tableIdentifier)) { - throw new IllegalArgumentException("Table already exists: " + tableIdentifier + ", we do not support " + - "adding a deephaven table with partitioning columns to an existing iceberg table."); - } - createNamespaceIfNotExists(tableIdentifier.namespace()); - final org.apache.iceberg.Table icebergTable = - createNewIcebergTable(tableIdentifier, specAndSchema, writeInstructions); - try { - final List parquetFilesWritten = new ArrayList<>(); - final ParquetInstructions parquetInstructions = - writeInstructions.toParquetInstructions(parquetFilesWritten); - // TODO Find data location properly - final String destinationDir = icebergTable.location() + "/data"; - if (dhTable instanceof PartitionedTable) { - // TODO This duplicated code doesn't look good, do something about it - final PartitionedTable partitionedTable = (PartitionedTable) dhTable; - if (partitionedTable.table().isEmpty()) { - return; + deletes.commit(); + + // Update the spec and schema of the existing table. + // If we have already verified the schema, we don't need to update it again. + if (!writeInstructions.verifySchema()) { + if (!icebergTable.schema().sameSchema(newSpecAndSchema.schema)) { + final UpdateSchema updateSchema = icebergTransaction.updateSchema().allowIncompatibleChanges(); + icebergTable.schema().columns().stream() + .map(Types.NestedField::name) + .forEach(updateSchema::deleteColumn); + newSpecAndSchema.schema.columns() + .forEach(column -> updateSchema.addColumn(column.name(), column.type())); + updateSchema.commit(); } - ParquetTools.writeKeyValuePartitionedTable(partitionedTable, destinationDir, parquetInstructions); - } else { - final Table table = (Table) dhTable; - if (table.isEmpty()) { - return; + if (!icebergTable.spec().compatibleWith(newSpecAndSchema.partitionSpec)) { + final UpdatePartitionSpec updateSpec = icebergTransaction.updateSpec(); + icebergTable.spec().fields().forEach(field -> updateSpec.removeField(field.name())); + newSpecAndSchema.partitionSpec.fields().forEach(field -> updateSpec.addField(field.name())); + updateSpec.commit(); } - ParquetTools.writeKeyValuePartitionedTable(table, destinationDir, parquetInstructions); - } - createSnapshot(icebergTable, parquetFilesWritten, true); - } catch (final Exception writeException) { - // If we fail to write the table, we should delete the table to avoid leaving a partial table in the catalog - try { - catalog.dropTable(tableIdentifier, true); - } catch (final Exception dropException) { - writeException.addSuppressed(dropException); } - throw writeException; } + + // Append the new data files to the table + final AppendFiles append = icebergTransaction.newAppend(); + appendFiles.forEach(append::appendFile); + append.commit(); + + // Commit the transaction to create a new snapshot + icebergTransaction.commitTransaction(); } - private static PartitionSpec createPartitionSpec( - @NotNull final Schema schema, - @NotNull final Iterable partitionColumnNames) { - // Create the partition spec - final PartitionSpec.Builder partitionSpecBuilder = PartitionSpec.builderFor(schema); - for (final String partitioningColumnName : partitionColumnNames) { - partitionSpecBuilder.identity(partitioningColumnName); + /** + * Generate a list of {@link DataFile} objects from a list of parquet files written. + */ + private static List dataFilesFromParquet( + @NotNull final Collection parquetFilesWritten) { + if (parquetFilesWritten.isEmpty()) { + throw new UncheckedDeephavenException("Failed to generate data files because no parquet files written"); } - return partitionSpecBuilder.build(); + // TODO This assumes no partition data is written + return parquetFilesWritten.stream() + .map(parquetFileWritten -> DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(parquetFileWritten.destination().toString()) + .withFormat(FileFormat.PARQUET) + .withRecordCount(parquetFileWritten.numRows()) + .withFileSizeInBytes(parquetFileWritten.numBytes()) + .build()) + .collect(Collectors.toList()); } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java index 105b7768d19..31cf7eb2f60 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java @@ -77,9 +77,11 @@ public int targetPageSize() { * Convert this {@link IcebergParquetWriteInstructions} to a {@link ParquetInstructions}. * * @param completedWrites List of completed writes to be set in the {@link ParquetInstructions} + * @param fieldIdToName Mapping of field id to field name, to be populated inside the parquet file's schema */ ParquetInstructions toParquetInstructions( - @NotNull final List completedWrites) { + @NotNull final List completedWrites, + @NotNull final Map fieldIdToName) { final ParquetInstructions.Builder builder = new ParquetInstructions.Builder(); tableDefinition().ifPresent(builder::setTableDefinition); @@ -93,6 +95,7 @@ ParquetInstructions toParquetInstructions( } // Add parquet writing specific instructions. + builder.addFieldIdMapping(fieldIdToName); builder.setCompressionCodecName(compressionCodecName()); builder.setMaximumDictionaryKeys(maximumDictionaryKeys()); builder.setMaximumDictionarySize(maximumDictionarySize()); @@ -102,13 +105,7 @@ ParquetInstructions toParquetInstructions( return builder.build(); } - public interface Builder extends IcebergBaseInstructions.Builder { - @SuppressWarnings("unused") - Builder createTableIfNotExist(boolean createTableIfNotExist); - - @SuppressWarnings("unused") - Builder verifySchema(boolean verifySchema); - + public interface Builder extends IcebergWriteInstructions.Builder { @SuppressWarnings("unused") Builder compressionCodecName(String compressionCodecName); 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 index 30551101c1c..8f256bdd1a6 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteInstructions.java @@ -10,7 +10,7 @@ * tables. The default values documented in this class may change in the future. As such, callers may wish to explicitly * set the values. */ -abstract class IcebergWriteInstructions implements IcebergBaseInstructions { +public abstract class IcebergWriteInstructions implements IcebergBaseInstructions { /** * While appending a partition to an iceberg table, whether to create the iceberg table if it does not exist; * defaults to {@code false}. @@ -21,8 +21,8 @@ public boolean createTableIfNotExist() { } /** - * While appending a partition to an iceberg table, whether to verify that the schema of the table being appended is - * consistent with the iceberg table; defaults to {@code false}. + * While appending a partition to an iceberg table, whether to verify that the partition spec and schema of the + * table being appended is consistent with the iceberg table; defaults to {@code false}. */ @Default public boolean verifySchema() { diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java index d08e1735db0..2f0372f3f76 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java @@ -6,13 +6,17 @@ import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.util.TableTools; import io.deephaven.iceberg.junit5.CatalogAdapterBase; +import io.deephaven.iceberg.util.IcebergParquetWriteInstructions; +import io.deephaven.iceberg.util.IcebergWriteInstructions; import org.apache.iceberg.Schema; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.Test; +import static io.deephaven.engine.testutil.TstUtils.assertTableEquals; import static org.assertj.core.api.Assertions.assertThat; public class CatalogAdapterTest extends CatalogAdapterBase { @@ -46,4 +50,172 @@ void createEmptyTable() { // Note: this is failing w/ NPE, assumes that Snapshot is non-null. // 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"); + try { + catalogAdapter.append("MyNamespace.MyTable", source, null); + } catch (RuntimeException e) { + assertThat(e.getMessage()).contains("Table does not exist"); + } + final IcebergWriteInstructions writeInstructions = IcebergParquetWriteInstructions.builder() + .createTableIfNotExist(true) + .build(); + catalogAdapter.append("MyNamespace.MyTable", source, writeInstructions); + Table fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + assertTableEquals(source, fromIceberg); + + // Append more data + final Table moreData = TableTools.emptyTable(5) + .update("intCol = (int) 3 * i + 20", + "doubleCol = (double) 3.5 * i + 20"); + catalogAdapter.append("MyNamespace.MyTable", moreData, writeInstructions); + fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + final Table expected = TableTools.merge(source, moreData); + assertTableEquals(expected, fromIceberg); + + // Append an empty table + final Table emptyTable = TableTools.emptyTable(0) + .update("intCol = (int) 4 * i + 30", + "doubleCol = (double) 4.5 * i + 30"); + catalogAdapter.append("MyNamespace.MyTable", emptyTable, writeInstructions); + fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + assertTableEquals(expected, fromIceberg); + + // Append multiple tables in a single call + final Table someMoreData = TableTools.emptyTable(3) + .update("intCol = (int) 5 * i + 40", + "doubleCol = (double) 5.5 * i + 40"); + catalogAdapter.append("MyNamespace.MyTable", new Table[] {someMoreData, moreData, emptyTable}, + writeInstructions); + fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + final Table expected2 = TableTools.merge(expected, someMoreData, moreData); + assertTableEquals(expected2, fromIceberg); + } + + @Test + void overwriteTablesBasicTest() { + final Table source = TableTools.emptyTable(10) + .update("intCol = (int) 2 * i + 10", + "doubleCol = (double) 2.5 * i + 10"); + try { + catalogAdapter.overwrite("MyNamespace.MyTable", source, null); + } catch (RuntimeException e) { + assertThat(e.getMessage()).contains("Table does not exist"); + } + final IcebergWriteInstructions writeInstructions = IcebergParquetWriteInstructions.builder() + .createTableIfNotExist(true) + .build(); + catalogAdapter.overwrite("MyNamespace.MyTable", source, writeInstructions); + Table fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + assertTableEquals(source, fromIceberg); + + // Overwrite with more data + final Table moreData = TableTools.emptyTable(5) + .update("intCol = (int) 3 * i + 20", + "doubleCol = (double) 3.5 * i + 20"); + catalogAdapter.overwrite("MyNamespace.MyTable", moreData, writeInstructions); + fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + assertTableEquals(moreData, fromIceberg); + + // Overwrite with an empty table + final Table emptyTable = TableTools.emptyTable(0) + .update("intCol = (int) 4 * i + 30", + "doubleCol = (double) 4.5 * i + 30"); + catalogAdapter.overwrite("MyNamespace.MyTable", emptyTable, writeInstructions); + fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + assertTableEquals(emptyTable, fromIceberg); + + // Overwrite with multiple tables in a single call + final Table someMoreData = TableTools.emptyTable(3) + .update("intCol = (int) 5 * i + 40", + "doubleCol = (double) 5.5 * i + 40"); + catalogAdapter.overwrite("MyNamespace.MyTable", new Table[] {someMoreData, moreData, emptyTable}, + writeInstructions); + fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + final Table expected2 = TableTools.merge(someMoreData, moreData); + assertTableEquals(expected2, fromIceberg); + } + + @Test + void overwriteWithDifferentDefinition() { + final Table source = TableTools.emptyTable(10) + .update("intCol = (int) 2 * i + 10", + "doubleCol = (double) 2.5 * i + 10"); + final IcebergWriteInstructions writeInstructionsWithSchemaMatching = IcebergParquetWriteInstructions.builder() + .createTableIfNotExist(true) + .verifySchema(true) + .build(); + catalogAdapter.append("MyNamespace.MyTable", source, writeInstructionsWithSchemaMatching); + Table fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + assertTableEquals(source, fromIceberg); + + final Table differentSource = TableTools.emptyTable(10) + .update("intCol = (int) 2 * i + 10"); + try { + catalogAdapter.overwrite("MyNamespace.MyTable", differentSource, writeInstructionsWithSchemaMatching); + } catch (RuntimeException e) { + assertThat(e.getMessage()).contains("Schema mismatch"); + } + + final IcebergWriteInstructions writeInstructionsWithoutSchemaMatching = + IcebergParquetWriteInstructions.builder() + .verifySchema(false) + .build(); + catalogAdapter.overwrite("MyNamespace.MyTable", differentSource, writeInstructionsWithoutSchemaMatching); + fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + assertTableEquals(differentSource, fromIceberg); + + // Append more data to this table with the updated schema + final Table moreData = TableTools.emptyTable(5) + .update("intCol = (int) 3 * i + 20"); + catalogAdapter.append("MyNamespace.MyTable", moreData, writeInstructionsWithoutSchemaMatching); + fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + final Table expected = TableTools.merge(differentSource, moreData); + assertTableEquals(expected, fromIceberg); + } + + @Test + void appendWithDifferentDefinition() { + final Table source = TableTools.emptyTable(10) + .update("intCol = (int) 2 * i + 10", + "doubleCol = (double) 2.5 * i + 10"); + final IcebergWriteInstructions writeInstructions = IcebergParquetWriteInstructions.builder() + .createTableIfNotExist(true) + .verifySchema(true) + .build(); + catalogAdapter.append("MyNamespace.MyTable", source, writeInstructions); + Table fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + assertTableEquals(source, fromIceberg); + + final Table differentSource = TableTools.emptyTable(10) + .update("shortCol = (short) 2 * i + 10"); + try { + catalogAdapter.append("MyNamespace.MyTable", differentSource, writeInstructions); + } catch (RuntimeException e) { + assertThat(e.getMessage()).contains("Schema mismatch"); + } + + // Append a table with just the int column, should be compatible with the existing schema + final Table compatibleSource = TableTools.emptyTable(10) + .update("intCol = (int) 5 * i + 10"); + catalogAdapter.append("MyNamespace.MyTable", compatibleSource, writeInstructions); + fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + final Table expected = TableTools.merge(source, compatibleSource.update("doubleCol = NULL_DOUBLE")); + assertTableEquals(expected, fromIceberg); + + // Append more data + final Table moreData = TableTools.emptyTable(5) + .update("intCol = (int) 3 * i + 20", + "doubleCol = (double) 3.5 * i + 20"); + catalogAdapter.append("MyNamespace.MyTable", moreData, writeInstructions); + fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + final Table expected2 = TableTools.merge(expected, moreData); + assertTableEquals(expected2, fromIceberg); + } + + // TODO Add more unit tests to get all data types and full coverage } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/MappedSchema.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/MappedSchema.java index d818443bb58..3eedfa05f15 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/MappedSchema.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/MappedSchema.java @@ -26,21 +26,14 @@ static MappedSchema create( final TableDefinition definition, final RowSet rowSet, final Map> columnSourceMap, - final ParquetInstructions instructions, - final ColumnDefinition... extraColumns) { + final ParquetInstructions instructions) { final MessageTypeBuilder builder = Types.buildMessage(); for (final ColumnDefinition columnDefinition : definition.getColumns()) { - TypeInfos.TypeInfo typeInfo = + final TypeInfos.TypeInfo typeInfo = getTypeInfo(computedCache, columnDefinition, rowSet, columnSourceMap, instructions); - Type schemaType = typeInfo.createSchemaType(columnDefinition, instructions); + final Type schemaType = typeInfo.createSchemaType(columnDefinition, instructions); builder.addField(schemaType); } - - for (final ColumnDefinition extraColumn : extraColumns) { - builder.addField(getTypeInfo(computedCache, extraColumn, rowSet, columnSourceMap, instructions) - .createSchemaType(extraColumn, instructions)); - } - final MessageType schema = builder.named("root"); return new MappedSchema(definition, schema); } 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 e992c875f39..a6843919838 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 @@ -3,6 +3,7 @@ // package io.deephaven.parquet.table; +import gnu.trove.map.hash.TIntObjectHashMap; import io.deephaven.base.verify.Require; import io.deephaven.configuration.Configuration; import io.deephaven.engine.table.TableDefinition; @@ -19,8 +20,10 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.OptionalInt; import java.util.Set; import java.util.function.Function; import java.util.function.Predicate; @@ -103,6 +106,8 @@ public final String getColumnNameFromParquetColumnNameOrDefault(final String par */ public abstract boolean useDictionary(String columnName); + public abstract OptionalInt getFieldId(final String columnName); + public abstract Object getSpecialInstructions(); public abstract String getCompressionCodecName(); @@ -247,6 +252,11 @@ public boolean useDictionary(final String columnName) { return false; } + @Override + public OptionalInt getFieldId(final String columnName) { + return OptionalInt.empty(); + } + @Override @Nullable public Object getSpecialInstructions() { @@ -348,6 +358,7 @@ private static class ColumnInstructions { private String codecName; private String codecArgs; private boolean useDictionary; + private int fieldId = Integer.MIN_VALUE; public ColumnInstructions(final String columnName) { this.columnName = columnName; @@ -391,6 +402,14 @@ public boolean useDictionary() { public void useDictionary(final boolean useDictionary) { this.useDictionary = useDictionary; } + + public OptionalInt getFieldId() { + return fieldId == Integer.MIN_VALUE ? OptionalInt.empty() : OptionalInt.of(fieldId); + } + + public void setFieldId(final int fieldId) { + this.fieldId = fieldId; + } } private static final class ReadOnly extends ParquetInstructions { @@ -451,8 +470,8 @@ private ReadOnly( this.completedWrites = completedWrites; } - private String getOrDefault(final String columnName, final String defaultValue, - final Function fun) { + private VALUE_TYPE getOrDefault(final String columnName, final VALUE_TYPE defaultValue, + final Function fun) { if (columnNameToInstructions == null) { return defaultValue; } @@ -507,6 +526,11 @@ public boolean useDictionary(final String columnName) { return getOrDefault(columnName, false, ColumnInstructions::useDictionary); } + @Override + public OptionalInt getFieldId(final String columnName) { + return getOrDefault(columnName, OptionalInt.empty(), ColumnInstructions::getFieldId); + } + @Override public String getCompressionCodecName() { return compressionCodecName; @@ -660,6 +684,7 @@ public static class Builder { private TableDefinition tableDefinition; private Collection> indexColumns; private List completedWrites; + private TIntObjectHashMap usedFieldIdToColumn; /** * For each additional field added, make sure to update the copy constructor builder @@ -790,6 +815,53 @@ public Builder useDictionary(final String columnName, final boolean useDictionar return this; } + /** + * Add a mapping from field ID to column name. This field ID will be populated inside the parquet schema when + * writing the parquet file. + */ + public Builder addFieldId(final String columnName, final int fieldId) { + if (usedFieldIdToColumn == null) { + usedFieldIdToColumn = new TIntObjectHashMap<>(); + usedFieldIdToColumn.put(fieldId, columnName); + getColumnInstructions(columnName).setFieldId(fieldId); + } else { + addFieldIdHelper(fieldId, columnName); + } + return this; + } + + /** + * Populate mapping from field ID to column name using the provided map. These field IDs will be populated + * inside the parquet schema when writing the parquet file. + * + * @param fieldIdToColumn A map from field ID to column name + */ + public Builder addFieldIdMapping(final Map fieldIdToColumn) { + if (usedFieldIdToColumn == null) { + usedFieldIdToColumn = new TIntObjectHashMap<>(fieldIdToColumn.size()); + for (final Map.Entry entry : fieldIdToColumn.entrySet()) { + final int fieldId = entry.getKey(); + final String column = entry.getValue(); + usedFieldIdToColumn.put(fieldId, column); + getColumnInstructions(column).setFieldId(fieldId); + } + } else { + for (final Map.Entry entry : fieldIdToColumn.entrySet()) { + addFieldIdHelper(entry.getKey(), entry.getValue()); + } + } + return this; + } + + private void addFieldIdHelper(final int fieldId, final String column) { + if (usedFieldIdToColumn.containsKey(fieldId)) { + throw new IllegalArgumentException("Field ID " + fieldId + " is already assigned to column " + + usedFieldIdToColumn.get(fieldId) + " and cannot be assigned to column " + column); + } + usedFieldIdToColumn.put(fieldId, column); + getColumnInstructions(column).setFieldId(fieldId); + } + private ColumnInstructions getColumnInstructions(final String columnName) { final ColumnInstructions ci; if (columnNameToInstructions == null) { diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/TypeInfos.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/TypeInfos.java index b95dfe98412..536e22e1786 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/TypeInfos.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/TypeInfos.java @@ -13,6 +13,7 @@ import io.deephaven.util.codec.SerializableCodec; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; +import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.LogicalTypeAnnotation; import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; @@ -474,10 +475,15 @@ default Type createSchemaType( builder = getBuilder(isRequired(columnDefinition), false, dataType); isRepeating = false; } + if (!isRepeating) { + instructions.getFieldId(columnDefinition.getName()).ifPresent(builder::id); return builder.named(parquetColumnName); } - return Types.buildGroup(Type.Repetition.OPTIONAL).addField( + // For repeated fields (like lists), we need to wrap the field in a group + final Types.GroupBuilder groupBuilder = Types.buildGroup(Type.Repetition.OPTIONAL); + instructions.getFieldId(columnDefinition.getName()).ifPresent(groupBuilder::id); + return groupBuilder.addField( Types.buildGroup(Type.Repetition.REPEATED).addField( builder.named("item")).named(parquetColumnName)) .as(LogicalTypeAnnotation.listType()).named(parquetColumnName); From d7f2c814a7d7395474ab2fbf338efa05e919333a Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Tue, 1 Oct 2024 11:45:00 -0500 Subject: [PATCH 08/28] Preparing change for code review Part 1 --- extensions/iceberg/s3/build.gradle | 2 +- .../iceberg/util/IcebergToolsS3.java | 12 ++--- .../iceberg/util/IcebergToolsTest.java | 29 +++++----- .../deephaven/iceberg/base/IcebergUtils.java | 6 +-- .../iceberg/util/IcebergCatalogAdapter.java | 53 ++++++++++--------- .../iceberg/util/IcebergInstructions.java | 1 - .../util/IcebergParquetWriteInstructions.java | 7 ++- .../util/IcebergWriteInstructions.java | 13 +++-- .../s3/S3CompletableOutputStream.java | 2 +- 9 files changed, 62 insertions(+), 63 deletions(-) diff --git a/extensions/iceberg/s3/build.gradle b/extensions/iceberg/s3/build.gradle index 75732f903b8..9923ea10fad 100644 --- a/extensions/iceberg/s3/build.gradle +++ b/extensions/iceberg/s3/build.gradle @@ -60,4 +60,4 @@ tasks.register('testOutOfBand', Test) { dependsOn Docker.registryTask(project, 'minio') systemProperty 'testcontainers.minio.image', Docker.localImageName('minio') -} \ No newline at end of file +} 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 72391187987..47a602ee773 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 @@ -5,12 +5,11 @@ import com.google.common.base.Strings; import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.aws.AwsClientProperties; import org.apache.iceberg.aws.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.io.FileIO; import org.apache.iceberg.rest.RESTCatalog; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -19,12 +18,10 @@ 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 class IcebergToolsS3 extends IcebergTools { - private static final String S3_FILE_IO_CLASS = "org.apache.iceberg.aws.s3.S3FileIO"; - /** * Create an Iceberg catalog adapter for a REST catalog backed by S3 storage. If {@code null} is provided for a * value, the system defaults will be used. @@ -100,8 +97,9 @@ private static IcebergCatalogAdapter createAdapterCommon( properties.put(CatalogProperties.URI, catalogURI); properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation); - final FileIO fileIO = CatalogUtil.loadFileIO(S3_FILE_IO_CLASS, properties, null); - properties.put(CatalogProperties.FILE_IO_IMPL, fileIO.getClass().getName()); + // 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.initialize(catalogName, 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 6b8e846e635..f72acd72e4b 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 @@ -12,7 +12,6 @@ import io.deephaven.engine.testutil.junit4.EngineCleanup; import io.deephaven.extensions.s3.S3Instructions; import io.deephaven.iceberg.TestCatalog.IcebergTestCatalog; -import junit.framework.TestCase; import org.apache.iceberg.Snapshot; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; @@ -338,7 +337,7 @@ void testOpenTableS3Only() throws ExecutionException, InterruptedException, Time void testOpenTableDefinition() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); - final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() + final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(SALES_PARTITIONED_DEFINITION) .dataInstructions(instructions.dataInstructions().get()) .build(); @@ -365,7 +364,7 @@ void testOpenTablePartitionTypeException() { ColumnDefinition.ofLong("Unit_Price"), ColumnDefinition.ofTime("Order_Date")); - final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() + final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) .dataInstructions(instructions.dataInstructions().get()) .build(); @@ -401,7 +400,7 @@ void testOpenTableDefinitionRename() throws ExecutionException, InterruptedExcep ColumnDefinition.ofDouble("UnitPrice"), ColumnDefinition.ofTime("OrderDate")); - final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() + final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(renamed) .dataInstructions(instructions.dataInstructions().get()) .putColumnRenames("Region", "RegionName") @@ -437,7 +436,7 @@ void testSkippedPartitioningColumn() throws ExecutionException, InterruptedExcep ColumnDefinition.ofDouble("Unit_Price"), ColumnDefinition.ofTime("Order_Date")); - final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() + final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) .dataInstructions(instructions.dataInstructions().get()) .build(); @@ -466,7 +465,7 @@ void testReorderedPartitioningColumn() throws ExecutionException, InterruptedExc ColumnDefinition.ofDouble("Unit_Price"), ColumnDefinition.ofTime("Order_Date")); - final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() + final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) .dataInstructions(instructions.dataInstructions().get()) .build(); @@ -486,7 +485,7 @@ void testReorderedPartitioningColumn() throws ExecutionException, InterruptedExc void testZeroPartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); - final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() + final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(SALES_MULTI_DEFINITION) .dataInstructions(instructions.dataInstructions().get()) .build(); @@ -513,7 +512,7 @@ void testIncorrectPartitioningColumns() throws ExecutionException, InterruptedEx ColumnDefinition.ofDouble("Unit_Price"), ColumnDefinition.ofTime("Order_Date")); - final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() + final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) .dataInstructions(instructions.dataInstructions().get()) .build(); @@ -548,7 +547,7 @@ void testMissingPartitioningColumns() { ColumnDefinition.ofLong("Unit_Price"), ColumnDefinition.ofTime("Order_Date")); - final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() + final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) .dataInstructions(instructions.dataInstructions().get()) .build(); @@ -575,7 +574,7 @@ void testMissingPartitioningColumns() { void testOpenTableColumnRename() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); - final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() + final IcebergInstructions localInstructions = IcebergInstructions.builder() .dataInstructions(instructions.dataInstructions().get()) .putColumnRenames("Region", "RegionName") .putColumnRenames("Item_Type", "ItemType") @@ -595,7 +594,7 @@ void testOpenTableColumnRename() throws ExecutionException, InterruptedException void testOpenTableColumnLegalization() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesRenamed(); - final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() + final IcebergInstructions localInstructions = IcebergInstructions.builder() .dataInstructions(instructions.dataInstructions().get()) .build(); @@ -615,7 +614,7 @@ void testOpenTableColumnLegalizationRename() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesRenamed(); - final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() + final IcebergInstructions localInstructions = IcebergInstructions.builder() .dataInstructions(instructions.dataInstructions().get()) .putColumnRenames("Item&Type", "Item_Type") .putColumnRenames("Units/Sold", "Units_Sold") @@ -645,7 +644,7 @@ void testOpenTableColumnLegalizationPartitionException() { ColumnDefinition.ofInt("Year").withPartitioning(), ColumnDefinition.ofInt("Month").withPartitioning()); - final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() + final IcebergInstructions localInstructions = IcebergInstructions.builder() .tableDefinition(tableDef) .putColumnRenames("Year", "Current Year") .putColumnRenames("Month", "Current Month") @@ -675,7 +674,7 @@ void testOpenTableColumnRenamePartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); - final IcebergBaseInstructions localInstructions = IcebergInstructions.builder() + final IcebergInstructions localInstructions = IcebergInstructions.builder() .dataInstructions(instructions.dataInstructions().get()) .putColumnRenames("VendorID", "vendor_id") .putColumnRenames("month", "__month") @@ -858,7 +857,7 @@ void testTableDefinitionTable() { void testTableDefinitionWithInstructions() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); - IcebergBaseInstructions localInstructions = IcebergInstructions.builder() + IcebergInstructions localInstructions = IcebergInstructions.builder() .dataInstructions(instructions.dataInstructions().get()) .putColumnRenames("Region", "Area") .putColumnRenames("Item_Type", "ItemType") 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 index f8acee66ffc..ddb9fc26409 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/base/IcebergUtils.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/base/IcebergUtils.java @@ -23,7 +23,7 @@ public final class IcebergUtils { /** - * Get a stream of all {@link DataFile} objects from the given {@link Snapshot} and {@link Table}. + * 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. @@ -56,14 +56,14 @@ public static Stream getAllDataFiles( throw new UncheckedIOException(e); } }); - } catch (final Exception e) { + } catch (final RuntimeException e) { throw new TableDataException( String.format("%s:%d:%s - error reading manifest file", table, snapshot.snapshotId(), manifestFile), e); } }); - } catch (final Exception e) { + } catch (final RuntimeException e) { throw new TableDataException( String.format("%s:%d - error retrieving manifest files", table, 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 8b9c1e102b5..545fa21132c 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 @@ -223,6 +223,12 @@ static io.deephaven.qst.type.Type convertToDHType(@NotNull final Type iceberg } } + /** + * Convert a Deephaven type to an Iceberg type. + * + * @param columnType The Deephaven type to be converted. + * @return The converted Iceberg type. + */ @VisibleForTesting static Type convertToIcebergType(final Class columnType) { if (columnType == Boolean.class) { @@ -250,7 +256,7 @@ static Type convertToIcebergType(final Class columnType) { } else { throw new TableDataException("Unsupported deephaven column type " + columnType.getName()); } - // TODO Add support for writing and appending big decimals + // TODO Add support for writing big decimals // TODO Add support for reading and writing lists } @@ -479,7 +485,7 @@ private Snapshot getSnapshot(@NotNull final TableIdentifier tableIdentifier, fin private Map getRenameColumnMap( @NotNull final org.apache.iceberg.Table table, @NotNull final Schema schema, - @NotNull final IcebergBaseInstructions instructions) { + @NotNull final IcebergInstructions instructions) { final Set takenNames = new HashSet<>(); @@ -524,7 +530,7 @@ private Map getRenameColumnMap( */ public TableDefinition getTableDefinition( @NotNull final String tableIdentifier, - @Nullable final IcebergBaseInstructions instructions) { + @Nullable final IcebergInstructions instructions) { final TableIdentifier tableId = TableIdentifier.parse(tableIdentifier); // Load the table from the catalog. return getTableDefinition(tableId, instructions); @@ -540,7 +546,7 @@ public TableDefinition getTableDefinition( */ public TableDefinition getTableDefinition( @NotNull final TableIdentifier tableIdentifier, - @Nullable final IcebergBaseInstructions instructions) { + @Nullable final IcebergInstructions instructions) { // Load the table from the catalog. return getTableDefinitionInternal(tableIdentifier, null, instructions); } @@ -557,7 +563,7 @@ public TableDefinition getTableDefinition( public TableDefinition getTableDefinition( @NotNull final String tableIdentifier, final long snapshotId, - @Nullable final IcebergBaseInstructions instructions) { + @Nullable final IcebergInstructions instructions) { final TableIdentifier tableId = TableIdentifier.parse(tableIdentifier); // Find the snapshot with the given snapshot id @@ -582,7 +588,7 @@ public TableDefinition getTableDefinition( public TableDefinition getTableDefinition( @NotNull final TableIdentifier tableIdentifier, @Nullable final Snapshot tableSnapshot, - @Nullable final IcebergBaseInstructions instructions) { + @Nullable final IcebergInstructions instructions) { // Load the table from the catalog. return getTableDefinitionInternal(tableIdentifier, tableSnapshot, instructions); } @@ -597,7 +603,7 @@ public TableDefinition getTableDefinition( */ public Table getTableDefinitionTable( @NotNull final String tableIdentifier, - @Nullable final IcebergBaseInstructions instructions) { + @Nullable final IcebergInstructions instructions) { final TableIdentifier tableId = TableIdentifier.parse(tableIdentifier); return getTableDefinitionTable(tableId, instructions); } @@ -612,7 +618,7 @@ public Table getTableDefinitionTable( */ public Table getTableDefinitionTable( @NotNull final TableIdentifier tableIdentifier, - @Nullable final IcebergBaseInstructions instructions) { + @Nullable final IcebergInstructions instructions) { final TableDefinition definition = getTableDefinition(tableIdentifier, instructions); return TableTools.metaTable(definition); } @@ -629,7 +635,7 @@ public Table getTableDefinitionTable( public Table getTableDefinitionTable( @NotNull final String tableIdentifier, final long snapshotId, - @Nullable final IcebergBaseInstructions instructions) { + @Nullable final IcebergInstructions instructions) { final TableIdentifier tableId = TableIdentifier.parse(tableIdentifier); // Find the snapshot with the given snapshot id @@ -653,7 +659,7 @@ public Table getTableDefinitionTable( public Table getTableDefinitionTable( @NotNull final TableIdentifier tableIdentifier, @Nullable final Snapshot tableSnapshot, - @Nullable final IcebergBaseInstructions instructions) { + @Nullable final IcebergInstructions instructions) { final TableDefinition definition = getTableDefinition(tableIdentifier, tableSnapshot, instructions); return TableTools.metaTable(definition); } @@ -664,7 +670,7 @@ public Table getTableDefinitionTable( private TableDefinition getTableDefinitionInternal( @NotNull final TableIdentifier tableIdentifier, @Nullable final Snapshot tableSnapshot, - @Nullable final IcebergBaseInstructions instructions) { + @Nullable final IcebergInstructions instructions) { final org.apache.iceberg.Table table = catalog.loadTable(tableIdentifier); if (table == null) { throw new IllegalArgumentException("Table not found: " + tableIdentifier); @@ -673,7 +679,7 @@ private TableDefinition getTableDefinitionInternal( final Snapshot snapshot = tableSnapshot != null ? tableSnapshot : table.currentSnapshot(); final Schema schema = snapshot != null ? table.schemas().get(snapshot.schemaId()) : table.schema(); - final IcebergBaseInstructions userInstructions = + final IcebergInstructions userInstructions = instructions == null ? IcebergInstructions.DEFAULT : instructions; return fromSchema(schema, @@ -691,7 +697,7 @@ private TableDefinition getTableDefinitionInternal( */ public Table readTable( @NotNull final TableIdentifier tableIdentifier, - @Nullable final IcebergBaseInstructions instructions) { + @Nullable final IcebergInstructions instructions) { return readTableInternal(tableIdentifier, null, instructions); } @@ -704,7 +710,7 @@ public Table readTable( */ public Table readTable( @NotNull final String tableIdentifier, - @Nullable final IcebergBaseInstructions instructions) { + @Nullable final IcebergInstructions instructions) { return readTable(TableIdentifier.parse(tableIdentifier), instructions); } @@ -767,7 +773,7 @@ public Table readTable(@NotNull final String tableIdentifier, final long tableSn public Table readTable( @NotNull final TableIdentifier tableIdentifier, final long tableSnapshotId, - @Nullable final IcebergBaseInstructions instructions) { + @Nullable final IcebergInstructions instructions) { // Find the snapshot with the given snapshot id final Snapshot tableSnapshot = getSnapshot(tableIdentifier, tableSnapshotId); if (tableSnapshot == null) { @@ -787,7 +793,7 @@ public Table readTable( public Table readTable( @NotNull final String tableIdentifier, final long tableSnapshotId, - @Nullable final IcebergBaseInstructions instructions) { + @Nullable final IcebergInstructions instructions) { return readTable(TableIdentifier.parse(tableIdentifier), tableSnapshotId, instructions); } @@ -802,14 +808,14 @@ public Table readTable( public Table readTable( @NotNull final TableIdentifier tableIdentifier, @NotNull final Snapshot tableSnapshot, - @Nullable final IcebergBaseInstructions instructions) { + @Nullable final IcebergInstructions instructions) { return readTableInternal(tableIdentifier, tableSnapshot, instructions); } private Table readTableInternal( @NotNull final TableIdentifier tableIdentifier, @Nullable final Snapshot tableSnapshot, - @Nullable final IcebergBaseInstructions instructions) { + @Nullable final IcebergInstructions instructions) { // Load the table from the catalog. final org.apache.iceberg.Table table = catalog.loadTable(tableIdentifier); if (table == null) { @@ -824,8 +830,7 @@ private Table readTableInternal( final org.apache.iceberg.PartitionSpec partitionSpec = table.spec(); // Get default instructions if none are provided - final IcebergBaseInstructions userInstructions = - instructions == null ? IcebergInstructions.DEFAULT : instructions; + final IcebergInstructions userInstructions = instructions == null ? IcebergInstructions.DEFAULT : instructions; // Get the user supplied table definition. final TableDefinition userTableDef = userInstructions.tableDefinition().orElse(null); @@ -1067,7 +1072,7 @@ public List write( } /** - * Appends or overwrites data in an Iceberg table with the provided Deephaven table. + * Appends or overwrites data in an Iceberg table with the provided Deephaven tables. * * @param tableIdentifier The identifier for the Iceberg table to append to or overwrite * @param dhTables The Deephaven tables to write @@ -1096,7 +1101,7 @@ private List writeImpl( for (int idx = 1; idx < dhTables.length; idx++) { if (!firstDefinition.equals(dhTables[idx].getDefinition())) { throw new IllegalArgumentException( - "All Deephacen tables must have the same definition, else table definition shuold be " + + "All Deephaven tables must have the same definition, else table definition should be " + "provided when writing multiple tables with different definitions"); } } @@ -1354,7 +1359,7 @@ private static void commit( deletes.commit(); // Update the spec and schema of the existing table. - // If we have already verified the schema, we don't need to update it again. + // If we have already verified the schema, we don't need to update it. if (!writeInstructions.verifySchema()) { if (!icebergTable.schema().sameSchema(newSpecAndSchema.schema)) { final UpdateSchema updateSchema = icebergTransaction.updateSchema().allowIncompatibleChanges(); @@ -1391,7 +1396,7 @@ private static List dataFilesFromParquet( if (parquetFilesWritten.isEmpty()) { throw new UncheckedDeephavenException("Failed to generate data files because no parquet files written"); } - // TODO This assumes no partition data is written + // TODO This assumes no partition data is written, is that okay? return parquetFilesWritten.stream() .map(parquetFileWritten -> DataFiles.builder(PartitionSpec.unpartitioned()) .withPath(parquetFileWritten.destination().toString()) diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java index 724e8987176..db46b09302d 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java @@ -14,7 +14,6 @@ @BuildableStyle // TODO I propose renaming, but this will be breaking change: // IcebergInstructions -> IcebergReadInstructions -// IcebergBaseInstructions -> IcebergInstructions public abstract class IcebergInstructions implements IcebergBaseInstructions { /** * The default {@link IcebergInstructions} to use when reading Iceberg data files. Providing this will use system diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java index 31cf7eb2f60..eb74e2ea102 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java @@ -16,9 +16,8 @@ import static io.deephaven.parquet.table.ParquetInstructions.MIN_TARGET_PAGE_SIZE; /** - * This class provides instructions intended for writing Iceberg tables as Parquet data files as well as reading for - * reading Iceberg catalogs and tables. The default values documented in this class may change in the future. As such, - * callers may wish to explicitly set the values. + * This class provides instructions 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. */ @Immutable @BuildableStyle @@ -136,7 +135,7 @@ final void boundsCheckMaxDictionarySize() { } @Check - final void boundsCheckMinTargetPageSize() { + 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/IcebergWriteInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteInstructions.java index 8f256bdd1a6..92ea5fe0fea 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteInstructions.java @@ -6,14 +6,13 @@ import org.immutables.value.Value.Default; /** - * This class provides instructions intended for writing Iceberg tables as well as for reading Iceberg catalogs and - * tables. The default values documented in this class may change in the future. As such, callers may wish to explicitly - * set the values. + * This class provides instructions intended for writing Iceberg tables. The default values documented in this class may + * change in the future. As such, callers may wish to explicitly set the values. */ public abstract class IcebergWriteInstructions implements IcebergBaseInstructions { /** - * While appending a partition to an iceberg table, whether to create the iceberg table if it does not exist; - * defaults to {@code false}. + * While writing to an iceberg table, whether to create the iceberg table if it does not exist, defaults to + * {@code false}. */ @Default public boolean createTableIfNotExist() { @@ -21,8 +20,8 @@ public boolean createTableIfNotExist() { } /** - * While appending a partition to an iceberg table, whether to verify that the partition spec and schema of the - * table being appended is consistent with the iceberg table; defaults to {@code false}. + * While writing to an iceberg table, whether to verify that the partition spec and schema of the table being + * written is consistent with the iceberg table; defaults to {@code false}. */ @Default public boolean verifySchema() { diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3CompletableOutputStream.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3CompletableOutputStream.java index 1db1a3c2957..af74d3cf3ae 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3CompletableOutputStream.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3CompletableOutputStream.java @@ -344,7 +344,7 @@ private void completeMultipartUpload() throws IOException { */ private void abortMultipartUpload() throws IOException { if (uploadId == null) { - // Upload not started, or already completed/aborted + // We didn't start the upload, so nothing to abort return; } final AbortMultipartUploadRequest abortRequest = AbortMultipartUploadRequest.builder() From 131a5523b2e8a1b88b20d3a7a9b151e65e6b8cbf Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Tue, 1 Oct 2024 12:16:20 -0500 Subject: [PATCH 09/28] Preparing for review Part 2 --- .../IcebergParquetWriteInstructionsTest.java | 22 +++---- .../iceberg/util/IcebergToolsTest.java | 2 +- .../iceberg/util/IcebergCatalogAdapter.java | 38 ++++++++--- .../util/IcebergParquetWriteInstructions.java | 6 +- .../deephaven/iceberg/CatalogAdapterTest.java | 2 +- .../parquet/table/ParquetInstructions.java | 64 +++++++------------ .../deephaven/parquet/table/ParquetTools.java | 38 +++-------- 7 files changed, 76 insertions(+), 96 deletions(-) diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java index 41fb92daa74..03398f64f9f 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java @@ -4,19 +4,16 @@ package io.deephaven.iceberg.util; import io.deephaven.parquet.table.ParquetInstructions; -import io.deephaven.test.types.OutOfBandTest; -import org.junit.experimental.categories.Category; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.Map; import static org.assertj.core.api.AssertionsForClassTypes.assertThat; -@Category(OutOfBandTest.class) -public class IcebergParquetWriteInstructionsTest { +class IcebergParquetWriteInstructionsTest { @Test - public void defaults() { + void defaults() { final IcebergParquetWriteInstructions instructions = IcebergParquetWriteInstructions.builder().build(); assertThat(instructions.tableDefinition().isEmpty()).isTrue(); assertThat(instructions.dataInstructions().isEmpty()).isTrue(); @@ -29,7 +26,7 @@ public void defaults() { } @Test - public void testSetCreateTableIfNotExist() { + void testSetCreateTableIfNotExist() { assertThat(IcebergParquetWriteInstructions.builder() .createTableIfNotExist(true) .build() @@ -38,7 +35,7 @@ public void testSetCreateTableIfNotExist() { } @Test - public void testSetVerifySchema() { + void testSetVerifySchema() { assertThat(IcebergParquetWriteInstructions.builder() .verifySchema(true) .build() @@ -56,7 +53,7 @@ public void testSetCompressionCodecName() { } @Test - public void testSetMaximumDictionaryKeys() { + void testSetMaximumDictionaryKeys() { assertThat(IcebergParquetWriteInstructions.builder() .maximumDictionaryKeys(100) .build() @@ -83,7 +80,7 @@ public void testSetTargetPageSize() { } @Test - public void testMinMaximumDictionaryKeys() { + void testMinMaximumDictionaryKeys() { try { IcebergParquetWriteInstructions.builder() .maximumDictionaryKeys(-1) @@ -105,7 +102,7 @@ public void testMinMaximumDictionarySize() { } @Test - public void testMinTargetPageSize() { + void testMinTargetPageSize() { try { IcebergParquetWriteInstructions.builder() .targetPageSize(1024) @@ -116,7 +113,7 @@ public void testMinTargetPageSize() { } @Test - public void toParquetInstructionTest() { + void toParquetInstructionTest() { final IcebergParquetWriteInstructions icebergInstructions = IcebergParquetWriteInstructions.builder() .compressionCodecName("GZIP") .maximumDictionaryKeys(100) @@ -133,5 +130,6 @@ public void toParquetInstructionTest() { assertThat(parquetInstructions.getFieldId("field1")).isEmpty(); assertThat(parquetInstructions.getFieldId("field2")).hasValue(2); assertThat(parquetInstructions.getFieldId("field3")).hasValue(3); + assertThat(parquetInstructions.onWriteCompleted()).isEmpty(); } } 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 f72acd72e4b..aaa19f57645 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 @@ -131,7 +131,7 @@ void setUp() throws Exception { } @AfterEach - public void tearDown() throws ExecutionException, InterruptedException { + void tearDown() throws ExecutionException, InterruptedException { for (String key : keys) { asyncClient.deleteObject(DeleteObjectRequest.builder().bucket(bucket).key(key).build()).get(); } 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 545fa21132c..8ae2888ecd8 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 @@ -52,6 +52,7 @@ import org.jetbrains.annotations.Nullable; import java.math.BigDecimal; +import java.net.URI; import java.time.Instant; import java.time.LocalDateTime; import java.time.LocalDate; @@ -679,8 +680,7 @@ private TableDefinition getTableDefinitionInternal( final Snapshot snapshot = tableSnapshot != null ? tableSnapshot : table.currentSnapshot(); final Schema schema = snapshot != null ? table.schemas().get(snapshot.schemaId()) : table.schema(); - final IcebergInstructions userInstructions = - instructions == null ? IcebergInstructions.DEFAULT : instructions; + final IcebergInstructions userInstructions = instructions == null ? IcebergInstructions.DEFAULT : instructions; return fromSchema(schema, table.spec(), @@ -1139,7 +1139,7 @@ private List writeImpl( } try { - final List parquetFileinfo = + final List parquetFileinfo = writeParquet(icebergTable, dhTables, writeInstructions); final List appendFiles = dataFilesFromParquet(parquetFileinfo); if (addSnapshot) { @@ -1314,14 +1314,32 @@ private org.apache.iceberg.Table createNewIcebergTable( StringUtils.toLowerCase(writeInstructions.compressionCodecName()))); } + private static class CompletedParquetWrite { + private final URI destination; + private final long numRows; + private final long numBytes; + + private CompletedParquetWrite(final URI destination, final long numRows, final long numBytes) { + this.destination = destination; + this.numRows = numRows; + this.numBytes = numBytes; + } + } + @NotNull - private static List writeParquet( + private static List writeParquet( @NotNull final org.apache.iceberg.Table icebergTable, @NotNull final Table[] dhTables, @NotNull final IcebergParquetWriteInstructions writeInstructions) { - final List parquetFilesWritten = new ArrayList<>(dhTables.length); + // Build the parquet instructions + final List parquetFilesWritten = new ArrayList<>(dhTables.length); + final ParquetInstructions.OnWriteCompleted onWriteCompleted = + (destination, numRows, numBytes) -> parquetFilesWritten + .add(new CompletedParquetWrite(destination, numRows, numBytes)); final ParquetInstructions parquetInstructions = writeInstructions.toParquetInstructions( - parquetFilesWritten, icebergTable.schema().idToName()); + onWriteCompleted, icebergTable.schema().idToName()); + + // Write the data to parquet files for (final Table dhTable : dhTables) { final long epochMicrosNow = Instant.now().getEpochSecond() * 1_000_000 + Instant.now().getNano() / 1_000; final String filename = new StringBuilder() @@ -1392,17 +1410,17 @@ private static void commit( * Generate a list of {@link DataFile} objects from a list of parquet files written. */ private static List dataFilesFromParquet( - @NotNull final Collection parquetFilesWritten) { + @NotNull final Collection parquetFilesWritten) { if (parquetFilesWritten.isEmpty()) { throw new UncheckedDeephavenException("Failed to generate data files because no parquet files written"); } // TODO This assumes no partition data is written, is that okay? return parquetFilesWritten.stream() .map(parquetFileWritten -> DataFiles.builder(PartitionSpec.unpartitioned()) - .withPath(parquetFileWritten.destination().toString()) + .withPath(parquetFileWritten.destination.toString()) .withFormat(FileFormat.PARQUET) - .withRecordCount(parquetFileWritten.numRows()) - .withFileSizeInBytes(parquetFileWritten.numBytes()) + .withRecordCount(parquetFileWritten.numRows) + .withFileSizeInBytes(parquetFileWritten.numBytes) .build()) .collect(Collectors.toList()); } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java index eb74e2ea102..4f4d3380d3a 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java @@ -75,11 +75,11 @@ public int targetPageSize() { /** * Convert this {@link IcebergParquetWriteInstructions} to a {@link ParquetInstructions}. * - * @param completedWrites List of completed writes to be set in the {@link ParquetInstructions} + * @param onWriteCompleted The callback to be invoked after writing the parquet file. * @param fieldIdToName Mapping of field id to field name, to be populated inside the parquet file's schema */ ParquetInstructions toParquetInstructions( - @NotNull final List completedWrites, + @NotNull final ParquetInstructions.OnWriteCompleted onWriteCompleted, @NotNull final Map fieldIdToName) { final ParquetInstructions.Builder builder = new ParquetInstructions.Builder(); @@ -99,7 +99,7 @@ ParquetInstructions toParquetInstructions( builder.setMaximumDictionaryKeys(maximumDictionaryKeys()); builder.setMaximumDictionarySize(maximumDictionarySize()); builder.setTargetPageSize(targetPageSize()); - builder.setCompletedWrites(completedWrites); + builder.setOnWriteCompleted(onWriteCompleted); return builder.build(); } diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java index 2f0372f3f76..492b783b2fa 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java @@ -19,7 +19,7 @@ import static io.deephaven.engine.testutil.TstUtils.assertTableEquals; import static org.assertj.core.api.Assertions.assertThat; -public class CatalogAdapterTest extends CatalogAdapterBase { +class CatalogAdapterTest extends CatalogAdapterBase { @Test void empty() { assertThat(catalogAdapter.listNamespaces()).isEmpty(); 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 a6843919838..a0b57161ee0 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 @@ -48,6 +48,10 @@ public abstract class ParquetInstructions implements ColumnToCodecMappings { private static final boolean DEFAULT_IS_REFRESHING = false; + public interface OnWriteCompleted { + void onWriteCompleted(URI destination, long numRows, long numBytes); + } + public enum ParquetFileLayout { /** * A single parquet file. @@ -179,35 +183,11 @@ public abstract ParquetInstructions withTableDefinitionAndLayout(final TableDefi */ public abstract String baseNameForPartitionedParquetData(); - public static class CompletedWrite { - private final URI destination; - private final long numRows; - private final long numBytes; - - CompletedWrite(final URI destination, final long numRows, final long numBytes) { - this.destination = destination; - this.numRows = numRows; - this.numBytes = numBytes; - } - - public URI destination() { - return destination; - } - - public long numRows() { - return numRows; - } - - public long numBytes() { - return numBytes; - } - } - /** - * @return A list to be populated with information about all the parquet files written, ignoring the index and - * metadata files. + * @return A callback to be executed when on completing each parquet data file write (excluding the index and + * metadata files). */ - public abstract Optional> completedWrites(); + public abstract Optional onWriteCompleted(); @VisibleForTesting public static boolean sameColumnNamesAndCodecMappings(final ParquetInstructions i1, final ParquetInstructions i2) { @@ -347,7 +327,7 @@ ParquetInstructions withIndexColumns(final Collection> indexColumns } @Override - public Optional> completedWrites() { + public Optional onWriteCompleted() { return Optional.empty(); } }; @@ -432,7 +412,7 @@ private static final class ReadOnly extends ParquetInstructions { private final ParquetFileLayout fileLayout; private final TableDefinition tableDefinition; private final Collection> indexColumns; - private final List completedWrites; + private final OnWriteCompleted onWriteCompleted; private ReadOnly( final KeyedObjectHashMap columnNameToInstructions, @@ -449,7 +429,7 @@ private ReadOnly( final ParquetFileLayout fileLayout, final TableDefinition tableDefinition, final Collection> indexColumns, - final List completedWrites) { + final OnWriteCompleted onWriteCompleted) { this.columnNameToInstructions = columnNameToInstructions; this.parquetColumnNameToInstructions = parquetColumnNameToColumnName; this.compressionCodecName = compressionCodecName; @@ -467,7 +447,7 @@ private ReadOnly( : indexColumns.stream() .map(List::copyOf) .collect(Collectors.toUnmodifiableList()); - this.completedWrites = completedWrites; + this.onWriteCompleted = onWriteCompleted; } private VALUE_TYPE getOrDefault(final String columnName, final VALUE_TYPE defaultValue, @@ -610,7 +590,7 @@ public ParquetInstructions withTableDefinitionAndLayout( getCompressionCodecName(), getMaximumDictionaryKeys(), getMaximumDictionarySize(), isLegacyParquet(), getTargetPageSize(), isRefreshing(), getSpecialInstructions(), generateMetadataFiles(), baseNameForPartitionedParquetData(), useLayout, useDefinition, - indexColumns, completedWrites); + indexColumns, onWriteCompleted); } @Override @@ -619,12 +599,12 @@ ParquetInstructions withIndexColumns(final Collection> useIndexColu getCompressionCodecName(), getMaximumDictionaryKeys(), getMaximumDictionarySize(), isLegacyParquet(), getTargetPageSize(), isRefreshing(), getSpecialInstructions(), generateMetadataFiles(), baseNameForPartitionedParquetData(), fileLayout, - tableDefinition, useIndexColumns, completedWrites); + tableDefinition, useIndexColumns, onWriteCompleted); } @Override - public Optional> completedWrites() { - return Optional.ofNullable(completedWrites); + public Optional onWriteCompleted() { + return Optional.ofNullable(onWriteCompleted); } KeyedObjectHashMap copyColumnNameToInstructions() { @@ -683,7 +663,7 @@ public static class Builder { private ParquetFileLayout fileLayout; private TableDefinition tableDefinition; private Collection> indexColumns; - private List completedWrites; + private OnWriteCompleted onWriteCompleted; private TIntObjectHashMap usedFieldIdToColumn; /** @@ -712,7 +692,7 @@ public Builder(final ParquetInstructions parquetInstructions) { fileLayout = readOnlyParquetInstructions.getFileLayout().orElse(null); tableDefinition = readOnlyParquetInstructions.getTableDefinition().orElse(null); indexColumns = readOnlyParquetInstructions.getIndexColumns().orElse(null); - completedWrites = readOnlyParquetInstructions.completedWrites().orElse(null); + onWriteCompleted = readOnlyParquetInstructions.onWriteCompleted().orElse(null); } private void newColumnNameToInstructionsMap() { @@ -1027,8 +1007,12 @@ public Builder addAllIndexColumns(final Iterable> indexColumns) { return this; } - public Builder setCompletedWrites(final List completedWrites) { - this.completedWrites = completedWrites; + /** + * 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; } @@ -1041,7 +1025,7 @@ public ParquetInstructions build() { return new ReadOnly(columnNameToInstructionsOut, parquetColumnNameToColumnNameOut, compressionCodecName, maximumDictionaryKeys, maximumDictionarySize, isLegacyParquet, targetPageSize, isRefreshing, specialInstructions, generateMetadataFiles, baseNameForPartitionedParquetData, fileLayout, - tableDefinition, indexColumns, completedWrites); + tableDefinition, indexColumns, onWriteCompleted); } } 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 790a0c299a2..43aa8b5fa2e 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 @@ -587,14 +587,6 @@ private static void writeTablesImpl( metadataFileWriter = NullParquetMetadataFileWriter.INSTANCE; } - final boolean collectCompletedWrites = writeInstructions.completedWrites().isPresent(); - final List completedWrites; - if (collectCompletedWrites) { - completedWrites = new ArrayList<>(destinations.length); - } else { - completedWrites = null; - } - // List of output streams created, to rollback in case of exceptions final List outputStreams = new ArrayList<>(destinations.length); try (final SafeCloseable ignored = () -> SafeCloseable.closeAll(outputStreams.stream())) { @@ -603,16 +595,15 @@ 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); final long numBytes = ParquetTableWriter.write(source, definition, writeInstructions, - destinations[tableIdx], outputStream, Collections.emptyMap(), + tableDestination, outputStream, Collections.emptyMap(), (List) null, metadataFileWriter, computedCache); - if (collectCompletedWrites) { - completedWrites.add(new ParquetInstructions.CompletedWrite(destinations[tableIdx], - source.size(), numBytes)); - } + writeInstructions.onWriteCompleted().ifPresent(callback -> callback.onWriteCompleted( + tableDestination, source.size(), numBytes)); } } else { // Shared parquet column names across all tables @@ -633,14 +624,12 @@ private static void writeTablesImpl( for (final ParquetTableWriter.IndexWritingInfo info : indexInfoList) { outputStreams.add(info.destOutputStream); } - final Table sourceTable = sources[tableIdx]; - final long numBytes = ParquetTableWriter.write(sourceTable, definition, writeInstructions, + final Table source = sources[tableIdx]; + final long numBytes = ParquetTableWriter.write(source, definition, writeInstructions, tableDestination, outputStream, Collections.emptyMap(), indexInfoList, metadataFileWriter, computedCache); - if (collectCompletedWrites) { - completedWrites.add(new ParquetInstructions.CompletedWrite(tableDestination, - sourceTable.size(), numBytes)); - } + writeInstructions.onWriteCompleted().ifPresent(callback -> callback.onWriteCompleted( + tableDestination, source.size(), numBytes)); } } @@ -669,18 +658,9 @@ private static void writeTablesImpl( log.error().append("Error in rolling back output stream ").append(e1).endl(); } } - if (completedWrites != null) { - completedWrites.clear(); - } throw new UncheckedDeephavenException("Error writing parquet tables", e); } } - if (collectCompletedWrites && !completedWrites.isEmpty()) { - // Add the completed writes to the list provided in the instructions - final List userCompletedWritesList = - writeInstructions.completedWrites().get(); - userCompletedWritesList.addAll(completedWrites); - } } /** From 302158537e0075de77756f85f15f9e88e12a7ac3 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Tue, 1 Oct 2024 13:59:49 -0500 Subject: [PATCH 10/28] Added more unit tests --- .../iceberg/util/IcebergCatalogAdapter.java | 21 ++- .../deephaven/iceberg/CatalogAdapterTest.java | 131 +++++++++++++++++- 2 files changed, 148 insertions(+), 4 deletions(-) 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 8ae2888ecd8..90d6369a30e 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 @@ -1113,10 +1113,12 @@ private List writeImpl( // Try loading the table from the catalog, or create if required final org.apache.iceberg.Table icebergTable; final SpecAndSchema newSpecAndSchema; + final boolean newNamespaceCreated; final boolean newTableCreated; if (catalog.tableExists(tableIdentifier)) { icebergTable = catalog.loadTable(tableIdentifier); newSpecAndSchema = fromTableDefinition(useDefinition, userInstructions); + newNamespaceCreated = false; newTableCreated = false; if (writeInstructions.verifySchema()) { if (overwrite) { @@ -1128,7 +1130,7 @@ private List writeImpl( } } } else if (writeInstructions.createTableIfNotExist()) { - createNamespaceIfNotExists(tableIdentifier.namespace()); + newNamespaceCreated = createNamespaceIfNotExists(tableIdentifier.namespace()); newSpecAndSchema = fromTableDefinition(useDefinition, userInstructions); icebergTable = createNewIcebergTable(tableIdentifier, newSpecAndSchema, writeInstructions); newTableCreated = true; @@ -1155,6 +1157,10 @@ private List writeImpl( writeException.addSuppressed(dropException); } } + if (newNamespaceCreated) { + // Delete it to avoid leaving a partial namespace in the catalog + dropNamespaceIfExists(tableIdentifier.namespace()); + } throw writeException; } } @@ -1244,14 +1250,23 @@ private static IcebergParquetWriteInstructions verifyWriteInstructions( return (IcebergParquetWriteInstructions) instructions; } - private Namespace createNamespaceIfNotExists(@NotNull final Namespace namespace) { + + private boolean createNamespaceIfNotExists(@NotNull final Namespace namespace) { if (catalog instanceof SupportsNamespaces) { final SupportsNamespaces nsCatalog = (SupportsNamespaces) catalog; if (!nsCatalog.namespaceExists(namespace)) { nsCatalog.createNamespace(namespace); + return true; } } - return namespace; + return false; + } + + private void dropNamespaceIfExists(@NotNull final Namespace namespace) { + if (catalog instanceof SupportsNamespaces) { + final SupportsNamespaces nsCatalog = (SupportsNamespaces) catalog; + nsCatalog.dropNamespace(namespace); + } } private static class SpecAndSchema { diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java index 492b783b2fa..4a0500092e0 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java @@ -3,13 +3,16 @@ // package io.deephaven.iceberg; +import io.deephaven.UncheckedDeephavenException; import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.table.impl.select.FormulaEvaluationException; import io.deephaven.engine.util.TableTools; import io.deephaven.iceberg.junit5.CatalogAdapterBase; import io.deephaven.iceberg.util.IcebergParquetWriteInstructions; import io.deephaven.iceberg.util.IcebergWriteInstructions; +import junit.framework.TestCase; import org.apache.iceberg.Schema; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; @@ -217,5 +220,131 @@ void appendWithDifferentDefinition() { assertTableEquals(expected2, fromIceberg); } - // TODO Add more unit tests to get all data types and full coverage + @Test + void appendMultipleTablesWithDefinitionTest() { + final Table source = TableTools.emptyTable(10) + .update("intCol = (int) 2 * i + 10", + "doubleCol = (double) 2.5 * i + 10"); + final IcebergWriteInstructions writeInstructions = IcebergParquetWriteInstructions.builder() + .createTableIfNotExist(true) + .verifySchema(true) + .build(); + catalogAdapter.append("MyNamespace.MyTable", source, writeInstructions); + Table fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + assertTableEquals(source, fromIceberg); + + final Table appendTable1 = TableTools.emptyTable(5) + .update("intCol = (int) 3 * i + 20", + "doubleCol = (double) 3.5 * i + 20", + "shortCol = (short) 3 * i + 20"); + final Table appendTable2 = TableTools.emptyTable(5) + .update( + "charCol = (char) 65 + i%26", + "intCol = (int) 4 * i + 30", + "doubleCol = (double) 4.5 * i + 30"); + + try { + catalogAdapter.append("MyNamespace.MyTable", new Table[] {appendTable1, appendTable2}, writeInstructions); + } catch (RuntimeException e) { + assertThat(e.getMessage()).contains("All Deephaven tables must have the same definition"); + } + + // Set a table definition that is compatible with all tables + final TableDefinition writeDefinition = TableDefinition.of( + ColumnDefinition.ofInt("intCol"), + ColumnDefinition.ofDouble("doubleCol")); + final IcebergWriteInstructions writeInstructionsWithDefinition = IcebergParquetWriteInstructions.builder() + .createTableIfNotExist(true) + .verifySchema(true) + .tableDefinition(writeDefinition) + .build(); + catalogAdapter.append("MyNamespace.MyTable", new Table[] {appendTable1, appendTable2}, + writeInstructionsWithDefinition); + fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + final Table expected = TableTools.merge( + source, + appendTable1.dropColumns("shortCol"), + appendTable2.dropColumns("charCol")); + assertTableEquals(expected, fromIceberg); + } + + @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 data = 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}"); + catalogAdapter.append(myTableId.toString(), data, null); + final Table fromIceberg = catalogAdapter.readTable(myTableId, null); + assertTableEquals(data, 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 IcebergWriteInstructions writeInstructions = IcebergParquetWriteInstructions.builder() + .createTableIfNotExist(true) + .verifySchema(true) + .build(); + final Namespace myNamespace = Namespace.of("MyNamespace"); + final TableIdentifier myTableId = TableIdentifier.of(myNamespace, "MyTable"); + + try { + catalogAdapter.append(myTableId, badSource, writeInstructions); + TestCase.fail("Exception expected for invalid formula in table"); + } catch (UncheckedDeephavenException e) { + assertThat(e.getCause() instanceof FormulaEvaluationException).isTrue(); + } + assertThat(catalogAdapter.listNamespaces()).isEmpty(); + + // 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"); + catalogAdapter.append(myTableId, goodSource, writeInstructions); + Table fromIceberg = catalogAdapter.readTable(myTableId, null); + assertTableEquals(goodSource, fromIceberg); + + try { + catalogAdapter.append(myTableId, badSource, writeInstructions); + TestCase.fail("Exception expected for invalid formula in table"); + } catch (UncheckedDeephavenException e) { + assertThat(e.getCause() instanceof FormulaEvaluationException).isTrue(); + } + + // Make sure existing good data is not deleted + assertThat(catalogAdapter.listNamespaces()).contains(myNamespace); + assertThat(catalogAdapter.listTables(myNamespace)).containsExactly(myTableId); + fromIceberg = catalogAdapter.readTable(myTableId, null); + assertTableEquals(goodSource, fromIceberg); + } } From 9f82ba06682604df27ef9951b4f944302fa2706f Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Thu, 3 Oct 2024 11:09:52 -0500 Subject: [PATCH 11/28] Review with Larry part 1 --- .../IcebergParquetWriteInstructionsTest.java | 4 +- .../iceberg/util/IcebergToolsTest.java | 5 +- .../deephaven/iceberg/base/IcebergUtils.java | 190 ++++++++++++++ .../iceberg/util/IcebergCatalogAdapter.java | 238 ++++-------------- .../util/IcebergWriteInstructions.java | 26 +- .../deephaven/iceberg/CatalogAdapterTest.java | 12 +- 6 files changed, 268 insertions(+), 207 deletions(-) diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java index 03398f64f9f..d9a0e436367 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java @@ -19,7 +19,7 @@ void defaults() { assertThat(instructions.dataInstructions().isEmpty()).isTrue(); assertThat(instructions.columnRenames().isEmpty()).isTrue(); assertThat(instructions.createTableIfNotExist()).isFalse(); - assertThat(instructions.verifySchema()).isFalse(); + assertThat(instructions.verifySchema()).isEmpty(); assertThat(instructions.compressionCodecName()).isEqualTo("SNAPPY"); assertThat(instructions.maximumDictionaryKeys()).isEqualTo(1048576); assertThat(instructions.maximumDictionarySize()).isEqualTo(1048576); @@ -40,7 +40,7 @@ void testSetVerifySchema() { .verifySchema(true) .build() .verifySchema()) - .isTrue(); + .isEqualTo(true); } @Test 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 aaa19f57645..ae8a251d794 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 @@ -12,6 +12,7 @@ import io.deephaven.engine.testutil.junit4.EngineCleanup; import io.deephaven.extensions.s3.S3Instructions; import io.deephaven.iceberg.TestCatalog.IcebergTestCatalog; +import io.deephaven.iceberg.base.IcebergUtils; import org.apache.iceberg.Snapshot; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; @@ -903,10 +904,10 @@ void testConvertToIcebergTypeAndBack() { for (final Class javaType : javaTypes) { // Java type -> Iceberg type - final Type icebergType = IcebergCatalogAdapter.convertToIcebergType(javaType); + final Type icebergType = IcebergUtils.convertToIcebergType(javaType); // Iceberg type -> Deephaven type - final io.deephaven.qst.type.Type deephavenType = IcebergCatalogAdapter.convertToDHType(icebergType); + 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/src/main/java/io/deephaven/iceberg/base/IcebergUtils.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/base/IcebergUtils.java index ddb9fc26409..2558ea55ade 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/base/IcebergUtils.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/base/IcebergUtils.java @@ -3,20 +3,34 @@ // package io.deephaven.iceberg.base; +import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.impl.locations.TableDataException; import org.apache.iceberg.DataFile; import org.apache.iceberg.ManifestContent; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.ManifestFiles; import org.apache.iceberg.ManifestReader; +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.io.FileIO; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; import org.jetbrains.annotations.NotNull; 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.List; +import java.util.Set; +import java.util.stream.Collectors; import java.util.stream.Stream; import java.util.stream.StreamSupport; @@ -68,4 +82,180 @@ public static Stream getAllDataFiles( String.format("%s:%d - error retrieving manifest files", table, snapshot.snapshotId()), 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; + 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(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) { + if (columnType == Boolean.class) { + return Types.BooleanType.get(); + } else if (columnType == double.class) { + return Types.DoubleType.get(); + } else if (columnType == float.class) { + return Types.FloatType.get(); + } else if (columnType == int.class) { + return Types.IntegerType.get(); + } else if (columnType == long.class) { + return Types.LongType.get(); + } else if (columnType == String.class) { + return Types.StringType.get(); + } else if (columnType == Instant.class) { + return Types.TimestampType.withZone(); + } else if (columnType == LocalDateTime.class) { + return Types.TimestampType.withoutZone(); + } else if (columnType == LocalDate.class) { + return Types.DateType.get(); + } else if (columnType == LocalTime.class) { + return Types.TimeType.get(); + } else if (columnType == byte[].class) { + return Types.BinaryType.get(); + } else { + throw new TableDataException("Unsupported deephaven column type " + columnType.getName()); + } + // TODO Add support for writing big decimals + // TODO Add support for reading and writing lists + } + + /** + * Check if an existing iceberg table with provided schema is compatible for overwriting with a new table with given + * schema. + * + * @param icebergSchema The schema of the existing iceberg table. + * @param newSchema The schema of the new table. + * + * @throws IllegalArgumentException if the schemas are not compatible. + */ + public static void verifyOverwriteCompatibility( + final Schema icebergSchema, + final Schema newSchema) { + if (!icebergSchema.sameSchema(newSchema)) { + throw new IllegalArgumentException("Schema mismatch, iceberg table schema: " + icebergSchema + + ", schema derived from the table definition: " + newSchema); + } + } + + /** + * Check if an existing iceberg table with provided partition spec is compatible for overwriting with a new table + * with given partition spec. + * + * @param icebergPartitionSpec The partition spec of the existing iceberg table. + * @param newPartitionSpec The partition spec of the new table. + * + * @throws IllegalArgumentException if the partition spec are not compatible. + */ + public static void verifyOverwriteCompatibility( + final PartitionSpec icebergPartitionSpec, + final PartitionSpec newPartitionSpec) { + if (!icebergPartitionSpec.compatibleWith(newPartitionSpec)) { + throw new IllegalArgumentException("Partition spec mismatch, iceberg table partition spec: " + + icebergPartitionSpec + ", partition spec derived from table definition: " + newPartitionSpec); + } + } + + /** + * Check if an existing iceberg table with provided schema is compatible for appending deephaven table with provided + * definition. + * + * @param icebergSchema The schema of the iceberg table. + * @param tableDefinition The table definition of the deephaven table. + * + * @throws IllegalArgumentException if the schemas are not compatible. + */ + public static void verifyAppendCompatibility( + final Schema icebergSchema, + final TableDefinition tableDefinition) { + // Check that all columns in the table definition are part of the Iceberg schema and have the same type + for (final ColumnDefinition dhColumn : tableDefinition.getColumns()) { + final Types.NestedField icebergColumn = icebergSchema.findField(dhColumn.getName()); + if (icebergColumn == null || !icebergColumn.type().equals(convertToIcebergType(dhColumn.getDataType()))) { + throw new IllegalArgumentException("Schema mismatch, column " + dhColumn.getName() + " from Deephaven " + + "table definition: " + tableDefinition + " is not found or has a different type in Iceberg " + + "table schema: " + icebergSchema); + } + } + + // Check that all required columns in the Iceberg schema are part of the table definition + for (final Types.NestedField icebergColumn : icebergSchema.columns()) { + if (icebergColumn.isOptional()) { + continue; + } + if (tableDefinition.getColumn(icebergColumn.name()) == null) { + throw new IllegalArgumentException("Partition spec mismatch, required column " + icebergColumn.name() + + " from Iceberg table schema: " + icebergSchema + " not found in Deephaven table definition: " + + tableDefinition); + } + } + } + + /** + * Check if an existing iceberg table with provided partition spec is compatible for appending deephaven table with + * provided definition. + * + * @param partitionSpec The partition spec of the iceberg table. + * @param tableDefinition The table definition of the deephaven table. + * + * @throws IllegalArgumentException if the partition spec are not compatible. + */ + public static void verifyAppendCompatibility( + final PartitionSpec partitionSpec, + final TableDefinition tableDefinition) { + final Set icebergPartitionColumns = partitionSpec.fields().stream() + .map(PartitionField::name) + .collect(Collectors.toSet()); + final Set dhPartitioningColumns = tableDefinition.getColumns().stream() + .filter(ColumnDefinition::isPartitioning) + .map(ColumnDefinition::getName) + .collect(Collectors.toSet()); + if (!icebergPartitionColumns.equals(dhPartitioningColumns)) { + throw new IllegalArgumentException("Partitioning column mismatch, iceberg table partition spec: " + + partitionSpec + ", deephaven table definition: " + tableDefinition); + } + } } 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 90d6369a30e..1fa711a08b9 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 @@ -51,17 +51,17 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.math.BigDecimal; import java.net.URI; import java.time.Instant; -import java.time.LocalDateTime; -import java.time.LocalDate; -import java.time.LocalTime; import java.util.*; import java.util.stream.Collectors; import java.util.stream.Stream; +import static io.deephaven.iceberg.base.IcebergUtils.convertToDHType; +import static io.deephaven.iceberg.base.IcebergUtils.convertToIcebergType; import static io.deephaven.iceberg.base.IcebergUtils.getAllDataFiles; +import static io.deephaven.iceberg.base.IcebergUtils.verifyAppendCompatibility; +import static io.deephaven.iceberg.base.IcebergUtils.verifyOverwriteCompatibility; public class IcebergCatalogAdapter { @@ -180,87 +180,6 @@ private static TableDefinition fromSchema( return tableDef; } - /** - * Convert an Iceberg data type to a Deephaven type. - * - * @param icebergType The Iceberg data type to be converted. - * @return The converted Deephaven type. - */ - static io.deephaven.qst.type.Type 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; - 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(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. - */ - @VisibleForTesting - static Type convertToIcebergType(final Class columnType) { - if (columnType == Boolean.class) { - return Types.BooleanType.get(); - } else if (columnType == double.class) { - return Types.DoubleType.get(); - } else if (columnType == float.class) { - return Types.FloatType.get(); - } else if (columnType == int.class) { - return Types.IntegerType.get(); - } else if (columnType == long.class) { - return Types.LongType.get(); - } else if (columnType == String.class) { - return Types.StringType.get(); - } else if (columnType == Instant.class) { - return Types.TimestampType.withZone(); - } else if (columnType == LocalDateTime.class) { - return Types.TimestampType.withoutZone(); - } else if (columnType == LocalDate.class) { - return Types.DateType.get(); - } else if (columnType == LocalTime.class) { - return Types.TimeType.get(); - } else if (columnType == byte[].class) { - return Types.BinaryType.get(); - } else { - throw new TableDataException("Unsupported deephaven column type " + columnType.getName()); - } - // TODO Add support for writing big decimals - // TODO Add support for reading and writing lists - } - /** * 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)}. @@ -737,7 +656,6 @@ private Snapshot getTableSnapshot(@NotNull TableIdentifier tableIdentifier, long * @param tableSnapshotId The snapshot id to load * @return The loaded table */ - @SuppressWarnings("unused") public Table readTable(@NotNull final TableIdentifier tableIdentifier, final long tableSnapshotId) { // Find the snapshot with the given snapshot id final Snapshot tableSnapshot = getTableSnapshot(tableIdentifier, tableSnapshotId); @@ -880,13 +798,13 @@ private Table readTableInternal( /** * Returns the underlying Iceberg {@link Catalog catalog} used by this adapter. */ - @SuppressWarnings("unused") public Catalog catalog() { return catalog; } /** - * Append the provided deephaven table as a new partition to the existing iceberg table in a single snapshot. + * Append the provided deephaven table as a new partition to the existing iceberg table in a single snapshot. This + * will not change the schema of the existing table. * * @param tableIdentifier The identifier string for the iceberg table to append to * @param dhTable The deephaven table to append @@ -900,7 +818,8 @@ public void append( } /** - * Append the provided deephaven table as a new partition to the existing iceberg table in a single snapshot. + * Append the provided deephaven table as a new partition to the existing iceberg table in a single snapshot. This + * will not change the schema of the existing table. * * @param tableIdentifier The identifier for the iceberg table to append to * @param dhTable The deephaven table to append @@ -915,7 +834,8 @@ public void append( /** * Append the provided deephaven tables as new partitions to the existing iceberg table in a single snapshot. All - * tables should have the same definition, else a table definition should be provided in the instructions. + * tables should have the same definition, else a table definition should be provided in the instructions. This will + * not change the schema of the existing table. * * @param tableIdentifier The identifier string for the iceberg table to append to * @param dhTables The deephaven tables to append @@ -930,7 +850,8 @@ public void append( /** * Append the provided deephaven tables as new partitions to the existing iceberg table in a single snapshot. All - * tables should have the same definition, else a table definition should be provided in the instructions. + * tables should have the same definition, else a table definition should be provided in the instructions. This will + * not change the schema of the existing table. * * @param tableIdentifier The identifier for the iceberg table to append to * @param dhTables The deephaven tables to append @@ -944,7 +865,8 @@ public void append( } /** - * Overwrite the existing iceberg table with the provided deephaven table in a single snapshot. + * Overwrite the existing iceberg table with the provided deephaven table in a single snapshot. This will change the + * schema of the existing table to match the provided deephaven table. * * @param tableIdentifier The identifier string for the iceberg table to overwrite * @param dhTable The deephaven table to overwrite with @@ -958,7 +880,8 @@ public void overwrite( } /** - * Overwrite the existing iceberg table with the provided deephaven table in a single snapshot. + * Overwrite the existing iceberg table with the provided deephaven table in a single snapshot. This will change the + * schema of the existing table to match the provided deephaven table. * * @param tableIdentifier The identifier for the iceberg table to overwrite * @param dhTable The deephaven table to overwrite with @@ -973,7 +896,8 @@ public void overwrite( /** * Overwrite the existing iceberg table with the provided deephaven tables appended together in a single snapshot. - * All tables should have the same definition, else a table definition should be provided in the instructions. + * All tables should have the same definition, else a table definition should be provided in the instructions. This + * will change the schema of the existing table to match the provided deephaven tables. * * @param tableIdentifier The identifier string for the iceberg table to overwrite * @param dhTables The deephaven tables to overwrite with @@ -988,7 +912,8 @@ public void overwrite( /** * Overwrite the existing iceberg table with the provided deephaven tables appended together in a single snapshot. - * All tables should have the same definition, else a table definition should be provided in the instructions. + * All tables should have the same definition, else a table definition should be provided in the instructions. This + * will change the schema of the existing table to match the provided deephaven tables. * * @param tableIdentifier The identifier for the iceberg table to overwrite * @param dhTables The deephaven tables to overwrite with @@ -1089,13 +1014,13 @@ private List writeImpl( @Nullable final IcebergWriteInstructions instructions, final boolean overwrite, final boolean addSnapshot) { - final IcebergWriteInstructions userInstructions = - instructions == null ? IcebergParquetWriteInstructions.DEFAULT : instructions; + final IcebergParquetWriteInstructions writeInstructions = verifyWriteInstructions(instructions); + final boolean verifySchema = writeInstructions.verifySchema().orElse(!overwrite); // Verify that all tables have the same definition final TableDefinition useDefinition; - if (userInstructions.tableDefinition().isPresent()) { - useDefinition = userInstructions.tableDefinition().get(); + if (writeInstructions.tableDefinition().isPresent()) { + useDefinition = writeInstructions.tableDefinition().get(); } else { final TableDefinition firstDefinition = dhTables[0].getDefinition(); for (int idx = 1; idx < dhTables.length; idx++) { @@ -1108,8 +1033,6 @@ private List writeImpl( useDefinition = firstDefinition; } - final IcebergParquetWriteInstructions writeInstructions = verifyWriteInstructions(userInstructions); - // Try loading the table from the catalog, or create if required final org.apache.iceberg.Table icebergTable; final SpecAndSchema newSpecAndSchema; @@ -1117,21 +1040,27 @@ private List writeImpl( final boolean newTableCreated; if (catalog.tableExists(tableIdentifier)) { icebergTable = catalog.loadTable(tableIdentifier); - newSpecAndSchema = fromTableDefinition(useDefinition, userInstructions); + newSpecAndSchema = fromTableDefinition(useDefinition, writeInstructions); newNamespaceCreated = false; newTableCreated = false; - if (writeInstructions.verifySchema()) { - if (overwrite) { - verifyOverwriteCompatibility(icebergTable.schema(), newSpecAndSchema.schema); - verifyOverwriteCompatibility(icebergTable.spec(), newSpecAndSchema.partitionSpec); - } else { - verifyAppendCompatibility(icebergTable.schema(), useDefinition); - verifyAppendCompatibility(icebergTable.spec(), useDefinition); + if (verifySchema) { + try { + if (overwrite) { + verifyOverwriteCompatibility(icebergTable.schema(), newSpecAndSchema.schema); + verifyOverwriteCompatibility(icebergTable.spec(), newSpecAndSchema.partitionSpec); + } else { + verifyAppendCompatibility(icebergTable.schema(), useDefinition); + verifyAppendCompatibility(icebergTable.spec(), useDefinition); + } + } catch (final IllegalArgumentException e) { + throw new IllegalArgumentException("Schema verification failed. Please provide a compatible " + + "schema or disable verification in the Iceberg instructions. See the linked exception " + + "for more details.", e); } } } else if (writeInstructions.createTableIfNotExist()) { newNamespaceCreated = createNamespaceIfNotExists(tableIdentifier.namespace()); - newSpecAndSchema = fromTableDefinition(useDefinition, userInstructions); + newSpecAndSchema = fromTableDefinition(useDefinition, writeInstructions); icebergTable = createNewIcebergTable(tableIdentifier, newSpecAndSchema, writeInstructions); newTableCreated = true; } else { @@ -1145,7 +1074,7 @@ private List writeImpl( writeParquet(icebergTable, dhTables, writeInstructions); final List appendFiles = dataFilesFromParquet(parquetFileinfo); if (addSnapshot) { - commit(icebergTable, newSpecAndSchema, appendFiles, writeInstructions, overwrite && !newTableCreated); + commit(icebergTable, newSpecAndSchema, appendFiles, overwrite && !newTableCreated, verifySchema); } return appendFiles; } catch (final RuntimeException writeException) { @@ -1165,83 +1094,11 @@ private List writeImpl( } } - /** - * Check if the schema for the iceberg table is compatible for overwriting with a deephaven table with provided - * definition. - */ - private static void verifyOverwriteCompatibility( - final Schema icebergSchema, - final Schema newSchema) { - if (!icebergSchema.sameSchema(newSchema)) { - throw new IllegalArgumentException("Schema mismatch, iceberg table schema: " + icebergSchema + - ", schema derived from the table definition: " + newSchema); - } - } - - /** - * Check if the partition spec for the iceberg table is compatible for overwriting with a deephaven table with - * provided definition. - */ - private static void verifyOverwriteCompatibility( - final PartitionSpec icebergPartitionSpec, - final PartitionSpec newPartitionSpec) { - if (!icebergPartitionSpec.compatibleWith(newPartitionSpec)) { - throw new IllegalArgumentException("Partition spec mismatch, iceberg table partition spec: " + - icebergPartitionSpec + ", partition spec derived from table definition: " + newPartitionSpec); - } - } - - /** - * Check if the schema for the iceberg table is compatible for appending a deephaven table with provided definition. - */ - private static void verifyAppendCompatibility( - final Schema icebergSchema, - final TableDefinition tableDefinition) { - // Check that all columns in the table definition are part of the Iceberg schema - for (final ColumnDefinition dhColumn : tableDefinition.getColumns()) { - final Types.NestedField icebergColumn = icebergSchema.findField(dhColumn.getName()); - if (icebergColumn == null || !icebergColumn.type().equals(convertToIcebergType(dhColumn.getDataType()))) { - throw new IllegalArgumentException("Schema mismatch, column " + dhColumn.getName() + " from Deephaven " - + "table definition: " + tableDefinition + " is not found or has a different type in Iceberg " - + "table schema: " + icebergSchema); - } - } - - // Check that all required columns in the Iceberg schema are part of the table definition - for (final Types.NestedField icebergColumn : icebergSchema.columns()) { - if (icebergColumn.isOptional()) { - continue; - } - if (tableDefinition.getColumn(icebergColumn.name()) == null) { - throw new IllegalArgumentException("Partition spec mismatch, required column " + icebergColumn.name() + - " from Iceberg table schema: " + icebergSchema + " not found in Deephaven table definition: " - + tableDefinition); - } - } - } - - /** - * Check if the partition spec for the Iceberg table is compatible for appending deephaven table with provided - * definition. - */ - private static void verifyAppendCompatibility( - final PartitionSpec partitionSpec, - final TableDefinition tableDefinition) { - final Set icebergPartitionColumns = partitionSpec.fields().stream() - .map(PartitionField::name) - .collect(Collectors.toSet()); - final Set dhPartitioningColumns = tableDefinition.getColumns().stream() - .filter(ColumnDefinition::isPartitioning) - .map(ColumnDefinition::getName) - .collect(Collectors.toSet()); - if (!icebergPartitionColumns.equals(dhPartitioningColumns)) { - throw new IllegalArgumentException("Partitioning column mismatch, iceberg table partition spec: " + - partitionSpec + ", deephaven table definition: " + tableDefinition); - } - } - private static IcebergParquetWriteInstructions verifyWriteInstructions( - @NotNull final IcebergWriteInstructions instructions) { + @Nullable final IcebergWriteInstructions instructions) { + if (instructions == null) { + return IcebergParquetWriteInstructions.DEFAULT; + } // We ony support writing to Parquet files if (!(instructions instanceof IcebergParquetWriteInstructions)) { throw new IllegalArgumentException("Unsupported instructions of class " + instructions.getClass() + " for" + @@ -1250,7 +1107,6 @@ private static IcebergParquetWriteInstructions verifyWriteInstructions( return (IcebergParquetWriteInstructions) instructions; } - private boolean createNamespaceIfNotExists(@NotNull final Namespace namespace) { if (catalog instanceof SupportsNamespaces) { final SupportsNamespaces nsCatalog = (SupportsNamespaces) catalog; @@ -1377,8 +1233,8 @@ private static void commit( @NotNull final org.apache.iceberg.Table icebergTable, @NotNull final SpecAndSchema newSpecAndSchema, @NotNull final Iterable appendFiles, - @NotNull final IcebergWriteInstructions writeInstructions, - final boolean overwrite) { + final boolean overwrite, + final boolean schemaVerified) { // Append new data files to the table final Transaction icebergTransaction = icebergTable.newTransaction(); @@ -1393,7 +1249,7 @@ private static void commit( // Update the spec and schema of the existing table. // If we have already verified the schema, we don't need to update it. - if (!writeInstructions.verifySchema()) { + if (!schemaVerified) { if (!icebergTable.schema().sameSchema(newSpecAndSchema.schema)) { final UpdateSchema updateSchema = icebergTransaction.updateSchema().allowIncompatibleChanges(); icebergTable.schema().columns().stream() 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 index 92ea5fe0fea..8124e948d43 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteInstructions.java @@ -5,6 +5,8 @@ import org.immutables.value.Value.Default; +import java.util.Optional; + /** * This class provides instructions intended for writing Iceberg tables. The default values documented in this class may * change in the future. As such, callers may wish to explicitly set the values. @@ -19,14 +21,26 @@ public boolean createTableIfNotExist() { return false; } + // @formatter:off /** - * While writing to an iceberg table, whether to verify that the partition spec and schema of the table being - * written is consistent with the iceberg table; defaults to {@code false}. + * Specifies whether to verify that the partition spec and schema of the table being written are consistent with the + * Iceberg table. + * + *

Verification behavior differs based on the operation type:

+ *
    + *
  • Appending Data or Writing Data Files: Verification is enabled by default. It ensures that: + *
      + *
    • All columns from the Deephaven table are present in the Iceberg table and have compatible types.
    • + *
    • All required columns in the Iceberg table are present in the Deephaven table.
    • + *
    • The set of partitioning columns in both the Iceberg and Deephaven tables are identical.
    • + *
    + *
  • + *
  • Overwriting Data: Verification is disabled by default. When enabled, it ensures that the + * schema and partition spec of the table being written are identical to those of the Iceberg table.
  • + *
*/ - @Default - public boolean verifySchema() { - return false; - } + public abstract Optional verifySchema(); + // @formatter:on public interface Builder extends IcebergBaseInstructions.Builder { @SuppressWarnings("unused") diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java index 4a0500092e0..133633ecec5 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java @@ -161,13 +161,12 @@ void overwriteWithDifferentDefinition() { try { catalogAdapter.overwrite("MyNamespace.MyTable", differentSource, writeInstructionsWithSchemaMatching); } catch (RuntimeException e) { - assertThat(e.getMessage()).contains("Schema mismatch"); + assertThat(e.getMessage()).contains("Schema verification failed"); } + // By default, schema verification should be disabled for overwriting final IcebergWriteInstructions writeInstructionsWithoutSchemaMatching = - IcebergParquetWriteInstructions.builder() - .verifySchema(false) - .build(); + IcebergParquetWriteInstructions.builder().build(); catalogAdapter.overwrite("MyNamespace.MyTable", differentSource, writeInstructionsWithoutSchemaMatching); fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); assertTableEquals(differentSource, fromIceberg); @@ -186,9 +185,10 @@ void appendWithDifferentDefinition() { final Table source = TableTools.emptyTable(10) .update("intCol = (int) 2 * i + 10", "doubleCol = (double) 2.5 * i + 10"); + + // By default, schema verification should be enabled for appending final IcebergWriteInstructions writeInstructions = IcebergParquetWriteInstructions.builder() .createTableIfNotExist(true) - .verifySchema(true) .build(); catalogAdapter.append("MyNamespace.MyTable", source, writeInstructions); Table fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); @@ -199,7 +199,7 @@ void appendWithDifferentDefinition() { try { catalogAdapter.append("MyNamespace.MyTable", differentSource, writeInstructions); } catch (RuntimeException e) { - assertThat(e.getMessage()).contains("Schema mismatch"); + assertThat(e.getMessage()).contains("Schema verification failed"); } // Append a table with just the int column, should be compatible with the existing schema From cbae64e08581790802a7eeb144cd7ecae3707be4 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Thu, 3 Oct 2024 11:49:02 -0500 Subject: [PATCH 12/28] Fix for failing job --- .../util/IcebergParquetWriteInstructionsTest.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java index d9a0e436367..9fa7a8243c9 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java @@ -7,6 +7,7 @@ import org.junit.jupiter.api.Test; import java.util.Map; +import java.util.Optional; import static org.assertj.core.api.AssertionsForClassTypes.assertThat; @@ -40,11 +41,11 @@ void testSetVerifySchema() { .verifySchema(true) .build() .verifySchema()) - .isEqualTo(true); + .isEqualTo(Optional.of(true)); } @Test - public void testSetCompressionCodecName() { + void testSetCompressionCodecName() { assertThat(IcebergParquetWriteInstructions.builder() .compressionCodecName("GZIP") .build() @@ -62,7 +63,7 @@ void testSetMaximumDictionaryKeys() { } @Test - public void testSetMaximumDictionarySize() { + void testSetMaximumDictionarySize() { assertThat(IcebergParquetWriteInstructions.builder() .maximumDictionarySize(100) .build() @@ -71,7 +72,7 @@ public void testSetMaximumDictionarySize() { } @Test - public void testSetTargetPageSize() { + void testSetTargetPageSize() { assertThat(IcebergParquetWriteInstructions.builder() .targetPageSize(1024 * 1024) .build() @@ -91,7 +92,7 @@ void testMinMaximumDictionaryKeys() { } @Test - public void testMinMaximumDictionarySize() { + void testMinMaximumDictionarySize() { try { IcebergParquetWriteInstructions.builder() .maximumDictionarySize(-1) From 7de59b0969d9c9318afd1f322a3dbd055780791a Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Thu, 3 Oct 2024 15:40:08 -0500 Subject: [PATCH 13/28] Review with Larry Part 2 --- .../java/io/deephaven/iceberg/util/IcebergToolsS3.java | 8 +++----- .../main/java/io/deephaven/iceberg/base/IcebergUtils.java | 6 +++--- .../io/deephaven/iceberg/layout/IcebergBaseLayout.java | 2 +- .../io/deephaven/iceberg/util/IcebergCatalogAdapter.java | 3 +-- 4 files changed, 8 insertions(+), 11 deletions(-) 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 47a602ee773..7e6bf6a842e 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 @@ -45,12 +45,8 @@ public static IcebergCatalogAdapter createS3Rest( @Nullable final String secretAccessKey, @Nullable final String endpointOverride) { - // Set up the properties map for the Iceberg catalog + // Set up the properties map for the Iceberg catalog, and configure it from Iceberg instructions. final Map properties = new HashMap<>(); - - final Catalog catalog = new RESTCatalog(); - - // 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); @@ -61,6 +57,8 @@ public static IcebergCatalogAdapter createS3Rest( if (!Strings.isNullOrEmpty(endpointOverride)) { properties.put(S3FileIOProperties.ENDPOINT, endpointOverride); } + + final Catalog catalog = new RESTCatalog(); return createAdapterCommon(name, catalogURI, warehouseLocation, catalog, properties); } 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 index 2558ea55ade..7cf7a028ec7 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/base/IcebergUtils.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/base/IcebergUtils.java @@ -41,13 +41,13 @@ public final class IcebergUtils { * * @param table The {@link Table} to retrieve data files for. * @param snapshot The {@link Snapshot} to retrieve data files from. - * @param fileIO The {@link FileIO} to use for reading manifest data files. + * * @return A stream of {@link DataFile} objects. */ public static Stream getAllDataFiles( @NotNull final Table table, - @NotNull final Snapshot snapshot, - @NotNull final FileIO fileIO) { + @NotNull final Snapshot snapshot) { + final FileIO fileIO = table.io(); try { // Retrieve the manifest files from the snapshot final List manifestFiles = snapshot.allManifests(fileIO); 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 d171bde0e49..76509d92ee5 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 @@ -144,7 +144,7 @@ private URI dataFileUri(@NotNull DataFile df) { @Override public synchronized void findKeys(@NotNull final Consumer locationKeyObserver) { - try (final Stream dataFiles = getAllDataFiles(table, snapshot, fileIO)) { + try (final Stream dataFiles = getAllDataFiles(table, snapshot)) { dataFiles.forEach(df -> { final URI fileUri = dataFileUri(df); final IcebergTableLocationKey locationKey = 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 1fa711a08b9..5771900f3db 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 @@ -1241,8 +1241,7 @@ private static void commit( if (overwrite) { // Delete all the existing data files in the table final DeleteFiles deletes = icebergTransaction.newDelete(); - try (final Stream dataFiles = - getAllDataFiles(icebergTable, icebergTable.currentSnapshot(), icebergTable.io())) { + try (final Stream dataFiles = getAllDataFiles(icebergTable, icebergTable.currentSnapshot())) { dataFiles.forEach(deletes::deleteFile); } deletes.commit(); From c83ddbd95f5e23ab76e74599b4920e4d6e938294 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Mon, 7 Oct 2024 16:11:21 -0500 Subject: [PATCH 14/28] Review with Devin Part 1 --- .../IcebergParquetWriteInstructionsTest.java | 63 ++++++- .../iceberg/util/IcebergToolsTest.java | 37 ++-- .../deephaven/iceberg/base/IcebergUtils.java | 163 ++++++++++++++---- .../iceberg/layout/IcebergBaseLayout.java | 35 ++-- .../iceberg/layout/IcebergFlatLayout.java | 4 +- .../IcebergKeyValuePartitionedLayout.java | 4 +- .../iceberg/util/IcebergBaseInstructions.java | 12 -- .../iceberg/util/IcebergCatalogAdapter.java | 148 ++++++---------- .../iceberg/util/IcebergInstructions.java | 32 ---- .../util/IcebergParquetWriteInstructions.java | 30 ++-- .../iceberg/util/IcebergReadInstructions.java | 42 +++++ .../util/IcebergWriteInstructions.java | 24 ++- .../deephaven/iceberg/CatalogAdapterTest.java | 88 +++++++++- .../parquet/base/ParquetFileWriter.java | 12 +- .../parquet/table/ParquetTableWriter.java | 13 +- .../deephaven/parquet/table/ParquetTools.java | 1 + 16 files changed, 463 insertions(+), 245 deletions(-) delete mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergReadInstructions.java diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java index 9fa7a8243c9..69a4761efc0 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java @@ -9,6 +9,7 @@ import java.util.Map; import java.util.Optional; +import static org.assertj.core.api.Assertions.failBecauseExceptionWasNotThrown; import static org.assertj.core.api.AssertionsForClassTypes.assertThat; class IcebergParquetWriteInstructionsTest { @@ -18,7 +19,8 @@ void defaults() { final IcebergParquetWriteInstructions instructions = IcebergParquetWriteInstructions.builder().build(); assertThat(instructions.tableDefinition().isEmpty()).isTrue(); assertThat(instructions.dataInstructions().isEmpty()).isTrue(); - assertThat(instructions.columnRenames().isEmpty()).isTrue(); + assertThat(instructions.dhToParquetColumnRenames().isEmpty()).isTrue(); + assertThat(instructions.dhToIcebergColumnRenames().isEmpty()).isTrue(); assertThat(instructions.createTableIfNotExist()).isFalse(); assertThat(instructions.verifySchema()).isEmpty(); assertThat(instructions.compressionCodecName()).isEqualTo("SNAPPY"); @@ -82,10 +84,12 @@ void testSetTargetPageSize() { @Test void testMinMaximumDictionaryKeys() { + try { IcebergParquetWriteInstructions.builder() .maximumDictionaryKeys(-1) .build(); + failBecauseExceptionWasNotThrown(IllegalArgumentException.class); } catch (IllegalArgumentException e) { assertThat(e).hasMessageContaining("maximumDictionaryKeys"); } @@ -97,6 +101,7 @@ void testMinMaximumDictionarySize() { IcebergParquetWriteInstructions.builder() .maximumDictionarySize(-1) .build(); + failBecauseExceptionWasNotThrown(IllegalArgumentException.class); } catch (IllegalArgumentException e) { assertThat(e).hasMessageContaining("maximumDictionarySize"); } @@ -108,14 +113,61 @@ void testMinTargetPageSize() { IcebergParquetWriteInstructions.builder() .targetPageSize(1024) .build(); + failBecauseExceptionWasNotThrown(IllegalArgumentException.class); } catch (IllegalArgumentException e) { assertThat(e).hasMessageContaining("targetPageSize"); } } + @Test + void testSetToIcebergColumnRename() { + final IcebergParquetWriteInstructions instructions = IcebergParquetWriteInstructions.builder() + .putDhToIcebergColumnRenames("dh1", "ice1") + .putDhToIcebergColumnRenames("dh2", "ice2") + .build(); + assertThat(instructions.dhToIcebergColumnRenames().size()).isEqualTo(2); + assertThat(instructions.dhToIcebergColumnRenames().get("dh1")).isEqualTo("ice1"); + assertThat(instructions.dhToIcebergColumnRenames().get("dh2")).isEqualTo("ice2"); + + final IcebergParquetWriteInstructions instructions2 = IcebergParquetWriteInstructions.builder() + .putAllDhToIcebergColumnRenames(Map.of( + "dh1", "ice1", + "dh2", "ice2", + "dh3", "ice3")) + .build(); + assertThat(instructions2.dhToIcebergColumnRenames().size()).isEqualTo(3); + assertThat(instructions2.dhToIcebergColumnRenames().get("dh1")).isEqualTo("ice1"); + assertThat(instructions2.dhToIcebergColumnRenames().get("dh2")).isEqualTo("ice2"); + assertThat(instructions2.dhToIcebergColumnRenames().get("dh3")).isEqualTo("ice3"); + } + + @Test + void testSetToParquetColumnRename() { + final IcebergParquetWriteInstructions instructions = IcebergParquetWriteInstructions.builder() + .putDhToParquetColumnRenames("dh1", "parquet1") + .putDhToParquetColumnRenames("dh2", "parquet2") + .build(); + assertThat(instructions.dhToParquetColumnRenames().size()).isEqualTo(2); + assertThat(instructions.dhToParquetColumnRenames().get("dh1")).isEqualTo("parquet1"); + assertThat(instructions.dhToParquetColumnRenames().get("dh2")).isEqualTo("parquet2"); + + final IcebergParquetWriteInstructions instructions2 = IcebergParquetWriteInstructions.builder() + .putAllDhToParquetColumnRenames(Map.of( + "dh1", "parquet1", + "dh2", "parquet2", + "dh3", "parquet3")) + .build(); + assertThat(instructions2.dhToParquetColumnRenames().size()).isEqualTo(3); + assertThat(instructions2.dhToParquetColumnRenames().get("dh1")).isEqualTo("parquet1"); + assertThat(instructions2.dhToParquetColumnRenames().get("dh2")).isEqualTo("parquet2"); + assertThat(instructions2.dhToParquetColumnRenames().get("dh3")).isEqualTo("parquet3"); + } + @Test void toParquetInstructionTest() { final IcebergParquetWriteInstructions icebergInstructions = IcebergParquetWriteInstructions.builder() + .putDhToIcebergColumnRenames("dh1", "ice1") + .putDhToParquetColumnRenames("dh2", "parquet1") .compressionCodecName("GZIP") .maximumDictionaryKeys(100) .maximumDictionarySize(200) @@ -124,6 +176,15 @@ void toParquetInstructionTest() { final Map fieldIdToName = Map.of(2, "field2", 3, "field3"); final ParquetInstructions parquetInstructions = icebergInstructions.toParquetInstructions( null, fieldIdToName); + + assertThat(parquetInstructions.getParquetColumnNameFromColumnNameOrDefault("dh1")).isEqualTo("dh1"); + assertThat(parquetInstructions.getParquetColumnNameFromColumnNameOrDefault("ice1")).isEqualTo("ice1"); + assertThat(parquetInstructions.getParquetColumnNameFromColumnNameOrDefault("dh2")).isEqualTo("parquet1"); + + assertThat(parquetInstructions.getColumnNameFromParquetColumnNameOrDefault("dh1")).isEqualTo("dh1"); + assertThat(parquetInstructions.getColumnNameFromParquetColumnNameOrDefault("ice1")).isEqualTo("ice1"); + assertThat(parquetInstructions.getColumnNameFromParquetColumnNameOrDefault("parquet1")).isEqualTo("dh2"); + assertThat(parquetInstructions.getCompressionCodecName()).isEqualTo("GZIP"); assertThat(parquetInstructions.getMaximumDictionaryKeys()).isEqualTo(100); assertThat(parquetInstructions.getMaximumDictionarySize()).isEqualTo(200); 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 9b8d74f65f2..63c44001e89 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 @@ -94,7 +94,7 @@ abstract class IcebergToolsTest { ColumnDefinition.ofString("ColumnType"), ColumnDefinition.ofBoolean("IsPartitioning")); - private IcebergInstructions instructions; + private IcebergReadInstructions instructions; public abstract S3AsyncClient s3AsyncClient(); @@ -126,13 +126,14 @@ void setUp() throws Exception { final S3Instructions s3Instructions = s3Instructions(S3Instructions.builder()).build(); - instructions = IcebergInstructions.builder() + instructions = IcebergReadInstructions.builder() .dataInstructions(s3Instructions) .build(); } @AfterEach - void tearDown() throws ExecutionException, InterruptedException { + void tearDown() throws Exception { + engineCleanup.tearDown(); for (String key : keys) { asyncClient.deleteObject(DeleteObjectRequest.builder().bucket(bucket).key(key).build()).get(); } @@ -338,7 +339,7 @@ void testOpenTableS3Only() throws ExecutionException, InterruptedException, Time void testOpenTableDefinition() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergReadInstructions localInstructions = IcebergReadInstructions.builder() .tableDefinition(SALES_PARTITIONED_DEFINITION) .dataInstructions(instructions.dataInstructions().get()) .build(); @@ -365,7 +366,7 @@ void testOpenTablePartitionTypeException() { ColumnDefinition.ofLong("Unit_Price"), ColumnDefinition.ofTime("Order_Date")); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergReadInstructions localInstructions = IcebergReadInstructions.builder() .tableDefinition(tableDef) .dataInstructions(instructions.dataInstructions().get()) .build(); @@ -401,7 +402,7 @@ void testOpenTableDefinitionRename() throws ExecutionException, InterruptedExcep ColumnDefinition.ofDouble("UnitPrice"), ColumnDefinition.ofTime("OrderDate")); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergReadInstructions localInstructions = IcebergReadInstructions.builder() .tableDefinition(renamed) .dataInstructions(instructions.dataInstructions().get()) .putColumnRenames("Region", "RegionName") @@ -437,7 +438,7 @@ void testSkippedPartitioningColumn() throws ExecutionException, InterruptedExcep ColumnDefinition.ofDouble("Unit_Price"), ColumnDefinition.ofTime("Order_Date")); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergReadInstructions localInstructions = IcebergReadInstructions.builder() .tableDefinition(tableDef) .dataInstructions(instructions.dataInstructions().get()) .build(); @@ -466,7 +467,7 @@ void testReorderedPartitioningColumn() throws ExecutionException, InterruptedExc ColumnDefinition.ofDouble("Unit_Price"), ColumnDefinition.ofTime("Order_Date")); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergReadInstructions localInstructions = IcebergReadInstructions.builder() .tableDefinition(tableDef) .dataInstructions(instructions.dataInstructions().get()) .build(); @@ -486,7 +487,7 @@ void testReorderedPartitioningColumn() throws ExecutionException, InterruptedExc void testZeroPartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergReadInstructions localInstructions = IcebergReadInstructions.builder() .tableDefinition(SALES_MULTI_DEFINITION) .dataInstructions(instructions.dataInstructions().get()) .build(); @@ -513,7 +514,7 @@ void testIncorrectPartitioningColumns() throws ExecutionException, InterruptedEx ColumnDefinition.ofDouble("Unit_Price"), ColumnDefinition.ofTime("Order_Date")); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergReadInstructions localInstructions = IcebergReadInstructions.builder() .tableDefinition(tableDef) .dataInstructions(instructions.dataInstructions().get()) .build(); @@ -548,7 +549,7 @@ void testMissingPartitioningColumns() { ColumnDefinition.ofLong("Unit_Price"), ColumnDefinition.ofTime("Order_Date")); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergReadInstructions localInstructions = IcebergReadInstructions.builder() .tableDefinition(tableDef) .dataInstructions(instructions.dataInstructions().get()) .build(); @@ -575,7 +576,7 @@ void testMissingPartitioningColumns() { void testOpenTableColumnRename() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergReadInstructions localInstructions = IcebergReadInstructions.builder() .dataInstructions(instructions.dataInstructions().get()) .putColumnRenames("Region", "RegionName") .putColumnRenames("Item_Type", "ItemType") @@ -595,7 +596,7 @@ void testOpenTableColumnRename() throws ExecutionException, InterruptedException void testOpenTableColumnLegalization() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesRenamed(); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergReadInstructions localInstructions = IcebergReadInstructions.builder() .dataInstructions(instructions.dataInstructions().get()) .build(); @@ -615,7 +616,7 @@ void testOpenTableColumnLegalizationRename() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesRenamed(); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergReadInstructions localInstructions = IcebergReadInstructions.builder() .dataInstructions(instructions.dataInstructions().get()) .putColumnRenames("Item&Type", "Item_Type") .putColumnRenames("Units/Sold", "Units_Sold") @@ -645,7 +646,7 @@ void testOpenTableColumnLegalizationPartitionException() { ColumnDefinition.ofInt("Year").withPartitioning(), ColumnDefinition.ofInt("Month").withPartitioning()); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergReadInstructions localInstructions = IcebergReadInstructions.builder() .tableDefinition(tableDef) .putColumnRenames("Year", "Current Year") .putColumnRenames("Month", "Current Month") @@ -675,7 +676,7 @@ void testOpenTableColumnRenamePartitioningColumns() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); - final IcebergInstructions localInstructions = IcebergInstructions.builder() + final IcebergReadInstructions localInstructions = IcebergReadInstructions.builder() .dataInstructions(instructions.dataInstructions().get()) .putColumnRenames("VendorID", "vendor_id") .putColumnRenames("month", "__month") @@ -858,7 +859,7 @@ void testTableDefinitionTable() { void testTableDefinitionWithInstructions() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); - IcebergInstructions localInstructions = IcebergInstructions.builder() + IcebergReadInstructions localInstructions = IcebergReadInstructions.builder() .dataInstructions(instructions.dataInstructions().get()) .putColumnRenames("Region", "Area") .putColumnRenames("Item_Type", "ItemType") @@ -885,7 +886,7 @@ void testTableDefinitionWithInstructions() { ColumnDefinition.ofString("Item_Type"), ColumnDefinition.ofTime("Order_Date")); - localInstructions = IcebergInstructions.builder() + localInstructions = IcebergReadInstructions.builder() .dataInstructions(instructions.dataInstructions().get()) .tableDefinition(userTableDef) .build(); 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 index 7cf7a028ec7..827c6cc3698 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/base/IcebergUtils.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/base/IcebergUtils.java @@ -6,16 +6,16 @@ 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.IcebergWriteInstructions; import org.apache.iceberg.DataFile; -import org.apache.iceberg.ManifestContent; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.ManifestFiles; -import org.apache.iceberg.ManifestReader; 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.io.CloseableIterable; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; @@ -28,8 +28,13 @@ import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; +import java.util.ArrayList; +import java.util.Collection; import java.util.List; +import java.util.Map; import java.util.Set; +import java.util.Spliterator; +import java.util.Spliterators; import java.util.stream.Collectors; import java.util.stream.Stream; import java.util.stream.StreamSupport; @@ -44,36 +49,18 @@ public final class IcebergUtils { * * @return A stream of {@link DataFile} objects. */ - public static Stream getAllDataFiles( - @NotNull final Table table, - @NotNull final Snapshot snapshot) { + public static Stream allDataFiles(@NotNull final Table table, @NotNull final Snapshot snapshot) { final FileIO fileIO = table.io(); try { - // Retrieve the manifest files from the snapshot - final List manifestFiles = snapshot.allManifests(fileIO); - return manifestFiles.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())); - } - }) - .flatMap(manifestFile -> { + return allManifests(table, snapshot).stream() + .map(manifestFile -> ManifestFiles.read(manifestFile, fileIO)) + .flatMap(manifestReader -> { try { - final ManifestReader reader = ManifestFiles.read(manifestFile, fileIO); - return StreamSupport.stream(reader.spliterator(), false).onClose(() -> { - try { - reader.close(); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - }); + return toStream(manifestReader); } catch (final RuntimeException e) { throw new TableDataException( String.format("%s:%d:%s - error reading manifest file", table, - snapshot.snapshotId(), manifestFile), + snapshot.snapshotId(), manifestReader), e); } }); @@ -83,6 +70,34 @@ public static Stream getAllDataFiles( } } + /** + * 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); + } + } + + private static Stream toStream(final 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. * @@ -163,6 +178,62 @@ public static Type convertToIcebergType(final Class columnType) { // TODO Add support for reading and writing lists } + public static class SpecAndSchema { + private final PartitionSpec partitionSpec; + private Schema schema; + + private SpecAndSchema(final PartitionSpec partitionSpec, final Schema schema) { + this.partitionSpec = partitionSpec; + this.schema = schema; + } + + public PartitionSpec partitionSpec() { + return partitionSpec; + } + + public Schema schema() { + return schema; + } + } + + /** + * Create {@link PartitionSpec} and {@link Schema} from a {@link TableDefinition} using the provided instructions. + */ + public static SpecAndSchema createSpecAndSchema( + @NotNull final TableDefinition tableDefinition, + @NotNull final IcebergWriteInstructions instructions) { + 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 + final Map dhToIcebergColumnRenames = instructions.dhToIcebergColumnRenames(); + for (final ColumnDefinition columnDefinition : tableDefinition.getColumns()) { + final String dhColumnName = columnDefinition.getName(); + final String icebergColName = dhToIcebergColumnRenames.getOrDefault(dhColumnName, dhColumnName); + final Type icebergType = convertToIcebergType(columnDefinition.getDataType()); + fields.add(Types.NestedField.optional(fieldID, icebergColName, icebergType)); + if (columnDefinition.isPartitioning()) { + partitioningColumnNames.add(icebergColName); + } + fieldID++; + } + final Schema schema = new Schema(fields); + + final PartitionSpec partitionSpec = createPartitionSpec(schema, partitioningColumnNames); + return new SpecAndSchema(partitionSpec, schema); + } + + private 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(); + } + /** * Check if an existing iceberg table with provided schema is compatible for overwriting with a new table with given * schema. @@ -209,27 +280,40 @@ public static void verifyOverwriteCompatibility( * @throws IllegalArgumentException if the schemas are not compatible. */ public static void verifyAppendCompatibility( - final Schema icebergSchema, - final TableDefinition tableDefinition) { + @NotNull final Schema icebergSchema, + @NotNull final TableDefinition tableDefinition, + @NotNull final IcebergWriteInstructions instructions) { // Check that all columns in the table definition are part of the Iceberg schema and have the same type + final Map dhToIcebergColumnRenames = instructions.dhToIcebergColumnRenames(); for (final ColumnDefinition dhColumn : tableDefinition.getColumns()) { - final Types.NestedField icebergColumn = icebergSchema.findField(dhColumn.getName()); - if (icebergColumn == null || !icebergColumn.type().equals(convertToIcebergType(dhColumn.getDataType()))) { + final String dhColumnName = dhColumn.getName(); + final String icebergColName = dhToIcebergColumnRenames.getOrDefault(dhColumnName, dhColumnName); + final Types.NestedField icebergColumn = icebergSchema.findField(icebergColName); + if (icebergColumn == null) { + throw new IllegalArgumentException("Schema mismatch, column " + dhColumn.getName() + " from Deephaven " + + "table definition: " + tableDefinition + " is not found in Iceberg table schema: " + + icebergSchema); + } + if (!icebergColumn.type().equals(convertToIcebergType(dhColumn.getDataType()))) { throw new IllegalArgumentException("Schema mismatch, column " + dhColumn.getName() + " from Deephaven " - + "table definition: " + tableDefinition + " is not found or has a different type in Iceberg " - + "table schema: " + icebergSchema); + + "table definition: " + tableDefinition + " has type " + dhColumn.getDataType() + + " which does not match the type " + icebergColumn.type() + " in Iceberg table schema: " + + icebergSchema); } } // Check that all required columns in the Iceberg schema are part of the table definition + final Map icebergToDhColumnRenames = instructions.icebergToDhColumnRenames(); for (final Types.NestedField icebergColumn : icebergSchema.columns()) { if (icebergColumn.isOptional()) { continue; } - if (tableDefinition.getColumn(icebergColumn.name()) == null) { - throw new IllegalArgumentException("Partition spec mismatch, required column " + icebergColumn.name() + - " from Iceberg table schema: " + icebergSchema + " not found in Deephaven table definition: " - + tableDefinition); + final String icebergColumnName = icebergColumn.name(); + final String dhColName = icebergToDhColumnRenames.getOrDefault(icebergColumnName, icebergColumnName); + if (tableDefinition.getColumn(dhColName) == null) { + throw new IllegalArgumentException("Schema mismatch, required column " + icebergColumnName + + " from Iceberg table schema: " + icebergSchema + " is not found in Deephaven table " + + "definition: " + tableDefinition); } } } @@ -244,10 +328,13 @@ public static void verifyAppendCompatibility( * @throws IllegalArgumentException if the partition spec are not compatible. */ public static void verifyAppendCompatibility( - final PartitionSpec partitionSpec, - final TableDefinition tableDefinition) { + @NotNull final PartitionSpec partitionSpec, + @NotNull final TableDefinition tableDefinition, + @NotNull final IcebergWriteInstructions instructions) { + final Map icebergToDhColumnRenames = instructions.icebergToDhColumnRenames(); final Set icebergPartitionColumns = partitionSpec.fields().stream() .map(PartitionField::name) + .map(icebergColumnName -> icebergToDhColumnRenames.getOrDefault(icebergColumnName, icebergColumnName)) .collect(Collectors.toSet()); final Set dhPartitioningColumns = tableDefinition.getColumns().stream() .filter(ColumnDefinition::isPartitioning) 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 76509d92ee5..6e94ab3db4c 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 @@ -9,8 +9,8 @@ import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; import io.deephaven.iceberg.location.IcebergTableLocationKey; import io.deephaven.iceberg.location.IcebergTableParquetLocationKey; -import io.deephaven.iceberg.util.IcebergBaseInstructions; import io.deephaven.iceberg.relative.RelativeFileIO; +import io.deephaven.iceberg.util.IcebergReadInstructions; import io.deephaven.parquet.table.ParquetInstructions; import io.deephaven.iceberg.internal.DataInstructionsProviderLoader; import org.apache.iceberg.*; @@ -21,10 +21,11 @@ import java.net.URI; import java.util.HashMap; import java.util.Map; +import java.util.Objects; import java.util.function.Consumer; import java.util.stream.Stream; -import static io.deephaven.iceberg.base.IcebergUtils.getAllDataFiles; +import static io.deephaven.iceberg.base.IcebergUtils.allDataFiles; public abstract class IcebergBaseLayout implements TableLocationKeyFinder { /** @@ -50,7 +51,7 @@ public abstract class IcebergBaseLayout implements TableLocationKeyFinder locationKeyObserver) { - try (final Stream dataFiles = getAllDataFiles(table, snapshot)) { - dataFiles.forEach(df -> { - final URI fileUri = dataFileUri(df); - final IcebergTableLocationKey locationKey = - cache.computeIfAbsent(fileUri, uri -> keyFromDataFile(df, fileUri)); - if (locationKey != null) { - locationKeyObserver.accept(locationKey); - } - }); - } catch (final Exception e) { + try (final Stream dataFiles = allDataFiles(table, snapshot)) { + dataFiles + .map(df -> { + final URI fileUri = dataFileUri(df); + return cache.computeIfAbsent(fileUri, uri -> keyFromDataFile(df, fileUri)); + }) + .filter(Objects::nonNull) + .forEach(locationKeyObserver); + } catch (final RuntimeException e) { throw new TableDataException( String.format("%s:%d - error finding Iceberg locations", table, snapshot.snapshotId()), e); } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java index 3dae4e21e36..064f30d8065 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java @@ -6,7 +6,7 @@ import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; import io.deephaven.iceberg.location.IcebergTableLocationKey; -import io.deephaven.iceberg.util.IcebergBaseInstructions; +import io.deephaven.iceberg.util.IcebergReadInstructions; import io.deephaven.iceberg.internal.DataInstructionsProviderLoader; import org.apache.iceberg.*; import org.apache.iceberg.io.FileIO; @@ -31,7 +31,7 @@ public IcebergFlatLayout( @NotNull final Table table, @NotNull final Snapshot tableSnapshot, @NotNull final FileIO fileIO, - @NotNull final IcebergBaseInstructions instructions, + @NotNull final IcebergReadInstructions instructions, @NotNull final DataInstructionsProviderLoader dataInstructionsProvider) { super(tableDef, table, tableSnapshot, fileIO, instructions, dataInstructionsProvider); } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java index 981f8fd8b69..0614dbf7e5d 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergKeyValuePartitionedLayout.java @@ -8,7 +8,7 @@ import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; import io.deephaven.iceberg.location.IcebergTableLocationKey; -import io.deephaven.iceberg.util.IcebergBaseInstructions; +import io.deephaven.iceberg.util.IcebergReadInstructions; import io.deephaven.iceberg.internal.DataInstructionsProviderLoader; import io.deephaven.util.type.TypeUtils; import org.apache.commons.lang3.mutable.MutableInt; @@ -53,7 +53,7 @@ public IcebergKeyValuePartitionedLayout( @NotNull final org.apache.iceberg.Snapshot tableSnapshot, @NotNull final FileIO fileIO, @NotNull final PartitionSpec partitionSpec, - @NotNull final IcebergBaseInstructions instructions, + @NotNull final IcebergReadInstructions instructions, @NotNull final DataInstructionsProviderLoader dataInstructionsProvider) { super(tableDef, table, tableSnapshot, fileIO, instructions, dataInstructionsProvider); diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergBaseInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergBaseInstructions.java index 643fe4d3bd0..172cee8da53 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergBaseInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergBaseInstructions.java @@ -5,7 +5,6 @@ import io.deephaven.engine.table.TableDefinition; -import java.util.Map; import java.util.Optional; public interface IcebergBaseInstructions { @@ -21,20 +20,9 @@ public interface IcebergBaseInstructions { */ Optional dataInstructions(); - /** - * A {@link Map map} of rename instructions from Iceberg to Deephaven column names to use when reading/writing the - * Iceberg data files. - */ - Map columnRenames(); - interface Builder { INSTRUCTIONS_BUILDER tableDefinition(TableDefinition tableDefinition); INSTRUCTIONS_BUILDER dataInstructions(Object s3Instructions); - - INSTRUCTIONS_BUILDER putColumnRenames(String key, String value); - - @SuppressWarnings("unused") - INSTRUCTIONS_BUILDER putAllColumnRenames(Map entries); } } 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 5771900f3db..f2f212c5227 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 @@ -57,9 +57,10 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static io.deephaven.iceberg.base.IcebergUtils.SpecAndSchema; import static io.deephaven.iceberg.base.IcebergUtils.convertToDHType; -import static io.deephaven.iceberg.base.IcebergUtils.convertToIcebergType; -import static io.deephaven.iceberg.base.IcebergUtils.getAllDataFiles; +import static io.deephaven.iceberg.base.IcebergUtils.allDataFiles; +import static io.deephaven.iceberg.base.IcebergUtils.createSpecAndSchema; import static io.deephaven.iceberg.base.IcebergUtils.verifyAppendCompatibility; import static io.deephaven.iceberg.base.IcebergUtils.verifyOverwriteCompatibility; @@ -405,7 +406,7 @@ private Snapshot getSnapshot(@NotNull final TableIdentifier tableIdentifier, fin private Map getRenameColumnMap( @NotNull final org.apache.iceberg.Table table, @NotNull final Schema schema, - @NotNull final IcebergInstructions instructions) { + @NotNull final IcebergReadInstructions instructions) { final Set takenNames = new HashSet<>(); @@ -450,7 +451,7 @@ private Map getRenameColumnMap( */ public TableDefinition getTableDefinition( @NotNull final String tableIdentifier, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergReadInstructions instructions) { final TableIdentifier tableId = TableIdentifier.parse(tableIdentifier); // Load the table from the catalog. return getTableDefinition(tableId, instructions); @@ -466,7 +467,7 @@ public TableDefinition getTableDefinition( */ public TableDefinition getTableDefinition( @NotNull final TableIdentifier tableIdentifier, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergReadInstructions instructions) { // Load the table from the catalog. return getTableDefinitionInternal(tableIdentifier, null, instructions); } @@ -483,7 +484,7 @@ public TableDefinition getTableDefinition( public TableDefinition getTableDefinition( @NotNull final String tableIdentifier, final long snapshotId, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergReadInstructions instructions) { final TableIdentifier tableId = TableIdentifier.parse(tableIdentifier); // Find the snapshot with the given snapshot id @@ -508,7 +509,7 @@ public TableDefinition getTableDefinition( public TableDefinition getTableDefinition( @NotNull final TableIdentifier tableIdentifier, @Nullable final Snapshot tableSnapshot, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergReadInstructions instructions) { // Load the table from the catalog. return getTableDefinitionInternal(tableIdentifier, tableSnapshot, instructions); } @@ -523,7 +524,7 @@ public TableDefinition getTableDefinition( */ public Table getTableDefinitionTable( @NotNull final String tableIdentifier, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergReadInstructions instructions) { final TableIdentifier tableId = TableIdentifier.parse(tableIdentifier); return getTableDefinitionTable(tableId, instructions); } @@ -538,7 +539,7 @@ public Table getTableDefinitionTable( */ public Table getTableDefinitionTable( @NotNull final TableIdentifier tableIdentifier, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergReadInstructions instructions) { final TableDefinition definition = getTableDefinition(tableIdentifier, instructions); return TableTools.metaTable(definition); } @@ -555,7 +556,7 @@ public Table getTableDefinitionTable( public Table getTableDefinitionTable( @NotNull final String tableIdentifier, final long snapshotId, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergReadInstructions instructions) { final TableIdentifier tableId = TableIdentifier.parse(tableIdentifier); // Find the snapshot with the given snapshot id @@ -579,7 +580,7 @@ public Table getTableDefinitionTable( public Table getTableDefinitionTable( @NotNull final TableIdentifier tableIdentifier, @Nullable final Snapshot tableSnapshot, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergReadInstructions instructions) { final TableDefinition definition = getTableDefinition(tableIdentifier, tableSnapshot, instructions); return TableTools.metaTable(definition); } @@ -590,7 +591,7 @@ public Table getTableDefinitionTable( private TableDefinition getTableDefinitionInternal( @NotNull final TableIdentifier tableIdentifier, @Nullable final Snapshot tableSnapshot, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergReadInstructions instructions) { final org.apache.iceberg.Table table = catalog.loadTable(tableIdentifier); if (table == null) { throw new IllegalArgumentException("Table not found: " + tableIdentifier); @@ -599,7 +600,8 @@ private TableDefinition getTableDefinitionInternal( final Snapshot snapshot = tableSnapshot != null ? tableSnapshot : table.currentSnapshot(); final Schema schema = snapshot != null ? table.schemas().get(snapshot.schemaId()) : table.schema(); - final IcebergInstructions userInstructions = instructions == null ? IcebergInstructions.DEFAULT : instructions; + final IcebergReadInstructions userInstructions = + instructions == null ? IcebergReadInstructions.DEFAULT : instructions; return fromSchema(schema, table.spec(), @@ -616,7 +618,7 @@ private TableDefinition getTableDefinitionInternal( */ public Table readTable( @NotNull final TableIdentifier tableIdentifier, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergReadInstructions instructions) { return readTableInternal(tableIdentifier, null, instructions); } @@ -629,7 +631,7 @@ public Table readTable( */ public Table readTable( @NotNull final String tableIdentifier, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergReadInstructions instructions) { return readTable(TableIdentifier.parse(tableIdentifier), instructions); } @@ -691,7 +693,7 @@ public Table readTable(@NotNull final String tableIdentifier, final long tableSn public Table readTable( @NotNull final TableIdentifier tableIdentifier, final long tableSnapshotId, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergReadInstructions instructions) { // Find the snapshot with the given snapshot id final Snapshot tableSnapshot = getSnapshot(tableIdentifier, tableSnapshotId); if (tableSnapshot == null) { @@ -711,7 +713,7 @@ public Table readTable( public Table readTable( @NotNull final String tableIdentifier, final long tableSnapshotId, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergReadInstructions instructions) { return readTable(TableIdentifier.parse(tableIdentifier), tableSnapshotId, instructions); } @@ -726,14 +728,14 @@ public Table readTable( public Table readTable( @NotNull final TableIdentifier tableIdentifier, @NotNull final Snapshot tableSnapshot, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergReadInstructions instructions) { return readTableInternal(tableIdentifier, tableSnapshot, instructions); } private Table readTableInternal( @NotNull final TableIdentifier tableIdentifier, @Nullable final Snapshot tableSnapshot, - @Nullable final IcebergInstructions instructions) { + @Nullable final IcebergReadInstructions instructions) { // Load the table from the catalog. final org.apache.iceberg.Table table = catalog.loadTable(tableIdentifier); if (table == null) { @@ -748,7 +750,8 @@ private Table readTableInternal( final org.apache.iceberg.PartitionSpec partitionSpec = table.spec(); // Get default instructions if none are provided - final IcebergInstructions userInstructions = instructions == null ? IcebergInstructions.DEFAULT : instructions; + final IcebergReadInstructions userInstructions = + instructions == null ? IcebergReadInstructions.DEFAULT : instructions; // Get the user supplied table definition. final TableDefinition userTableDef = userInstructions.tableDefinition().orElse(null); @@ -936,11 +939,11 @@ public void overwrite( * * @return A list of {@link DataFile} objects representing the written data files. */ - public List write( + public List writeDataFiles( @NotNull final String tableIdentifier, @NotNull final Table dhTable, @Nullable final IcebergWriteInstructions instructions) { - return write(tableIdentifier, new Table[] {dhTable}, instructions); + return writeDataFiles(tableIdentifier, new Table[] {dhTable}, instructions); } /** @@ -953,11 +956,11 @@ public List write( * * @return A list of {@link DataFile} objects representing the written data files. */ - public List write( + public List writeDataFiles( @NotNull final TableIdentifier tableIdentifier, @NotNull final Table dhTable, @Nullable final IcebergWriteInstructions instructions) { - return write(tableIdentifier, new Table[] {dhTable}, instructions); + return writeDataFiles(tableIdentifier, new Table[] {dhTable}, instructions); } /** @@ -971,11 +974,11 @@ public List write( * * @return A list of {@link DataFile} objects representing the written data files. */ - public List write( + public List writeDataFiles( @NotNull final String tableIdentifier, @NotNull final Table[] dhTables, @Nullable final IcebergWriteInstructions instructions) { - return write(TableIdentifier.parse(tableIdentifier), dhTables, instructions); + return writeDataFiles(TableIdentifier.parse(tableIdentifier), dhTables, instructions); } /** @@ -989,7 +992,7 @@ public List write( * * @return A list of {@link DataFile} objects representing the written data files. */ - public List write( + public List writeDataFiles( @NotNull final TableIdentifier tableIdentifier, @NotNull final Table[] dhTables, @Nullable final IcebergWriteInstructions instructions) { @@ -1014,14 +1017,14 @@ private List writeImpl( @Nullable final IcebergWriteInstructions instructions, final boolean overwrite, final boolean addSnapshot) { - final IcebergParquetWriteInstructions writeInstructions = verifyWriteInstructions(instructions); + IcebergParquetWriteInstructions writeInstructions = verifyWriteInstructions(instructions); final boolean verifySchema = writeInstructions.verifySchema().orElse(!overwrite); - // Verify that all tables have the same definition final TableDefinition useDefinition; if (writeInstructions.tableDefinition().isPresent()) { useDefinition = writeInstructions.tableDefinition().get(); } else { + // Verify that all tables have the same definition final TableDefinition firstDefinition = dhTables[0].getDefinition(); for (int idx = 1; idx < dhTables.length; idx++) { if (!firstDefinition.equals(dhTables[idx].getDefinition())) { @@ -1031,6 +1034,7 @@ private List writeImpl( } } useDefinition = firstDefinition; + writeInstructions = writeInstructions.withTableDefinition(useDefinition); } // Try loading the table from the catalog, or create if required @@ -1040,17 +1044,17 @@ private List writeImpl( final boolean newTableCreated; if (catalog.tableExists(tableIdentifier)) { icebergTable = catalog.loadTable(tableIdentifier); - newSpecAndSchema = fromTableDefinition(useDefinition, writeInstructions); + newSpecAndSchema = createSpecAndSchema(useDefinition, writeInstructions); newNamespaceCreated = false; newTableCreated = false; if (verifySchema) { try { if (overwrite) { - verifyOverwriteCompatibility(icebergTable.schema(), newSpecAndSchema.schema); - verifyOverwriteCompatibility(icebergTable.spec(), newSpecAndSchema.partitionSpec); + verifyOverwriteCompatibility(icebergTable.schema(), newSpecAndSchema.schema()); + verifyOverwriteCompatibility(icebergTable.spec(), newSpecAndSchema.partitionSpec()); } else { - verifyAppendCompatibility(icebergTable.schema(), useDefinition); - verifyAppendCompatibility(icebergTable.spec(), useDefinition); + verifyAppendCompatibility(icebergTable.schema(), useDefinition, writeInstructions); + verifyAppendCompatibility(icebergTable.spec(), useDefinition, writeInstructions); } } catch (final IllegalArgumentException e) { throw new IllegalArgumentException("Schema verification failed. Please provide a compatible " + @@ -1060,7 +1064,7 @@ private List writeImpl( } } else if (writeInstructions.createTableIfNotExist()) { newNamespaceCreated = createNamespaceIfNotExists(tableIdentifier.namespace()); - newSpecAndSchema = fromTableDefinition(useDefinition, writeInstructions); + newSpecAndSchema = createSpecAndSchema(useDefinition, writeInstructions); icebergTable = createNewIcebergTable(tableIdentifier, newSpecAndSchema, writeInstructions); newTableCreated = true; } else { @@ -1125,53 +1129,6 @@ private void dropNamespaceIfExists(@NotNull final Namespace namespace) { } } - private static class SpecAndSchema { - private final PartitionSpec partitionSpec; - private Schema schema; - - private SpecAndSchema(final PartitionSpec partitionSpec, final Schema schema) { - this.partitionSpec = partitionSpec; - this.schema = schema; - } - } - - /** - * Create {@link PartitionSpec} and {@link Schema} from a {@link TableDefinition}. - */ - private static SpecAndSchema fromTableDefinition( - @NotNull final TableDefinition tableDefinition, - @NotNull final IcebergBaseInstructions instructions) { - 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 String icebergColName = instructions.columnRenames().getOrDefault(dhColumnName, dhColumnName); - final Type icebergType = convertToIcebergType(columnDefinition.getDataType()); - fields.add(Types.NestedField.optional(fieldID, icebergColName, icebergType)); - if (columnDefinition.isPartitioning()) { - partitioningColumnNames.add(icebergColName); - } - fieldID++; - } - final Schema schema = new Schema(fields); - - final PartitionSpec partitionSpec = createPartitionSpec(schema, partitioningColumnNames); - return new SpecAndSchema(partitionSpec, schema); - } - - private 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(); - } - private org.apache.iceberg.Table createNewIcebergTable( @NotNull final TableIdentifier tableIdentifier, @NotNull final SpecAndSchema specAndSchema, @@ -1179,7 +1136,7 @@ private org.apache.iceberg.Table createNewIcebergTable( if (catalog.tableExists(tableIdentifier)) { throw new IllegalArgumentException("Table already exists: " + tableIdentifier); } - return catalog.createTable(tableIdentifier, specAndSchema.schema, specAndSchema.partitionSpec, null, + return catalog.createTable(tableIdentifier, specAndSchema.schema(), specAndSchema.partitionSpec(), null, Map.of(TableProperties.DEFAULT_FILE_FORMAT, DEFAULT_GENERATED_FILE_FORMAT, TableProperties.PARQUET_COMPRESSION, StringUtils.toLowerCase(writeInstructions.compressionCodecName()))); @@ -1211,16 +1168,13 @@ private static List writeParquet( onWriteCompleted, icebergTable.schema().idToName()); // Write the data to parquet files + int count = 0; for (final Table dhTable : dhTables) { - final long epochMicrosNow = Instant.now().getEpochSecond() * 1_000_000 + Instant.now().getNano() / 1_000; - final String filename = new StringBuilder() - .append(epochMicrosNow) // To keep the data ordered by time - .append("-") - .append(UUID.randomUUID()) - .append(".parquet") - .toString(); - final String newDataLocation = - icebergTable.locationProvider().newDataLocation(filename); + final String filename = String.format( + "00000-%d-%s.parquet", + count++, + UUID.randomUUID()); + final String newDataLocation = icebergTable.locationProvider().newDataLocation(filename); ParquetTools.writeTable(dhTable, newDataLocation, parquetInstructions); } return parquetFilesWritten; @@ -1241,7 +1195,7 @@ private static void commit( if (overwrite) { // Delete all the existing data files in the table final DeleteFiles deletes = icebergTransaction.newDelete(); - try (final Stream dataFiles = getAllDataFiles(icebergTable, icebergTable.currentSnapshot())) { + try (final Stream dataFiles = allDataFiles(icebergTable, icebergTable.currentSnapshot())) { dataFiles.forEach(deletes::deleteFile); } deletes.commit(); @@ -1249,19 +1203,19 @@ private static void commit( // Update the spec and schema of the existing table. // If we have already verified the schema, we don't need to update it. if (!schemaVerified) { - if (!icebergTable.schema().sameSchema(newSpecAndSchema.schema)) { + if (!icebergTable.schema().sameSchema(newSpecAndSchema.schema())) { final UpdateSchema updateSchema = icebergTransaction.updateSchema().allowIncompatibleChanges(); icebergTable.schema().columns().stream() .map(Types.NestedField::name) .forEach(updateSchema::deleteColumn); - newSpecAndSchema.schema.columns() + newSpecAndSchema.schema().columns() .forEach(column -> updateSchema.addColumn(column.name(), column.type())); updateSchema.commit(); } - if (!icebergTable.spec().compatibleWith(newSpecAndSchema.partitionSpec)) { + if (!icebergTable.spec().compatibleWith(newSpecAndSchema.partitionSpec())) { final UpdatePartitionSpec updateSpec = icebergTransaction.updateSpec(); icebergTable.spec().fields().forEach(field -> updateSpec.removeField(field.name())); - newSpecAndSchema.partitionSpec.fields().forEach(field -> updateSpec.addField(field.name())); + newSpecAndSchema.partitionSpec().fields().forEach(field -> updateSpec.addField(field.name())); updateSpec.commit(); } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java deleted file mode 100644 index db46b09302d..00000000000 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java +++ /dev/null @@ -1,32 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.iceberg.util; - -import io.deephaven.annotations.BuildableStyle; -import org.immutables.value.Value.Immutable; - -/** - * This class provides instructions intended for reading Iceberg catalogs and tables. The default values documented in - * this class may change in the future. As such, callers may wish to explicitly set the values. - */ -@Immutable -@BuildableStyle -// TODO I propose renaming, but this will be breaking change: -// IcebergInstructions -> IcebergReadInstructions -public abstract class IcebergInstructions implements IcebergBaseInstructions { - /** - * The default {@link IcebergInstructions} to use when reading Iceberg data files. Providing this will use system - * defaults for cloud provider-specific parameters. - */ - @SuppressWarnings("unused") - public static final IcebergInstructions DEFAULT = builder().build(); - - public static Builder builder() { - return ImmutableIcebergInstructions.builder(); - } - - public interface Builder extends IcebergBaseInstructions.Builder { - IcebergInstructions build(); - } -} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java index 4f4d3380d3a..1e9e0ff72f3 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java @@ -3,14 +3,14 @@ // package io.deephaven.iceberg.util; -import io.deephaven.annotations.BuildableStyle; +import io.deephaven.annotations.CopyableStyle; +import io.deephaven.engine.table.TableDefinition; import io.deephaven.parquet.table.ParquetInstructions; import org.immutables.value.Value.Default; import org.immutables.value.Value.Immutable; import org.immutables.value.Value.Check; import org.jetbrains.annotations.NotNull; -import java.util.List; import java.util.Map; import static io.deephaven.parquet.table.ParquetInstructions.MIN_TARGET_PAGE_SIZE; @@ -20,7 +20,7 @@ * documented in this class may change in the future. As such, callers may wish to explicitly set the values. */ @Immutable -@BuildableStyle +@CopyableStyle public abstract class IcebergParquetWriteInstructions extends IcebergWriteInstructions { /** * The default {@link IcebergParquetWriteInstructions} to use when reading/writing Iceberg tables as Parquet data @@ -72,6 +72,14 @@ public int targetPageSize() { return ParquetInstructions.DEFAULT_TARGET_PAGE_SIZE; } + /** + * A one-to-one {@link Map map} from Deephaven to Parquet column names to use when writing deephaven tables to + * Iceberg tables. + */ + public abstract Map dhToParquetColumnRenames(); + + abstract IcebergParquetWriteInstructions withTableDefinition(@NotNull final TableDefinition tableDefinition); + /** * Convert this {@link IcebergParquetWriteInstructions} to a {@link ParquetInstructions}. * @@ -86,10 +94,10 @@ ParquetInstructions toParquetInstructions( tableDefinition().ifPresent(builder::setTableDefinition); dataInstructions().ifPresent(builder::setSpecialInstructions); - // Add any column rename mappings. - if (!columnRenames().isEmpty()) { - for (final Map.Entry entry : columnRenames().entrySet()) { - builder.addColumnNameMapping(entry.getKey(), entry.getValue()); + // Add any parquet column rename mappings. + if (!dhToParquetColumnRenames().isEmpty()) { + for (final Map.Entry entry : dhToParquetColumnRenames().entrySet()) { + builder.addColumnNameMapping(entry.getValue(), entry.getKey()); } } @@ -105,18 +113,18 @@ ParquetInstructions toParquetInstructions( } public interface Builder extends IcebergWriteInstructions.Builder { - @SuppressWarnings("unused") Builder compressionCodecName(String compressionCodecName); - @SuppressWarnings("unused") Builder maximumDictionaryKeys(int maximumDictionaryKeys); - @SuppressWarnings("unused") Builder maximumDictionarySize(int maximumDictionarySize); - @SuppressWarnings("unused") Builder targetPageSize(int targetPageSize); + Builder putDhToParquetColumnRenames(String key, String value); + + Builder putAllDhToParquetColumnRenames(Map entries); + IcebergParquetWriteInstructions build(); } 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 new file mode 100644 index 00000000000..0bd1405a69d --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergReadInstructions.java @@ -0,0 +1,42 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.util; + +import io.deephaven.annotations.BuildableStyle; +import org.immutables.value.Value.Immutable; + +import java.util.Map; + +/** + * This class provides instructions intended for reading Iceberg catalogs and tables. The default values documented in + * this class may change in the future. As such, callers may wish to explicitly set the values. + */ +@Immutable +@BuildableStyle +public abstract class IcebergReadInstructions implements IcebergBaseInstructions { + /** + * The default {@link IcebergReadInstructions} to use when reading Iceberg data files. Providing this will use + * system defaults for cloud provider-specific parameters. + */ + public static final IcebergReadInstructions DEFAULT = builder().build(); + + public static Builder builder() { + return ImmutableIcebergReadInstructions.builder(); + } + + /** + * A {@link Map map} of rename instructions from Iceberg to Deephaven column names to use when reading the Iceberg + * data files. + */ + public abstract Map columnRenames(); + + public interface Builder extends IcebergBaseInstructions.Builder { + Builder putColumnRenames(String key, String value); + + @SuppressWarnings("unused") + Builder putAllColumnRenames(Map entries); + + IcebergReadInstructions build(); + } +} 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 index 8124e948d43..4b1467b4af6 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteInstructions.java @@ -3,9 +3,12 @@ // package io.deephaven.iceberg.util; +import org.immutables.value.Value; import org.immutables.value.Value.Default; +import java.util.Map; import java.util.Optional; +import java.util.stream.Collectors; /** * This class provides instructions intended for writing Iceberg tables. The default values documented in this class may @@ -42,11 +45,28 @@ public boolean createTableIfNotExist() { public abstract Optional verifySchema(); // @formatter:on + /** + * A one-to-one {@link Map map} from Deephaven to Iceberg column names to use when writing deephaven tables to + * Iceberg tables. + */ + public abstract Map dhToIcebergColumnRenames(); + + /** + * The inverse map of {@link #dhToIcebergColumnRenames()}. + */ + @Value.Lazy + public Map icebergToDhColumnRenames() { + return dhToIcebergColumnRenames().entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey)); + } + public interface Builder extends IcebergBaseInstructions.Builder { - @SuppressWarnings("unused") INSTRUCTIONS_BUILDER createTableIfNotExist(boolean createTableIfNotExist); - @SuppressWarnings("unused") INSTRUCTIONS_BUILDER verifySchema(boolean verifySchema); + + INSTRUCTIONS_BUILDER putDhToIcebergColumnRenames(String key, String value); + + INSTRUCTIONS_BUILDER putAllDhToIcebergColumnRenames(Map entries); } } diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java index 133633ecec5..e1c8fd73aff 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java @@ -9,16 +9,22 @@ import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.impl.select.FormulaEvaluationException; import io.deephaven.engine.util.TableTools; +import io.deephaven.iceberg.base.IcebergUtils; import io.deephaven.iceberg.junit5.CatalogAdapterBase; import io.deephaven.iceberg.util.IcebergParquetWriteInstructions; import io.deephaven.iceberg.util.IcebergWriteInstructions; +import io.deephaven.parquet.table.ParquetTools; import junit.framework.TestCase; +import org.apache.iceberg.DataFile; import org.apache.iceberg.Schema; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.Test; +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; @@ -77,7 +83,7 @@ void appendTableBasicTest() { "doubleCol = (double) 3.5 * i + 20"); catalogAdapter.append("MyNamespace.MyTable", moreData, writeInstructions); fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); - final Table expected = TableTools.merge(source, moreData); + final Table expected = TableTools.merge(moreData, source); assertTableEquals(expected, fromIceberg); // Append an empty table @@ -95,7 +101,7 @@ void appendTableBasicTest() { catalogAdapter.append("MyNamespace.MyTable", new Table[] {someMoreData, moreData, emptyTable}, writeInstructions); fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); - final Table expected2 = TableTools.merge(expected, someMoreData, moreData); + final Table expected2 = TableTools.merge(someMoreData, moreData, expected); assertTableEquals(expected2, fromIceberg); } @@ -176,7 +182,7 @@ void overwriteWithDifferentDefinition() { .update("intCol = (int) 3 * i + 20"); catalogAdapter.append("MyNamespace.MyTable", moreData, writeInstructionsWithoutSchemaMatching); fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); - final Table expected = TableTools.merge(differentSource, moreData); + final Table expected = TableTools.merge(moreData, differentSource); assertTableEquals(expected, fromIceberg); } @@ -207,7 +213,7 @@ void appendWithDifferentDefinition() { .update("intCol = (int) 5 * i + 10"); catalogAdapter.append("MyNamespace.MyTable", compatibleSource, writeInstructions); fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); - final Table expected = TableTools.merge(source, compatibleSource.update("doubleCol = NULL_DOUBLE")); + final Table expected = TableTools.merge(compatibleSource.update("doubleCol = NULL_DOUBLE"), source); assertTableEquals(expected, fromIceberg); // Append more data @@ -216,7 +222,7 @@ void appendWithDifferentDefinition() { "doubleCol = (double) 3.5 * i + 20"); catalogAdapter.append("MyNamespace.MyTable", moreData, writeInstructions); fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); - final Table expected2 = TableTools.merge(expected, moreData); + final Table expected2 = TableTools.merge(moreData, expected); assertTableEquals(expected2, fromIceberg); } @@ -262,9 +268,9 @@ void appendMultipleTablesWithDefinitionTest() { writeInstructionsWithDefinition); fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); final Table expected = TableTools.merge( - source, appendTable1.dropColumns("shortCol"), - appendTable2.dropColumns("charCol")); + appendTable2.dropColumns("charCol"), + source); assertTableEquals(expected, fromIceberg); } @@ -347,4 +353,72 @@ void testFailureInWrite() { fromIceberg = catalogAdapter.readTable(myTableId, null); assertTableEquals(goodSource, fromIceberg); } + + @Test + void testColumnRenameWhileWriting() { + final Table source = TableTools.emptyTable(10) + .update("intCol = (int) 2 * i + 10", + "doubleCol = (double) 2.5 * i + 10"); + IcebergWriteInstructions writeInstructions = IcebergParquetWriteInstructions.builder() + .createTableIfNotExist(true) + .verifySchema(true) + .putDhToParquetColumnRenames("intCol", "numbers") + .putDhToParquetColumnRenames("doubleCol", "decimals") + .build(); + + catalogAdapter.append("MyNamespace.MyTable", source, writeInstructions); + // TODO: This is failing because we don't map columns based on the column ID when reading. Uncomment when this + // is fixed. + // final Table fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + // assertTableEquals(source, fromIceberg); + + // Verify that the columns are renamed in the parquet file + final TableIdentifier tableIdentifier = TableIdentifier.of("MyNamespace", "MyTable"); + final String firstParquetFilePath; + { + final org.apache.iceberg.Table table = catalogAdapter.catalog().loadTable(tableIdentifier); + final List dataFileList = + IcebergUtils.allDataFiles(table, table.currentSnapshot()).collect(Collectors.toList()); + assertThat(dataFileList).hasSize(1); + firstParquetFilePath = dataFileList.get(0).path().toString(); + final Table fromParquet = ParquetTools.readTable(firstParquetFilePath); + assertTableEquals(source.renameColumns("numbers=intCol", "decimals=doubleCol"), fromParquet); + } + + // TODO Verify that the column ID is set correctly after #6156 is merged + + // Now append more data to it + final Table moreData = TableTools.emptyTable(5) + .update("newIntCol = (int) 3 * i + 20", + "newDoubleCol = (double) 3.5 * i + 20"); + writeInstructions = IcebergParquetWriteInstructions.builder() + .verifySchema(true) + .putDhToIcebergColumnRenames("newIntCol", "intCol") + .putDhToIcebergColumnRenames("newDoubleCol", "doubleCol") + .putDhToParquetColumnRenames("newIntCol", "integers") + .putDhToParquetColumnRenames("newDoubleCol", "fractions") + .build(); + catalogAdapter.append("MyNamespace.MyTable", moreData, writeInstructions); + + // Verify that the columns are renamed in the parquet file + { + final org.apache.iceberg.Table table = catalogAdapter.catalog().loadTable(tableIdentifier); + final List dataFileList = + IcebergUtils.allDataFiles(table, table.currentSnapshot()).collect(Collectors.toList()); + assertThat(dataFileList).hasSize(2); + String secondParquetFilePath = null; + for (final DataFile df : dataFileList) { + final String parquetFilePath = df.path().toString(); + if (!parquetFilePath.equals(firstParquetFilePath)) { + secondParquetFilePath = parquetFilePath; + break; + } + } + assertThat(secondParquetFilePath).isNotNull(); + final Table fromParquet = ParquetTools.readTable(secondParquetFilePath); + assertTableEquals(moreData.renameColumns("integers=newIntCol", "fractions=newDoubleCol"), fromParquet); + } + + // TODO Verify that the column ID is set correctly after #6156 is merged + } } 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 7a44b097137..05be3271f15 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 @@ -18,6 +18,7 @@ import org.jetbrains.annotations.NotNull; import java.io.IOException; +import java.io.OutputStream; import java.net.URI; import java.util.ArrayList; import java.util.HashMap; @@ -44,7 +45,7 @@ public final class ParquetFileWriter implements AutoCloseable { public ParquetFileWriter( final URI dest, - final CountingOutputStream countingOutput, + final OutputStream destOutputStream, final int targetPageSize, final ByteBufferAllocator allocator, final MessageType type, @@ -54,7 +55,7 @@ public ParquetFileWriter( this.targetPageSize = targetPageSize; this.allocator = allocator; this.extraMetaData = new HashMap<>(extraMetaData); - this.countingOutput = countingOutput; + this.countingOutput = new CountingOutputStream(destOutputStream); countingOutput.write(MAGIC); this.type = type; this.compressorAdapter = DeephavenCompressorAdapterFactory.getInstance().getByName(codecName); @@ -71,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 getCount() { + return countingOutput.getCount(); + } + @Override public void close() throws IOException { serializeOffsetIndexes(); 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 40200b48358..d9354c636ab 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 @@ -211,15 +211,16 @@ private static long write( final Table t = pretransformTable(table, definition); final TrackingRowSet tableRowSet = t.getRowSet(); final Map> columnSourceMap = t.getColumnSourceMap(); - final CountingOutputStream countingOutput = new CountingOutputStream(destOutputStream); + final long numBytesWritten; try (final ParquetFileWriter parquetFileWriter = getParquetFileWriter(computedCache, definition, - tableRowSet, columnSourceMap, dest, countingOutput, writeInstructions, tableMeta, + 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); + numBytesWritten = parquetFileWriter.getCount(); } destOutputStream.done(); - return countingOutput.getCount(); + return numBytesWritten; } } @@ -323,7 +324,7 @@ private static Table pretransformTable(@NotNull final Table table, @NotNull fina * @param tableRowSet The row set being written * @param columnSourceMap The columns of the table * @param dest The destination URI to write to - * @param countingOutput The output stream to write to dest + * @param destOutputStream The output stream to write to dest * @param writeInstructions Write instructions for the file * @param tableMeta Metadata to include in the parquet metadata * @param tableInfoBuilder Builder for accumulating per-column information to construct the deephaven metadata @@ -338,7 +339,7 @@ private static ParquetFileWriter getParquetFileWriter( @NotNull final RowSet tableRowSet, @NotNull final Map> columnSourceMap, @NotNull final URI dest, - @NotNull final CountingOutputStream countingOutput, + @NotNull final OutputStream destOutputStream, @NotNull final ParquetInstructions writeInstructions, @NotNull final Map tableMeta, @NotNull final TableInfo.Builder tableInfoBuilder, @@ -384,7 +385,7 @@ private static ParquetFileWriter getParquetFileWriter( final Map extraMetaData = new HashMap<>(tableMeta); extraMetaData.put(METADATA_KEY, tableInfoBuilder.build().serializeToJSON()); - return new ParquetFileWriter(dest, countingOutput, writeInstructions.getTargetPageSize(), + return new ParquetFileWriter(dest, destOutputStream, writeInstructions.getTargetPageSize(), new HeapByteBufferAllocator(), mappedSchema.getParquetSchema(), writeInstructions.getCompressionCodecName(), extraMetaData, metadataFileWriter); } 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 43aa8b5fa2e..d9c992e41c7 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 @@ -977,6 +977,7 @@ private static Table readPartitionedTableWithMetadata( @Nullable final SeekableChannelsProvider channelsProvider) { verifyFileLayout(readInstructions, ParquetFileLayout.METADATA_PARTITIONED); if (readInstructions.getTableDefinition().isPresent()) { + // TODO(deephaven-core#6174): Add support for reading metadata files with custom table definition throw new UnsupportedOperationException("Detected table definition inside read instructions, reading " + "metadata files with custom table definition is currently not supported"); } From f0f86cc217fdf312d2345556c9bd13512dd85bf8 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Mon, 7 Oct 2024 16:50:57 -0500 Subject: [PATCH 15/28] Fix for failing jobs --- .../util/IcebergParquetWriteInstructionsTest.java | 4 ++-- .../deephaven/iceberg/layout/IcebergBaseLayout.java | 2 +- py/server/deephaven/experimental/iceberg.py | 12 ++++++------ py/server/tests/test_iceberg.py | 10 +++++----- 4 files changed, 14 insertions(+), 14 deletions(-) diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java index 69a4761efc0..1ff297588a6 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java @@ -165,7 +165,7 @@ void testSetToParquetColumnRename() { @Test void toParquetInstructionTest() { - final IcebergParquetWriteInstructions icebergInstructions = IcebergParquetWriteInstructions.builder() + final IcebergParquetWriteInstructions writeInstructions = IcebergParquetWriteInstructions.builder() .putDhToIcebergColumnRenames("dh1", "ice1") .putDhToParquetColumnRenames("dh2", "parquet1") .compressionCodecName("GZIP") @@ -174,7 +174,7 @@ void toParquetInstructionTest() { .targetPageSize(1024 * 1024) .build(); final Map fieldIdToName = Map.of(2, "field2", 3, "field3"); - final ParquetInstructions parquetInstructions = icebergInstructions.toParquetInstructions( + final ParquetInstructions parquetInstructions = writeInstructions.toParquetInstructions( null, fieldIdToName); assertThat(parquetInstructions.getParquetColumnNameFromColumnNameOrDefault("dh1")).isEqualTo("dh1"); 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 6e94ab3db4c..86b99d84f0b 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 @@ -94,7 +94,7 @@ protected IcebergTableLocationKey locationKey( } } - // Add the data instructions if provided as part of the IcebergInstructions. + // Add the data instructions if provided as part of the iceberg instructions. if (instructions.dataInstructions().isPresent()) { builder.setSpecialInstructions(instructions.dataInstructions().get()); } else { diff --git a/py/server/deephaven/experimental/iceberg.py b/py/server/deephaven/experimental/iceberg.py index f8114ee538f..b00fbc59d8a 100644 --- a/py/server/deephaven/experimental/iceberg.py +++ b/py/server/deephaven/experimental/iceberg.py @@ -13,7 +13,7 @@ from deephaven.jcompat import j_hashmap -_JIcebergInstructions = jpy.get_type("io.deephaven.iceberg.util.IcebergInstructions") +_JIcebergReadInstructions = jpy.get_type("io.deephaven.iceberg.util.IcebergReadInstructions") _JIcebergCatalogAdapter = jpy.get_type("io.deephaven.iceberg.util.IcebergCatalogAdapter") _JIcebergTools = jpy.get_type("io.deephaven.iceberg.util.IcebergTools") @@ -28,13 +28,13 @@ _JSnapshot = jpy.get_type("org.apache.iceberg.Snapshot") -class IcebergInstructions(JObjectWrapper): +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. """ - j_object_type = _JIcebergInstructions + j_object_type = _JIcebergReadInstructions def __init__(self, table_definition: Optional[TableDefinitionLike] = None, @@ -133,14 +133,14 @@ def snapshots(self, table_identifier: str) -> Table: return self.j_object.listSnapshotsAsTable(table_identifier) - def read_table(self, table_identifier: str, instructions: Optional[IcebergInstructions] = None, snapshot_id: Optional[int] = None) -> Table: + def read_table(self, table_identifier: str, instructions: Optional[IcebergReadInstructions] = None, snapshot_id: Optional[int] = None) -> Table: """ Reads the table from the catalog using the provided instructions. Optionally, a snapshot id can be provided to read a specific snapshot of the table. Args: table_identifier (str): the table to read. - instructions (Optional[IcebergInstructions]): the instructions for reading the table. These instructions + instructions (Optional[IcebergReadInstructions]): the instructions for reading the table. These instructions can include column renames, table definition, and specific data instructions for reading the data files from the provider. If omitted, the table will be read with default instructions. snapshot_id (Optional[int]): the snapshot id to read; if omitted the most recent snapshot will be selected. @@ -152,7 +152,7 @@ def read_table(self, table_identifier: str, instructions: Optional[IcebergInstru if instructions is not None: instructions_object = instructions.j_object else: - instructions_object = _JIcebergInstructions.DEFAULT + instructions_object = _JIcebergReadInstructions.DEFAULT if snapshot_id is not None: return Table(self.j_object.readTable(table_identifier, snapshot_id, instructions_object)) diff --git a/py/server/tests/test_iceberg.py b/py/server/tests/test_iceberg.py index 8934299b74d..13331dffa5b 100644 --- a/py/server/tests/test_iceberg.py +++ b/py/server/tests/test_iceberg.py @@ -23,14 +23,14 @@ def tearDown(self): super().tearDown() def test_instruction_create_empty(self): - iceberg_instructions = iceberg.IcebergInstructions() + iceberg_instructions = iceberg.IcebergReadInstructions() def test_instruction_create_with_s3_instructions(self): s3_instructions = s3.S3Instructions(region_name="us-east-1", access_key_id="some_access_key_id", secret_access_key="som_secret_access_key" ) - iceberg_instructions = iceberg.IcebergInstructions(data_instructions=s3_instructions) + iceberg_instructions = iceberg.IcebergReadInstructions(data_instructions=s3_instructions) def test_instruction_create_with_col_renames(self): renames = { @@ -38,7 +38,7 @@ def test_instruction_create_with_col_renames(self): "old_name_b": "new_name_b", "old_name_c": "new_name_c" } - iceberg_instructions = iceberg.IcebergInstructions(column_renames=renames) + iceberg_instructions = iceberg.IcebergReadInstructions(column_renames=renames) col_rename_dict = j_map_to_dict(iceberg_instructions.j_object.columnRenames()) self.assertTrue(col_rename_dict["old_name_a"] == "new_name_a") @@ -52,7 +52,7 @@ def test_instruction_create_with_table_definition_dict(self): "z": dtypes.double, } - iceberg_instructions = iceberg.IcebergInstructions(table_definition=table_def) + iceberg_instructions = iceberg.IcebergReadInstructions(table_definition=table_def) col_names = j_list_to_list(iceberg_instructions.j_object.tableDefinition().get().getColumnNames()) self.assertTrue(col_names[0] == "x") self.assertTrue(col_names[1] == "y") @@ -66,7 +66,7 @@ def test_instruction_create_with_table_definition_list(self): col_def("z", dtypes.double), ] - iceberg_instructions = iceberg.IcebergInstructions(table_definition=table_def) + iceberg_instructions = iceberg.IcebergReadInstructions(table_definition=table_def) col_names = j_list_to_list(iceberg_instructions.j_object.tableDefinition().get().getColumnNames()) self.assertTrue(col_names[0] == "Partition") self.assertTrue(col_names[1] == "x") From adb21e9c851809b02599cf4992505e922725b717 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Tue, 8 Oct 2024 11:49:24 -0500 Subject: [PATCH 16/28] Review with Devin Part 2 --- .../IcebergParquetWriteInstructionsTest.java | 48 ++++++------------- .../iceberg/util/IcebergToolsTest.java | 2 +- .../deephaven/iceberg/base/IcebergUtils.java | 18 ++----- .../iceberg/layout/IcebergBaseLayout.java | 2 +- .../util/IcebergParquetWriteInstructions.java | 17 ------- .../util/IcebergWriteInstructions.java | 8 ++++ .../deephaven/iceberg/CatalogAdapterTest.java | 12 ++--- 7 files changed, 31 insertions(+), 76 deletions(-) diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java index 1ff297588a6..82cd4e9f13b 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java @@ -19,7 +19,6 @@ void defaults() { final IcebergParquetWriteInstructions instructions = IcebergParquetWriteInstructions.builder().build(); assertThat(instructions.tableDefinition().isEmpty()).isTrue(); assertThat(instructions.dataInstructions().isEmpty()).isTrue(); - assertThat(instructions.dhToParquetColumnRenames().isEmpty()).isTrue(); assertThat(instructions.dhToIcebergColumnRenames().isEmpty()).isTrue(); assertThat(instructions.createTableIfNotExist()).isFalse(); assertThat(instructions.verifySchema()).isEmpty(); @@ -76,10 +75,10 @@ void testSetMaximumDictionarySize() { @Test void testSetTargetPageSize() { assertThat(IcebergParquetWriteInstructions.builder() - .targetPageSize(1024 * 1024) + .targetPageSize(1 << 20) .build() .targetPageSize()) - .isEqualTo(1024 * 1024); + .isEqualTo(1 << 20); } @Test @@ -142,53 +141,34 @@ void testSetToIcebergColumnRename() { } @Test - void testSetToParquetColumnRename() { - final IcebergParquetWriteInstructions instructions = IcebergParquetWriteInstructions.builder() - .putDhToParquetColumnRenames("dh1", "parquet1") - .putDhToParquetColumnRenames("dh2", "parquet2") - .build(); - assertThat(instructions.dhToParquetColumnRenames().size()).isEqualTo(2); - assertThat(instructions.dhToParquetColumnRenames().get("dh1")).isEqualTo("parquet1"); - assertThat(instructions.dhToParquetColumnRenames().get("dh2")).isEqualTo("parquet2"); - - final IcebergParquetWriteInstructions instructions2 = IcebergParquetWriteInstructions.builder() - .putAllDhToParquetColumnRenames(Map.of( - "dh1", "parquet1", - "dh2", "parquet2", - "dh3", "parquet3")) - .build(); - assertThat(instructions2.dhToParquetColumnRenames().size()).isEqualTo(3); - assertThat(instructions2.dhToParquetColumnRenames().get("dh1")).isEqualTo("parquet1"); - assertThat(instructions2.dhToParquetColumnRenames().get("dh2")).isEqualTo("parquet2"); - assertThat(instructions2.dhToParquetColumnRenames().get("dh3")).isEqualTo("parquet3"); + void testToIcebergColumnRenameUniqueness() { + try { + IcebergParquetWriteInstructions.builder() + .putDhToIcebergColumnRenames("dh1", "ice1") + .putDhToIcebergColumnRenames("dh2", "ice1") + .build(); + failBecauseExceptionWasNotThrown(IllegalArgumentException.class); + } catch (final IllegalArgumentException e) { + assertThat(e).hasMessageContaining("Duplicate values in column renames"); + } } @Test void toParquetInstructionTest() { final IcebergParquetWriteInstructions writeInstructions = IcebergParquetWriteInstructions.builder() - .putDhToIcebergColumnRenames("dh1", "ice1") - .putDhToParquetColumnRenames("dh2", "parquet1") .compressionCodecName("GZIP") .maximumDictionaryKeys(100) .maximumDictionarySize(200) - .targetPageSize(1024 * 1024) + .targetPageSize(1 << 20) .build(); final Map fieldIdToName = Map.of(2, "field2", 3, "field3"); final ParquetInstructions parquetInstructions = writeInstructions.toParquetInstructions( null, fieldIdToName); - assertThat(parquetInstructions.getParquetColumnNameFromColumnNameOrDefault("dh1")).isEqualTo("dh1"); - assertThat(parquetInstructions.getParquetColumnNameFromColumnNameOrDefault("ice1")).isEqualTo("ice1"); - assertThat(parquetInstructions.getParquetColumnNameFromColumnNameOrDefault("dh2")).isEqualTo("parquet1"); - - assertThat(parquetInstructions.getColumnNameFromParquetColumnNameOrDefault("dh1")).isEqualTo("dh1"); - assertThat(parquetInstructions.getColumnNameFromParquetColumnNameOrDefault("ice1")).isEqualTo("ice1"); - assertThat(parquetInstructions.getColumnNameFromParquetColumnNameOrDefault("parquet1")).isEqualTo("dh2"); - assertThat(parquetInstructions.getCompressionCodecName()).isEqualTo("GZIP"); assertThat(parquetInstructions.getMaximumDictionaryKeys()).isEqualTo(100); assertThat(parquetInstructions.getMaximumDictionarySize()).isEqualTo(200); - assertThat(parquetInstructions.getTargetPageSize()).isEqualTo(1024 * 1024); + assertThat(parquetInstructions.getTargetPageSize()).isEqualTo(1 << 20); assertThat(parquetInstructions.getFieldId("field1")).isEmpty(); assertThat(parquetInstructions.getFieldId("field2")).hasValue(2); assertThat(parquetInstructions.getFieldId("field3")).hasValue(3); 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 63c44001e89..529309d7395 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 @@ -133,13 +133,13 @@ void setUp() throws Exception { @AfterEach void tearDown() throws Exception { - engineCleanup.tearDown(); for (String key : keys) { asyncClient.deleteObject(DeleteObjectRequest.builder().bucket(bucket).key(key).build()).get(); } keys.clear(); asyncClient.deleteBucket(DeleteBucketRequest.builder().bucket(bucket).build()).get(); asyncClient.close(); + engineCleanup.tearDown(); } private void uploadFiles(final File root, final String prefixToRemove) 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 index 827c6cc3698..acd038ce6cb 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/base/IcebergUtils.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/base/IcebergUtils.java @@ -33,8 +33,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.Spliterator; -import java.util.Spliterators; import java.util.stream.Collectors; import java.util.stream.Stream; import java.util.stream.StreamSupport; @@ -50,20 +48,10 @@ public final class IcebergUtils { * @return A stream of {@link DataFile} objects. */ public static Stream allDataFiles(@NotNull final Table table, @NotNull final Snapshot snapshot) { - final FileIO fileIO = table.io(); try { return allManifests(table, snapshot).stream() - .map(manifestFile -> ManifestFiles.read(manifestFile, fileIO)) - .flatMap(manifestReader -> { - try { - return toStream(manifestReader); - } catch (final RuntimeException e) { - throw new TableDataException( - String.format("%s:%d:%s - error reading manifest file", table, - snapshot.snapshotId(), manifestReader), - e); - } - }); + .map(manifestFile -> ManifestFiles.read(manifestFile, table.io())) + .flatMap(IcebergUtils::toStream); } catch (final RuntimeException e) { throw new TableDataException( String.format("%s:%d - error retrieving manifest files", table, snapshot.snapshotId()), e); @@ -180,7 +168,7 @@ public static Type convertToIcebergType(final Class columnType) { public static class SpecAndSchema { private final PartitionSpec partitionSpec; - private Schema schema; + private final Schema schema; private SpecAndSchema(final PartitionSpec partitionSpec, final Schema schema) { this.partitionSpec = partitionSpec; 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 86b99d84f0b..8d6bda501ce 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 @@ -154,7 +154,7 @@ public synchronized void findKeys(@NotNull final Consumer { final URI fileUri = dataFileUri(df); - return cache.computeIfAbsent(fileUri, uri -> keyFromDataFile(df, fileUri)); + return cache.computeIfAbsent(fileUri, uri -> keyFromDataFile(df, uri)); }) .filter(Objects::nonNull) .forEach(locationKeyObserver); diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java index 1e9e0ff72f3..c1277105043 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java @@ -72,12 +72,6 @@ public int targetPageSize() { return ParquetInstructions.DEFAULT_TARGET_PAGE_SIZE; } - /** - * A one-to-one {@link Map map} from Deephaven to Parquet column names to use when writing deephaven tables to - * Iceberg tables. - */ - public abstract Map dhToParquetColumnRenames(); - abstract IcebergParquetWriteInstructions withTableDefinition(@NotNull final TableDefinition tableDefinition); /** @@ -94,13 +88,6 @@ ParquetInstructions toParquetInstructions( tableDefinition().ifPresent(builder::setTableDefinition); dataInstructions().ifPresent(builder::setSpecialInstructions); - // Add any parquet column rename mappings. - if (!dhToParquetColumnRenames().isEmpty()) { - for (final Map.Entry entry : dhToParquetColumnRenames().entrySet()) { - builder.addColumnNameMapping(entry.getValue(), entry.getKey()); - } - } - // Add parquet writing specific instructions. builder.addFieldIdMapping(fieldIdToName); builder.setCompressionCodecName(compressionCodecName()); @@ -121,10 +108,6 @@ public interface Builder extends IcebergWriteInstructions.Builder { Builder targetPageSize(int targetPageSize); - Builder putDhToParquetColumnRenames(String key, String value); - - Builder putAllDhToParquetColumnRenames(Map entries); - IcebergParquetWriteInstructions build(); } 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 index 4b1467b4af6..6500872ce52 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteInstructions.java @@ -49,6 +49,7 @@ public boolean createTableIfNotExist() { * A one-to-one {@link Map map} from Deephaven to Iceberg column names to use when writing deephaven tables to * Iceberg tables. */ + // TODO Please suggest better name for this method, on the read side its just called columnRenames public abstract Map dhToIcebergColumnRenames(); /** @@ -69,4 +70,11 @@ public interface Builder extends IcebergBaseInstructions.B INSTRUCTIONS_BUILDER putAllDhToIcebergColumnRenames(Map entries); } + + @Value.Check + final void checkColumnRenamesUnique() { + if (dhToIcebergColumnRenames().size() != dhToIcebergColumnRenames().values().stream().distinct().count()) { + throw new IllegalArgumentException("Duplicate values in column renames, values must be unique"); + } + } } diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java index e1c8fd73aff..100ea58634a 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java @@ -362,8 +362,6 @@ void testColumnRenameWhileWriting() { IcebergWriteInstructions writeInstructions = IcebergParquetWriteInstructions.builder() .createTableIfNotExist(true) .verifySchema(true) - .putDhToParquetColumnRenames("intCol", "numbers") - .putDhToParquetColumnRenames("doubleCol", "decimals") .build(); catalogAdapter.append("MyNamespace.MyTable", source, writeInstructions); @@ -372,7 +370,7 @@ void testColumnRenameWhileWriting() { // final Table fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); // assertTableEquals(source, fromIceberg); - // Verify that the columns are renamed in the parquet file + // Verify that the column names in the parquet file are same as the table written final TableIdentifier tableIdentifier = TableIdentifier.of("MyNamespace", "MyTable"); final String firstParquetFilePath; { @@ -382,7 +380,7 @@ void testColumnRenameWhileWriting() { assertThat(dataFileList).hasSize(1); firstParquetFilePath = dataFileList.get(0).path().toString(); final Table fromParquet = ParquetTools.readTable(firstParquetFilePath); - assertTableEquals(source.renameColumns("numbers=intCol", "decimals=doubleCol"), fromParquet); + assertTableEquals(source, fromParquet); } // TODO Verify that the column ID is set correctly after #6156 is merged @@ -395,12 +393,10 @@ void testColumnRenameWhileWriting() { .verifySchema(true) .putDhToIcebergColumnRenames("newIntCol", "intCol") .putDhToIcebergColumnRenames("newDoubleCol", "doubleCol") - .putDhToParquetColumnRenames("newIntCol", "integers") - .putDhToParquetColumnRenames("newDoubleCol", "fractions") .build(); catalogAdapter.append("MyNamespace.MyTable", moreData, writeInstructions); - // Verify that the columns are renamed in the parquet file + // Verify that the column names in the parquet file are same as the table written { final org.apache.iceberg.Table table = catalogAdapter.catalog().loadTable(tableIdentifier); final List dataFileList = @@ -416,7 +412,7 @@ void testColumnRenameWhileWriting() { } assertThat(secondParquetFilePath).isNotNull(); final Table fromParquet = ParquetTools.readTable(secondParquetFilePath); - assertTableEquals(moreData.renameColumns("integers=newIntCol", "fractions=newDoubleCol"), fromParquet); + assertTableEquals(moreData, fromParquet); } // TODO Verify that the column ID is set correctly after #6156 is merged From 744ce60852955115994266f828188afd593622ad Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Tue, 8 Oct 2024 11:56:59 -0500 Subject: [PATCH 17/28] Review with Devin Part 3 --- .../io/deephaven/iceberg/util/IcebergWriteInstructions.java | 2 ++ 1 file changed, 2 insertions(+) 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 index 6500872ce52..a863030281f 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteInstructions.java @@ -3,6 +3,7 @@ // package io.deephaven.iceberg.util; +import io.deephaven.util.annotations.InternalUseOnly; import org.immutables.value.Value; import org.immutables.value.Value.Default; @@ -55,6 +56,7 @@ public boolean createTableIfNotExist() { /** * The inverse map of {@link #dhToIcebergColumnRenames()}. */ + @InternalUseOnly @Value.Lazy public Map icebergToDhColumnRenames() { return dhToIcebergColumnRenames().entrySet().stream() From a8252ce623139d8c6e83cbdfaffd6a5ba7400e71 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Tue, 8 Oct 2024 14:03:13 -0500 Subject: [PATCH 18/28] Review with Devin Part 4 --- .../IcebergParquetWriteInstructionsTest.java | 3 +- .../deephaven/iceberg/base/IcebergUtils.java | 57 ++++++++----------- 2 files changed, 25 insertions(+), 35 deletions(-) diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java index 82cd4e9f13b..d24a2691ccf 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java @@ -7,7 +7,6 @@ import org.junit.jupiter.api.Test; import java.util.Map; -import java.util.Optional; import static org.assertj.core.api.Assertions.failBecauseExceptionWasNotThrown; import static org.assertj.core.api.AssertionsForClassTypes.assertThat; @@ -42,7 +41,7 @@ void testSetVerifySchema() { .verifySchema(true) .build() .verifySchema()) - .isEqualTo(Optional.of(true)); + .hasValue(true); } @Test 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 index acd038ce6cb..2494dbf2c90 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/base/IcebergUtils.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/base/IcebergUtils.java @@ -16,7 +16,6 @@ import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.FileIO; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.jetbrains.annotations.NotNull; @@ -30,6 +29,7 @@ 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.Set; @@ -39,6 +39,23 @@ 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 Add support for writing big decimals and lists + } + /** * Get a stream of all {@link DataFile} objects from the given {@link Table} and {@link Snapshot}. * @@ -48,14 +65,9 @@ public final class IcebergUtils { * @return A stream of {@link DataFile} objects. */ public static Stream allDataFiles(@NotNull final Table table, @NotNull final Snapshot snapshot) { - try { - return allManifests(table, snapshot).stream() - .map(manifestFile -> ManifestFiles.read(manifestFile, table.io())) - .flatMap(IcebergUtils::toStream); - } catch (final RuntimeException e) { - throw new TableDataException( - String.format("%s:%d - error retrieving manifest files", table, snapshot.snapshotId()), e); - } + return allManifests(table, snapshot).stream() + .map(manifestFile -> ManifestFiles.read(manifestFile, table.io())) + .flatMap(IcebergUtils::toStream); } /** @@ -137,33 +149,12 @@ public static io.deephaven.qst.type.Type convertToDHType(@NotNull final Type * @return The converted Iceberg type. */ public static Type convertToIcebergType(final Class columnType) { - if (columnType == Boolean.class) { - return Types.BooleanType.get(); - } else if (columnType == double.class) { - return Types.DoubleType.get(); - } else if (columnType == float.class) { - return Types.FloatType.get(); - } else if (columnType == int.class) { - return Types.IntegerType.get(); - } else if (columnType == long.class) { - return Types.LongType.get(); - } else if (columnType == String.class) { - return Types.StringType.get(); - } else if (columnType == Instant.class) { - return Types.TimestampType.withZone(); - } else if (columnType == LocalDateTime.class) { - return Types.TimestampType.withoutZone(); - } else if (columnType == LocalDate.class) { - return Types.DateType.get(); - } else if (columnType == LocalTime.class) { - return Types.TimeType.get(); - } else if (columnType == byte[].class) { - return Types.BinaryType.get(); + 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()); } - // TODO Add support for writing big decimals - // TODO Add support for reading and writing lists } public static class SpecAndSchema { From 5a64fafe2f8ae0e30b77578c27b369736d7af3f9 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Mon, 14 Oct 2024 11:38:07 -0500 Subject: [PATCH 19/28] Minor tweaks --- .../src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 7e6bf6a842e..043ef29c3de 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 @@ -45,7 +45,7 @@ public static IcebergCatalogAdapter createS3Rest( @Nullable final String secretAccessKey, @Nullable final String endpointOverride) { - // Set up the properties map for the Iceberg catalog, and configure it from Iceberg instructions. + // Set up the properties map for the Iceberg catalog final Map properties = new HashMap<>(); if (!Strings.isNullOrEmpty(accessKeyId) && !Strings.isNullOrEmpty(secretAccessKey)) { properties.put(S3FileIOProperties.ACCESS_KEY_ID, accessKeyId); From ba70f1a1c8fc6614803dbb3e69b829b081595987 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Mon, 14 Oct 2024 13:16:11 -0500 Subject: [PATCH 20/28] More tweaks --- .../src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 043ef29c3de..462bd6a4d9c 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 @@ -102,6 +102,6 @@ private static IcebergCatalogAdapter createAdapterCommon( final String catalogName = name != null ? name : "IcebergCatalog-" + catalogURI; catalog.initialize(catalogName, properties); - return new IcebergCatalogAdapter(catalog, properties); + return IcebergCatalogAdapter.of(catalog, properties); } } From 96db353f1eae7b922e128ef2c571e4d932637ea9 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Mon, 14 Oct 2024 16:46:32 -0500 Subject: [PATCH 21/28] Updated some comments --- .../iceberg/util/IcebergCatalogAdapter.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) 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 3497b2f903b..2faa83acaf2 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 @@ -902,8 +902,8 @@ public void append( } /** - * Overwrite the existing iceberg table with the provided deephaven table in a single snapshot. This will change the - * schema of the existing table to match the provided deephaven table. + * Overwrite the existing iceberg table with the provided deephaven table in a single snapshot. This will overwrite + * the schema of the existing table to match the provided Deephaven table if they do not match. * * @param tableIdentifier The identifier string for the iceberg table to overwrite * @param dhTable The deephaven table to overwrite with @@ -917,8 +917,8 @@ public void overwrite( } /** - * Overwrite the existing iceberg table with the provided deephaven table in a single snapshot. This will change the - * schema of the existing table to match the provided deephaven table. + * Overwrite the existing iceberg table with the provided deephaven table in a single snapshot. This will overwrite + * the schema of the existing table to match the provided Deephaven table if they do not match. * * @param tableIdentifier The identifier for the iceberg table to overwrite * @param dhTable The deephaven table to overwrite with @@ -934,7 +934,7 @@ public void overwrite( /** * Overwrite the existing iceberg table with the provided deephaven tables appended together in a single snapshot. * All tables should have the same definition, else a table definition should be provided in the instructions. This - * will change the schema of the existing table to match the provided deephaven tables. + * will overwrite the schema of the existing table to match the provided Deephaven table if they do not match. * * @param tableIdentifier The identifier string for the iceberg table to overwrite * @param dhTables The deephaven tables to overwrite with @@ -950,7 +950,7 @@ public void overwrite( /** * Overwrite the existing iceberg table with the provided deephaven tables appended together in a single snapshot. * All tables should have the same definition, else a table definition should be provided in the instructions. This - * will change the schema of the existing table to match the provided deephaven tables. + * will overwrite the schema of the existing table to match the provided Deephaven table if they do not match. * * @param tableIdentifier The identifier for the iceberg table to overwrite * @param dhTables The deephaven tables to overwrite with From 6e2c233632e46e60bd684a0a56b003a877fd2ea2 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Tue, 15 Oct 2024 12:58:17 -0500 Subject: [PATCH 22/28] Updated javadoc and added new tests --- .../iceberg/util/IcebergCatalogAdapter.java | 29 +++--- .../deephaven/iceberg/CatalogAdapterTest.java | 98 +++++++++++++------ 2 files changed, 83 insertions(+), 44 deletions(-) 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 2faa83acaf2..6a5107c0cbb 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 @@ -902,8 +902,9 @@ public void append( } /** - * Overwrite the existing iceberg table with the provided deephaven table in a single snapshot. This will overwrite - * the schema of the existing table to match the provided Deephaven table if they do not match. + * Overwrite the existing iceberg table with the provided deephaven table, creating two snapshots: one for "delete" + * {@link Snapshot#operation() operation} and another for "append". This will overwrite the schema of the existing + * table to match the provided Deephaven table if they do not match. * * @param tableIdentifier The identifier string for the iceberg table to overwrite * @param dhTable The deephaven table to overwrite with @@ -917,8 +918,9 @@ public void overwrite( } /** - * Overwrite the existing iceberg table with the provided deephaven table in a single snapshot. This will overwrite - * the schema of the existing table to match the provided Deephaven table if they do not match. + * Overwrite the existing iceberg table with the provided deephaven table, creating two snapshots: one for "delete" + * {@link Snapshot#operation() operation} and another for "append". This will overwrite the schema of the existing + * table to match the provided Deephaven table if they do not match. * * @param tableIdentifier The identifier for the iceberg table to overwrite * @param dhTable The deephaven table to overwrite with @@ -932,9 +934,10 @@ public void overwrite( } /** - * Overwrite the existing iceberg table with the provided deephaven tables appended together in a single snapshot. - * All tables should have the same definition, else a table definition should be provided in the instructions. This - * will overwrite the schema of the existing table to match the provided Deephaven table if they do not match. + * Overwrite the existing iceberg table with the provided deephaven table, creating two snapshots: one for "delete" + * {@link Snapshot#operation() operation} and another for "append". All tables should have the same definition, else + * a table definition should be provided in the instructions. This will overwrite the schema of the existing table + * to match the provided Deephaven table if they do not match. * * @param tableIdentifier The identifier string for the iceberg table to overwrite * @param dhTables The deephaven tables to overwrite with @@ -948,9 +951,10 @@ public void overwrite( } /** - * Overwrite the existing iceberg table with the provided deephaven tables appended together in a single snapshot. - * All tables should have the same definition, else a table definition should be provided in the instructions. This - * will overwrite the schema of the existing table to match the provided Deephaven table if they do not match. + * Overwrite the existing iceberg table with the provided deephaven tables, creating two snapshots: one for "delete" + * {@link Snapshot#operation() operation} and another for "append". All tables should have the same definition, else + * a table definition should be provided in the instructions. This will overwrite the schema of the existing table + * to match the provided Deephaven table if they do not match. * * @param tableIdentifier The identifier for the iceberg table to overwrite * @param dhTables The deephaven tables to overwrite with @@ -1215,7 +1219,7 @@ private static List writeParquet( } /** - * Commit the changes to the Iceberg table by creating a single snapshot. + * Commit the changes to the Iceberg table by creating snapshots. */ private static void commit( @NotNull final org.apache.iceberg.Table icebergTable, @@ -1260,7 +1264,8 @@ private static void commit( appendFiles.forEach(append::appendFile); append.commit(); - // Commit the transaction to create a new snapshot + // Commit the transaction, creating new separate snapshots for "delete" and "append" operation. + // Note that no new snapshot will be created for the schema change. icebergTransaction.commitTransaction(); } diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java index 100ea58634a..ba30df88eb4 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java @@ -17,11 +17,13 @@ import junit.framework.TestCase; 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.junit.jupiter.api.Test; +import java.util.Iterator; import java.util.List; import java.util.stream.Collectors; @@ -65,44 +67,49 @@ void appendTableBasicTest() { final Table source = TableTools.emptyTable(10) .update("intCol = (int) 2 * i + 10", "doubleCol = (double) 2.5 * i + 10"); + final String tableIdentifier = "MyNamespace.MyTable"; try { - catalogAdapter.append("MyNamespace.MyTable", source, null); + catalogAdapter.append(tableIdentifier, source, null); } catch (RuntimeException e) { assertThat(e.getMessage()).contains("Table does not exist"); } final IcebergWriteInstructions writeInstructions = IcebergParquetWriteInstructions.builder() .createTableIfNotExist(true) .build(); - catalogAdapter.append("MyNamespace.MyTable", source, writeInstructions); - Table fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + catalogAdapter.append(tableIdentifier, source, writeInstructions); + Table fromIceberg = catalogAdapter.readTable(tableIdentifier, null); assertTableEquals(source, fromIceberg); + assertThat(catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots()).hasSize(1); // Append more data final Table moreData = TableTools.emptyTable(5) .update("intCol = (int) 3 * i + 20", "doubleCol = (double) 3.5 * i + 20"); - catalogAdapter.append("MyNamespace.MyTable", moreData, writeInstructions); - fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + catalogAdapter.append(tableIdentifier, moreData, writeInstructions); + fromIceberg = catalogAdapter.readTable(tableIdentifier, null); final Table expected = TableTools.merge(moreData, source); assertTableEquals(expected, fromIceberg); + assertThat(catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots()).hasSize(2); // Append an empty table final Table emptyTable = TableTools.emptyTable(0) .update("intCol = (int) 4 * i + 30", "doubleCol = (double) 4.5 * i + 30"); - catalogAdapter.append("MyNamespace.MyTable", emptyTable, writeInstructions); - fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + catalogAdapter.append(tableIdentifier, emptyTable, writeInstructions); + fromIceberg = catalogAdapter.readTable(tableIdentifier, null); assertTableEquals(expected, fromIceberg); + assertThat(catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots()).hasSize(3); // Append multiple tables in a single call final Table someMoreData = TableTools.emptyTable(3) .update("intCol = (int) 5 * i + 40", "doubleCol = (double) 5.5 * i + 40"); - catalogAdapter.append("MyNamespace.MyTable", new Table[] {someMoreData, moreData, emptyTable}, + catalogAdapter.append(tableIdentifier, new Table[] {someMoreData, moreData, emptyTable}, writeInstructions); - fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + fromIceberg = catalogAdapter.readTable(tableIdentifier, null); final Table expected2 = TableTools.merge(someMoreData, moreData, expected); assertTableEquals(expected2, fromIceberg); + assertThat(catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots()).hasSize(4); } @Test @@ -110,43 +117,62 @@ void overwriteTablesBasicTest() { final Table source = TableTools.emptyTable(10) .update("intCol = (int) 2 * i + 10", "doubleCol = (double) 2.5 * i + 10"); + final String tableIdentifier = "MyNamespace.MyTable"; try { - catalogAdapter.overwrite("MyNamespace.MyTable", source, null); + catalogAdapter.overwrite(tableIdentifier, source, null); } catch (RuntimeException e) { assertThat(e.getMessage()).contains("Table does not exist"); } final IcebergWriteInstructions writeInstructions = IcebergParquetWriteInstructions.builder() .createTableIfNotExist(true) .build(); - catalogAdapter.overwrite("MyNamespace.MyTable", source, writeInstructions); - Table fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + catalogAdapter.overwrite(tableIdentifier, source, writeInstructions); + Table fromIceberg = catalogAdapter.readTable(tableIdentifier, null); assertTableEquals(source, fromIceberg); + { + final Iterable snapshots = + catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots(); + assertThat(snapshots).hasSize(1); + assertThat(snapshots.iterator().next().operation()).isEqualTo("append"); + } // Overwrite with more data final Table moreData = TableTools.emptyTable(5) .update("intCol = (int) 3 * i + 20", "doubleCol = (double) 3.5 * i + 20"); - catalogAdapter.overwrite("MyNamespace.MyTable", moreData, writeInstructions); - fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + catalogAdapter.overwrite(tableIdentifier, moreData, writeInstructions); + fromIceberg = catalogAdapter.readTable(tableIdentifier, null); assertTableEquals(moreData, fromIceberg); + // 2 snapshots added, one for the delete and other for append + { + final Iterable snapshots = + catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots(); + assertThat(snapshots).hasSize(3); + final Iterator snapshotIter = snapshots.iterator(); + assertThat(snapshotIter.next().operation()).isEqualTo("append"); + assertThat(snapshotIter.next().operation()).isEqualTo("delete"); + assertThat(snapshotIter.next().operation()).isEqualTo("append"); + } // Overwrite with an empty table final Table emptyTable = TableTools.emptyTable(0) .update("intCol = (int) 4 * i + 30", "doubleCol = (double) 4.5 * i + 30"); - catalogAdapter.overwrite("MyNamespace.MyTable", emptyTable, writeInstructions); - fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + catalogAdapter.overwrite(tableIdentifier, emptyTable, writeInstructions); + fromIceberg = catalogAdapter.readTable(tableIdentifier, null); assertTableEquals(emptyTable, fromIceberg); + assertThat(catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots()).hasSize(5); // Overwrite with multiple tables in a single call final Table someMoreData = TableTools.emptyTable(3) .update("intCol = (int) 5 * i + 40", "doubleCol = (double) 5.5 * i + 40"); - catalogAdapter.overwrite("MyNamespace.MyTable", new Table[] {someMoreData, moreData, emptyTable}, + catalogAdapter.overwrite(tableIdentifier, new Table[] {someMoreData, moreData, emptyTable}, writeInstructions); - fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + fromIceberg = catalogAdapter.readTable(tableIdentifier, null); final Table expected2 = TableTools.merge(someMoreData, moreData); assertTableEquals(expected2, fromIceberg); + assertThat(catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots()).hasSize(7); } @Test @@ -154,18 +180,20 @@ void overwriteWithDifferentDefinition() { final Table source = TableTools.emptyTable(10) .update("intCol = (int) 2 * i + 10", "doubleCol = (double) 2.5 * i + 10"); + final String tableIdentifier = "MyNamespace.MyTable"; final IcebergWriteInstructions writeInstructionsWithSchemaMatching = IcebergParquetWriteInstructions.builder() .createTableIfNotExist(true) .verifySchema(true) .build(); - catalogAdapter.append("MyNamespace.MyTable", source, writeInstructionsWithSchemaMatching); - Table fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + catalogAdapter.append(tableIdentifier, source, writeInstructionsWithSchemaMatching); + Table fromIceberg = catalogAdapter.readTable(tableIdentifier, null); assertTableEquals(source, fromIceberg); + assertThat(catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots()).hasSize(1); final Table differentSource = TableTools.emptyTable(10) .update("intCol = (int) 2 * i + 10"); try { - catalogAdapter.overwrite("MyNamespace.MyTable", differentSource, writeInstructionsWithSchemaMatching); + catalogAdapter.overwrite(tableIdentifier, differentSource, writeInstructionsWithSchemaMatching); } catch (RuntimeException e) { assertThat(e.getMessage()).contains("Schema verification failed"); } @@ -173,17 +201,19 @@ void overwriteWithDifferentDefinition() { // By default, schema verification should be disabled for overwriting final IcebergWriteInstructions writeInstructionsWithoutSchemaMatching = IcebergParquetWriteInstructions.builder().build(); - catalogAdapter.overwrite("MyNamespace.MyTable", differentSource, writeInstructionsWithoutSchemaMatching); - fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + catalogAdapter.overwrite(tableIdentifier, differentSource, writeInstructionsWithoutSchemaMatching); + fromIceberg = catalogAdapter.readTable(tableIdentifier, null); assertTableEquals(differentSource, fromIceberg); + assertThat(catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots()).hasSize(3); // Append more data to this table with the updated schema final Table moreData = TableTools.emptyTable(5) .update("intCol = (int) 3 * i + 20"); - catalogAdapter.append("MyNamespace.MyTable", moreData, writeInstructionsWithoutSchemaMatching); - fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + catalogAdapter.append(tableIdentifier, moreData, writeInstructionsWithoutSchemaMatching); + fromIceberg = catalogAdapter.readTable(tableIdentifier, null); final Table expected = TableTools.merge(moreData, differentSource); assertTableEquals(expected, fromIceberg); + assertThat(catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots()).hasSize(4); } @Test @@ -191,19 +221,21 @@ void appendWithDifferentDefinition() { final Table source = TableTools.emptyTable(10) .update("intCol = (int) 2 * i + 10", "doubleCol = (double) 2.5 * i + 10"); + final String tableIdentifier = "MyNamespace.MyTable"; // By default, schema verification should be enabled for appending final IcebergWriteInstructions writeInstructions = IcebergParquetWriteInstructions.builder() .createTableIfNotExist(true) .build(); - catalogAdapter.append("MyNamespace.MyTable", source, writeInstructions); - Table fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + catalogAdapter.append(tableIdentifier, source, writeInstructions); + Table fromIceberg = catalogAdapter.readTable(tableIdentifier, null); assertTableEquals(source, fromIceberg); + assertThat(catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots()).hasSize(1); final Table differentSource = TableTools.emptyTable(10) .update("shortCol = (short) 2 * i + 10"); try { - catalogAdapter.append("MyNamespace.MyTable", differentSource, writeInstructions); + catalogAdapter.append(tableIdentifier, differentSource, writeInstructions); } catch (RuntimeException e) { assertThat(e.getMessage()).contains("Schema verification failed"); } @@ -211,19 +243,21 @@ void appendWithDifferentDefinition() { // Append a table with just the int column, should be compatible with the existing schema final Table compatibleSource = TableTools.emptyTable(10) .update("intCol = (int) 5 * i + 10"); - catalogAdapter.append("MyNamespace.MyTable", compatibleSource, writeInstructions); - fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + catalogAdapter.append(tableIdentifier, compatibleSource, writeInstructions); + fromIceberg = catalogAdapter.readTable(tableIdentifier, null); final Table expected = TableTools.merge(compatibleSource.update("doubleCol = NULL_DOUBLE"), source); assertTableEquals(expected, fromIceberg); + assertThat(catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots()).hasSize(2); // Append more data final Table moreData = TableTools.emptyTable(5) .update("intCol = (int) 3 * i + 20", "doubleCol = (double) 3.5 * i + 20"); - catalogAdapter.append("MyNamespace.MyTable", moreData, writeInstructions); - fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); + catalogAdapter.append(tableIdentifier, moreData, writeInstructions); + fromIceberg = catalogAdapter.readTable(tableIdentifier, null); final Table expected2 = TableTools.merge(moreData, expected); assertTableEquals(expected2, fromIceberg); + assertThat(catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots()).hasSize(3); } @Test From 0ebeba2b65d0cf0ac03e66ea809a8ac17aadd9d7 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Thu, 17 Oct 2024 14:55:03 -0500 Subject: [PATCH 23/28] Review with Ryan Part 1 --- .../iceberg/util/IcebergCatalogAdapter.java | 168 ++++++++++-------- .../deephaven/iceberg/CatalogAdapterTest.java | 112 +++++++----- 2 files changed, 165 insertions(+), 115 deletions(-) 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 6a5107c0cbb..f0647b93dca 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 @@ -3,7 +3,6 @@ // package io.deephaven.iceberg.util; -import io.deephaven.UncheckedDeephavenException; import io.deephaven.api.util.NameValidator; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.*; @@ -29,12 +28,11 @@ import io.deephaven.parquet.table.ParquetTools; import io.deephaven.time.DateTimeUtils; import io.deephaven.util.annotations.VisibleForTesting; -import org.apache.hadoop.util.StringUtils; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; -import org.apache.iceberg.DeleteFiles; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.OverwriteFiles; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -47,6 +45,8 @@ 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.apache.iceberg.exceptions.NamespaceNotEmptyException; import org.apache.iceberg.rest.RESTCatalog; import org.apache.iceberg.rest.ResourcePaths; import org.apache.iceberg.types.Type; @@ -776,9 +776,10 @@ private Table readTableInternal( throw new IllegalArgumentException("Table not found: " + tableIdentifier); } + final Schema schema = tableSnapshot == null ? table.schema() : table.schemas().get(tableSnapshot.schemaId()); + // Do we want the latest or a specific snapshot? final Snapshot snapshot = tableSnapshot != null ? tableSnapshot : table.currentSnapshot(); - final Schema schema = snapshot == null ? table.schema() : table.schemas().get(snapshot.schemaId()); // Load the partitioning schema. final org.apache.iceberg.PartitionSpec partitionSpec = table.spec(); @@ -814,7 +815,7 @@ private Table readTableInternal( refreshService = null; updateSourceRegistrar = null; - description = "Read static iceberg table with " + keyFinder; + description = "Read static Iceberg table with " + keyFinder; final AbstractTableLocationProvider locationProvider = new PollingTableLocationProvider<>( StandaloneTableKey.getInstance(), @@ -840,11 +841,11 @@ public Catalog catalog() { } /** - * Append the provided deephaven table as a new partition to the existing iceberg table in a single snapshot. This + * Append the provided Deephaven table as a new partition to the existing Iceberg table in a single snapshot. This * will not change the schema of the existing table. * - * @param tableIdentifier The identifier string for the iceberg table to append to - * @param dhTable The deephaven table to append + * @param tableIdentifier The identifier string for the Iceberg table to append to + * @param dhTable The Deephaven table to append * @param instructions The instructions for customizations while writing, or null to use default instructions */ public void append( @@ -855,11 +856,11 @@ public void append( } /** - * Append the provided deephaven table as a new partition to the existing iceberg table in a single snapshot. This + * Append the provided Deephaven table as a new partition to the existing Iceberg table in a single snapshot. This * will not change the schema of the existing table. * - * @param tableIdentifier The identifier for the iceberg table to append to - * @param dhTable The deephaven table to append + * @param tableIdentifier The identifier for the Iceberg table to append to + * @param dhTable The Deephaven table to append * @param instructions The instructions for customizations while writing, or null to use default instructions */ public void append( @@ -870,12 +871,12 @@ public void append( } /** - * Append the provided deephaven tables as new partitions to the existing iceberg table in a single snapshot. All + * Append the provided Deephaven tables as new partitions to the existing Iceberg table in a single snapshot. All * tables should have the same definition, else a table definition should be provided in the instructions. This will * not change the schema of the existing table. * - * @param tableIdentifier The identifier string for the iceberg table to append to - * @param dhTables The deephaven tables to append + * @param tableIdentifier The identifier string for the Iceberg table to append to + * @param dhTables The Deephaven tables to append * @param instructions The instructions for customizations while writing, or null to use default instructions */ public void append( @@ -886,12 +887,12 @@ public void append( } /** - * Append the provided deephaven tables as new partitions to the existing iceberg table in a single snapshot. All + * Append the provided Deephaven tables as new partitions to the existing Iceberg table in a single snapshot. All * tables should have the same definition, else a table definition should be provided in the instructions. This will * not change the schema of the existing table. * - * @param tableIdentifier The identifier for the iceberg table to append to - * @param dhTables The deephaven tables to append + * @param tableIdentifier The identifier for the Iceberg table to append to + * @param dhTables The Deephaven tables to append * @param instructions The instructions for customizations while writing, or null to use default instructions */ public void append( @@ -902,12 +903,13 @@ public void append( } /** - * Overwrite the existing iceberg table with the provided deephaven table, creating two snapshots: one for "delete" - * {@link Snapshot#operation() operation} and another for "append". This will overwrite the schema of the existing - * table to match the provided Deephaven table if they do not match. + * Overwrite the existing Iceberg table with the provided Deephaven table in a single snapshot. This will overwrite + * the schema of the existing table to match the provided Deephaven table if they do not match. + *

+ * Overwriting a table while racing with other writers can lead to undefined results. * - * @param tableIdentifier The identifier string for the iceberg table to overwrite - * @param dhTable The deephaven table to overwrite with + * @param tableIdentifier The identifier string for the Iceberg table to overwrite + * @param dhTable The Deephaven table to overwrite with; an empty array will overwrite with an empty table * @param instructions The instructions for customizations while writing, or null to use default instructions */ public void overwrite( @@ -918,12 +920,13 @@ public void overwrite( } /** - * Overwrite the existing iceberg table with the provided deephaven table, creating two snapshots: one for "delete" - * {@link Snapshot#operation() operation} and another for "append". This will overwrite the schema of the existing - * table to match the provided Deephaven table if they do not match. + * Overwrite the existing Iceberg table with the provided Deephaven table in a single snapshot. This will overwrite + * the schema of the existing table to match the provided Deephaven table if they do not match. + *

+ * Overwriting a table while racing with other writers can lead to undefined results. * - * @param tableIdentifier The identifier for the iceberg table to overwrite - * @param dhTable The deephaven table to overwrite with + * @param tableIdentifier The identifier for the Iceberg table to overwrite + * @param dhTable The Deephaven table to overwrite with; an empty array will overwrite with an empty table * @param instructions The instructions for customizations while writing, or null to use default instructions */ public void overwrite( @@ -934,13 +937,14 @@ public void overwrite( } /** - * Overwrite the existing iceberg table with the provided deephaven table, creating two snapshots: one for "delete" - * {@link Snapshot#operation() operation} and another for "append". All tables should have the same definition, else - * a table definition should be provided in the instructions. This will overwrite the schema of the existing table - * to match the provided Deephaven table if they do not match. + * Overwrite the existing Iceberg table with the provided Deephaven table in a single snapshot. All tables should + * have the same definition, else a table definition should be provided in the instructions. This will overwrite the + * schema of the existing table to match the provided Deephaven table if they do not match. + *

+ * Overwriting a table while racing with other writers can lead to undefined results. * - * @param tableIdentifier The identifier string for the iceberg table to overwrite - * @param dhTables The deephaven tables to overwrite with + * @param tableIdentifier The identifier string for the Iceberg table to overwrite + * @param dhTables The Deephaven tables to overwrite with; an empty array will overwrite with an empty table * @param instructions The instructions for customizations while writing, or null to use default instructions */ public void overwrite( @@ -951,13 +955,14 @@ public void overwrite( } /** - * Overwrite the existing iceberg table with the provided deephaven tables, creating two snapshots: one for "delete" - * {@link Snapshot#operation() operation} and another for "append". All tables should have the same definition, else - * a table definition should be provided in the instructions. This will overwrite the schema of the existing table - * to match the provided Deephaven table if they do not match. + * Overwrite the existing Iceberg table with the provided Deephaven tables in a single snapshot. All tables should + * have the same definition, else a table definition should be provided in the instructions. This will overwrite the + * schema of the existing table to match the provided Deephaven table if they do not match. + *

+ * Overwriting a table while racing with other writers can lead to undefined results. * - * @param tableIdentifier The identifier for the iceberg table to overwrite - * @param dhTables The deephaven tables to overwrite with + * @param tableIdentifier The identifier for the Iceberg table to overwrite + * @param dhTables The Deephaven tables to overwrite with; an empty array will overwrite with an empty table * @param instructions The instructions for customizations while writing, or null to use default instructions */ public void overwrite( @@ -1051,11 +1056,25 @@ public List writeDataFiles( */ private List writeImpl( @NotNull final TableIdentifier tableIdentifier, - @NotNull final Table[] dhTables, + @NotNull Table[] dhTables, @Nullable final IcebergWriteInstructions instructions, final boolean overwrite, final boolean addSnapshot) { + if (overwrite && !addSnapshot) { + throw new IllegalArgumentException("Cannot overwrite an Iceberg table without adding a snapshot"); + } + if (dhTables.length == 0) { + if (!overwrite) { + // Nothing to append + return Collections.emptyList(); + } + // Overwrite with an empty table + dhTables = new Table[] {TableTools.emptyTable(0)}; + } + IcebergParquetWriteInstructions writeInstructions = verifyWriteInstructions(instructions); + + // Don't verify schema by default if overwriting final boolean verifySchema = writeInstructions.verifySchema().orElse(!overwrite); final TableDefinition useDefinition; @@ -1112,27 +1131,31 @@ private List writeImpl( } try { - final List parquetFileinfo = + final List parquetFileInfo = writeParquet(icebergTable, dhTables, writeInstructions); - final List appendFiles = dataFilesFromParquet(parquetFileinfo); + final List appendFiles = dataFilesFromParquet(parquetFileInfo); if (addSnapshot) { commit(icebergTable, newSpecAndSchema, appendFiles, overwrite && !newTableCreated, verifySchema); } return appendFiles; - } catch (final RuntimeException writeException) { + } catch (final Throwable throwable) { if (newTableCreated) { // Delete it to avoid leaving a partial table in the catalog try { catalog.dropTable(tableIdentifier, true); } catch (final RuntimeException dropException) { - writeException.addSuppressed(dropException); + throwable.addSuppressed(dropException); } } if (newNamespaceCreated) { // Delete it to avoid leaving a partial namespace in the catalog - dropNamespaceIfExists(tableIdentifier.namespace()); + try { + dropNamespaceIfExists(tableIdentifier.namespace()); + } catch (final RuntimeException dropException) { + throwable.addSuppressed(dropException); + } } - throw writeException; + throw throwable; } } @@ -1152,32 +1175,34 @@ private static IcebergParquetWriteInstructions verifyWriteInstructions( private boolean createNamespaceIfNotExists(@NotNull final Namespace namespace) { if (catalog instanceof SupportsNamespaces) { final SupportsNamespaces nsCatalog = (SupportsNamespaces) catalog; - if (!nsCatalog.namespaceExists(namespace)) { + try { nsCatalog.createNamespace(namespace); return true; + } catch (final AlreadyExistsException | UnsupportedOperationException e) { + return false; } } return false; } - private void dropNamespaceIfExists(@NotNull final Namespace namespace) { + private boolean dropNamespaceIfExists(@NotNull final Namespace namespace) { if (catalog instanceof SupportsNamespaces) { final SupportsNamespaces nsCatalog = (SupportsNamespaces) catalog; - nsCatalog.dropNamespace(namespace); + try { + return nsCatalog.dropNamespace(namespace); + } catch (final NamespaceNotEmptyException e) { + return false; + } } + return false; } private org.apache.iceberg.Table createNewIcebergTable( @NotNull final TableIdentifier tableIdentifier, @NotNull final SpecAndSchema specAndSchema, @NotNull final IcebergParquetWriteInstructions writeInstructions) { - if (catalog.tableExists(tableIdentifier)) { - throw new IllegalArgumentException("Table already exists: " + tableIdentifier); - } - return catalog.createTable(tableIdentifier, specAndSchema.schema(), specAndSchema.partitionSpec(), null, - Map.of(TableProperties.DEFAULT_FILE_FORMAT, DEFAULT_GENERATED_FILE_FORMAT, - TableProperties.PARQUET_COMPRESSION, - StringUtils.toLowerCase(writeInstructions.compressionCodecName()))); + return catalog.createTable(tableIdentifier, specAndSchema.schema(), specAndSchema.partitionSpec(), + Map.of(TableProperties.DEFAULT_FILE_FORMAT, DEFAULT_GENERATED_FILE_FORMAT)); } private static class CompletedParquetWrite { @@ -1208,6 +1233,10 @@ private static List writeParquet( // Write the data to parquet files int count = 0; for (final Table dhTable : dhTables) { + if (dhTable.numColumns() == 0) { + // Skip writing empty tables with no columns + continue; + } final String filename = String.format( "00000-%d-%s.parquet", count++, @@ -1219,7 +1248,7 @@ private static List writeParquet( } /** - * Commit the changes to the Iceberg table by creating snapshots. + * Commit the changes to the Iceberg table by creating a snapshot. */ private static void commit( @NotNull final org.apache.iceberg.Table icebergTable, @@ -1227,16 +1256,16 @@ private static void commit( @NotNull final Iterable appendFiles, final boolean overwrite, final boolean schemaVerified) { - // Append new data files to the table final Transaction icebergTransaction = icebergTable.newTransaction(); - if (overwrite) { + final OverwriteFiles overwriteFiles = icebergTransaction.newOverwrite(); + // Delete all the existing data files in the table - final DeleteFiles deletes = icebergTransaction.newDelete(); try (final Stream dataFiles = allDataFiles(icebergTable, icebergTable.currentSnapshot())) { - dataFiles.forEach(deletes::deleteFile); + dataFiles.forEach(overwriteFiles::deleteFile); } - deletes.commit(); + appendFiles.forEach(overwriteFiles::addFile); + overwriteFiles.commit(); // Update the spec and schema of the existing table. // If we have already verified the schema, we don't need to update it. @@ -1257,14 +1286,14 @@ private static void commit( updateSpec.commit(); } } + } else { + // Append the new data files to the table + final AppendFiles append = icebergTransaction.newAppend(); + appendFiles.forEach(append::appendFile); + append.commit(); } - // Append the new data files to the table - final AppendFiles append = icebergTransaction.newAppend(); - appendFiles.forEach(append::appendFile); - append.commit(); - - // Commit the transaction, creating new separate snapshots for "delete" and "append" operation. + // Commit the transaction, creating new snapshot for append/overwrite. // Note that no new snapshot will be created for the schema change. icebergTransaction.commitTransaction(); } @@ -1274,9 +1303,6 @@ private static void commit( */ private static List dataFilesFromParquet( @NotNull final Collection parquetFilesWritten) { - if (parquetFilesWritten.isEmpty()) { - throw new UncheckedDeephavenException("Failed to generate data files because no parquet files written"); - } // TODO This assumes no partition data is written, is that okay? return parquetFilesWritten.stream() .map(parquetFileWritten -> DataFiles.builder(PartitionSpec.unpartitioned()) diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java index ba30df88eb4..08a369626e2 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java @@ -79,17 +79,20 @@ void appendTableBasicTest() { catalogAdapter.append(tableIdentifier, source, writeInstructions); Table fromIceberg = catalogAdapter.readTable(tableIdentifier, null); assertTableEquals(source, fromIceberg); - assertThat(catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots()).hasSize(1); + verifySnapshots(tableIdentifier, List.of("append")); - // Append more data + // 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"); - catalogAdapter.append(tableIdentifier, moreData, writeInstructions); + final IcebergWriteInstructions writeInstructionsLZ4 = IcebergParquetWriteInstructions.builder() + .compressionCodecName("LZ4") + .build(); + catalogAdapter.append(tableIdentifier, moreData, writeInstructionsLZ4); fromIceberg = catalogAdapter.readTable(tableIdentifier, null); final Table expected = TableTools.merge(moreData, source); assertTableEquals(expected, fromIceberg); - assertThat(catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots()).hasSize(2); + verifySnapshots(tableIdentifier, List.of("append", "append")); // Append an empty table final Table emptyTable = TableTools.emptyTable(0) @@ -98,18 +101,21 @@ void appendTableBasicTest() { catalogAdapter.append(tableIdentifier, emptyTable, writeInstructions); fromIceberg = catalogAdapter.readTable(tableIdentifier, null); assertTableEquals(expected, fromIceberg); - assertThat(catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots()).hasSize(3); + verifySnapshots(tableIdentifier, List.of("append", "append", "append")); - // Append multiple tables in a single call + // 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 IcebergWriteInstructions writeInstructionsGZIP = IcebergParquetWriteInstructions.builder() + .compressionCodecName("GZIP") + .build(); catalogAdapter.append(tableIdentifier, new Table[] {someMoreData, moreData, emptyTable}, - writeInstructions); + writeInstructionsGZIP); fromIceberg = catalogAdapter.readTable(tableIdentifier, null); final Table expected2 = TableTools.merge(someMoreData, moreData, expected); assertTableEquals(expected2, fromIceberg); - assertThat(catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots()).hasSize(4); + verifySnapshots(tableIdentifier, List.of("append", "append", "append", "append")); } @Test @@ -129,12 +135,7 @@ void overwriteTablesBasicTest() { catalogAdapter.overwrite(tableIdentifier, source, writeInstructions); Table fromIceberg = catalogAdapter.readTable(tableIdentifier, null); assertTableEquals(source, fromIceberg); - { - final Iterable snapshots = - catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots(); - assertThat(snapshots).hasSize(1); - assertThat(snapshots.iterator().next().operation()).isEqualTo("append"); - } + verifySnapshots(tableIdentifier, List.of("append")); // Overwrite with more data final Table moreData = TableTools.emptyTable(5) @@ -143,16 +144,7 @@ void overwriteTablesBasicTest() { catalogAdapter.overwrite(tableIdentifier, moreData, writeInstructions); fromIceberg = catalogAdapter.readTable(tableIdentifier, null); assertTableEquals(moreData, fromIceberg); - // 2 snapshots added, one for the delete and other for append - { - final Iterable snapshots = - catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots(); - assertThat(snapshots).hasSize(3); - final Iterator snapshotIter = snapshots.iterator(); - assertThat(snapshotIter.next().operation()).isEqualTo("append"); - assertThat(snapshotIter.next().operation()).isEqualTo("delete"); - assertThat(snapshotIter.next().operation()).isEqualTo("append"); - } + verifySnapshots(tableIdentifier, List.of("append", "overwrite")); // Overwrite with an empty table final Table emptyTable = TableTools.emptyTable(0) @@ -161,7 +153,7 @@ void overwriteTablesBasicTest() { catalogAdapter.overwrite(tableIdentifier, emptyTable, writeInstructions); fromIceberg = catalogAdapter.readTable(tableIdentifier, null); assertTableEquals(emptyTable, fromIceberg); - assertThat(catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots()).hasSize(5); + verifySnapshots(tableIdentifier, List.of("append", "overwrite", "overwrite")); // Overwrite with multiple tables in a single call final Table someMoreData = TableTools.emptyTable(3) @@ -172,7 +164,17 @@ void overwriteTablesBasicTest() { fromIceberg = catalogAdapter.readTable(tableIdentifier, null); final Table expected2 = TableTools.merge(someMoreData, moreData); assertTableEquals(expected2, fromIceberg); - assertThat(catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots()).hasSize(7); + verifySnapshots(tableIdentifier, List.of("append", "overwrite", "overwrite", "overwrite")); + } + + private void verifySnapshots(final String tableIdentifier, final List expectedOperations) { + final Iterable snapshots = + catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots(); + assertThat(snapshots).hasSize(expectedOperations.size()); + final Iterator snapshotIter = snapshots.iterator(); + for (final String expectedOperation : expectedOperations) { + assertThat(snapshotIter.next().operation()).isEqualTo(expectedOperation); + } } @Test @@ -185,10 +187,13 @@ void overwriteWithDifferentDefinition() { .createTableIfNotExist(true) .verifySchema(true) .build(); - catalogAdapter.append(tableIdentifier, source, writeInstructionsWithSchemaMatching); - Table fromIceberg = catalogAdapter.readTable(tableIdentifier, null); - assertTableEquals(source, fromIceberg); - assertThat(catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots()).hasSize(1); + + { + catalogAdapter.append(tableIdentifier, source, writeInstructionsWithSchemaMatching); + final Table fromIceberg = catalogAdapter.readTable(tableIdentifier, null); + assertTableEquals(source, fromIceberg); + verifySnapshots(tableIdentifier, List.of("append")); + } final Table differentSource = TableTools.emptyTable(10) .update("intCol = (int) 2 * i + 10"); @@ -201,19 +206,32 @@ void overwriteWithDifferentDefinition() { // By default, schema verification should be disabled for overwriting final IcebergWriteInstructions writeInstructionsWithoutSchemaMatching = IcebergParquetWriteInstructions.builder().build(); - catalogAdapter.overwrite(tableIdentifier, differentSource, writeInstructionsWithoutSchemaMatching); - fromIceberg = catalogAdapter.readTable(tableIdentifier, null); - assertTableEquals(differentSource, fromIceberg); - assertThat(catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots()).hasSize(3); + { + catalogAdapter.overwrite(tableIdentifier, differentSource, writeInstructionsWithoutSchemaMatching); + final Table fromIceberg = catalogAdapter.readTable(tableIdentifier, null); + assertTableEquals(differentSource, fromIceberg); + verifySnapshots(tableIdentifier, List.of("append", "overwrite")); + } // Append more data to this table with the updated schema - final Table moreData = TableTools.emptyTable(5) - .update("intCol = (int) 3 * i + 20"); - catalogAdapter.append(tableIdentifier, moreData, writeInstructionsWithoutSchemaMatching); - fromIceberg = catalogAdapter.readTable(tableIdentifier, null); - final Table expected = TableTools.merge(moreData, differentSource); - assertTableEquals(expected, fromIceberg); - assertThat(catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots()).hasSize(4); + { + final Table moreData = TableTools.emptyTable(5) + .update("intCol = (int) 3 * i + 20"); + catalogAdapter.append(tableIdentifier, moreData, writeInstructionsWithoutSchemaMatching); + final Table fromIceberg = catalogAdapter.readTable(tableIdentifier, null); + final Table expected = TableTools.merge(moreData, differentSource); + assertTableEquals(expected, fromIceberg); + verifySnapshots(tableIdentifier, List.of("append", "overwrite", "append")); + + } + + // Overwrite with an empty list + { + catalogAdapter.overwrite(tableIdentifier, new Table[] {}, writeInstructionsWithoutSchemaMatching); + final Table fromIceberg = catalogAdapter.readTable(tableIdentifier, null); + assertTableEquals(TableTools.emptyTable(0), fromIceberg); + verifySnapshots(tableIdentifier, List.of("append", "overwrite", "append", "overwrite")); + } } @Test @@ -230,7 +248,7 @@ void appendWithDifferentDefinition() { catalogAdapter.append(tableIdentifier, source, writeInstructions); Table fromIceberg = catalogAdapter.readTable(tableIdentifier, null); assertTableEquals(source, fromIceberg); - assertThat(catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots()).hasSize(1); + verifySnapshots(tableIdentifier, List.of("append")); final Table differentSource = TableTools.emptyTable(10) .update("shortCol = (short) 2 * i + 10"); @@ -247,7 +265,7 @@ void appendWithDifferentDefinition() { fromIceberg = catalogAdapter.readTable(tableIdentifier, null); final Table expected = TableTools.merge(compatibleSource.update("doubleCol = NULL_DOUBLE"), source); assertTableEquals(expected, fromIceberg); - assertThat(catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots()).hasSize(2); + verifySnapshots(tableIdentifier, List.of("append", "append")); // Append more data final Table moreData = TableTools.emptyTable(5) @@ -257,7 +275,13 @@ void appendWithDifferentDefinition() { fromIceberg = catalogAdapter.readTable(tableIdentifier, null); final Table expected2 = TableTools.merge(moreData, expected); assertTableEquals(expected2, fromIceberg); - assertThat(catalogAdapter.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).snapshots()).hasSize(3); + verifySnapshots(tableIdentifier, List.of("append", "append", "append")); + + // Append an empty list + catalogAdapter.append(tableIdentifier, new Table[] {}, writeInstructions); + fromIceberg = catalogAdapter.readTable(tableIdentifier, null); + assertTableEquals(expected2, fromIceberg); + verifySnapshots(tableIdentifier, List.of("append", "append", "append")); } @Test From 31f46bad3875cef50d12ebbeb5b19dce58e40669 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Fri, 18 Oct 2024 11:18:43 -0500 Subject: [PATCH 24/28] Review with Ryan Part 2 --- .../deephaven/iceberg/util/IcebergAppend.java | 51 ++++ .../iceberg/util/IcebergCatalogAdapter.java | 240 ++++-------------- .../iceberg/util/IcebergOverwrite.java | 51 ++++ .../iceberg/util/IcebergWriteDataFiles.java | 51 ++++ .../deephaven/iceberg/CatalogAdapterTest.java | 191 +++++++++++--- 5 files changed, 346 insertions(+), 238 deletions(-) create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergAppend.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergOverwrite.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteDataFiles.java diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergAppend.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergAppend.java new file mode 100644 index 00000000000..07e079f816f --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergAppend.java @@ -0,0 +1,51 @@ +// +// 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 java.util.List; + +@Value.Immutable +@BuildableStyle +public abstract class IcebergAppend { + /** + * The identifier string for the Iceberg table to append to + */ + public abstract String tableIdentifier(); + + /** + * The Deephaven tables to append. All tables should have the same definition, else a table definition should be + * provided in the {@link #instructions()}. + */ + public abstract List

dhTables(); + + /** + * The instructions for customizations while writing, defaults to {@link IcebergParquetWriteInstructions#DEFAULT}. + */ + @Value.Default + public IcebergWriteInstructions instructions() { + return IcebergParquetWriteInstructions.DEFAULT; + } + + public static Builder builder() { + return ImmutableIcebergAppend.builder(); + } + + public interface Builder { + Builder tableIdentifier(String tableIdentifier); + + Builder addDhTables(Table element); + + Builder addDhTables(Table... elements); + + Builder addAllDhTables(Iterable elements); + + Builder instructions(IcebergWriteInstructions instructions); + + IcebergAppend build(); + } +} 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 f0647b93dca..ab96ae61b2e 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 @@ -844,202 +844,45 @@ public Catalog catalog() { * Append the provided Deephaven table as a new partition to the existing Iceberg table in a single snapshot. This * will not change the schema of the existing table. * - * @param tableIdentifier The identifier string for the Iceberg table to append to - * @param dhTable The Deephaven table to append - * @param instructions The instructions for customizations while writing, or null to use default instructions - */ - public void append( - @NotNull final String tableIdentifier, - @NotNull final Table dhTable, - @Nullable final IcebergWriteInstructions instructions) { - append(tableIdentifier, new Table[] {dhTable}, instructions); - } - - /** - * Append the provided Deephaven table as a new partition to the existing Iceberg table in a single snapshot. This - * will not change the schema of the existing table. - * - * @param tableIdentifier The identifier for the Iceberg table to append to - * @param dhTable The Deephaven table to append - * @param instructions The instructions for customizations while writing, or null to use default instructions + * @param icebergAppend The {@link IcebergAppend} object containing the data/instructions for writing */ - public void append( - @NotNull final TableIdentifier tableIdentifier, - @NotNull final Table dhTable, - @Nullable final IcebergWriteInstructions instructions) { - append(tableIdentifier, new Table[] {dhTable}, instructions); - } - - /** - * Append the provided Deephaven tables as new partitions to the existing Iceberg table in a single snapshot. All - * tables should have the same definition, else a table definition should be provided in the instructions. This will - * not change the schema of the existing table. - * - * @param tableIdentifier The identifier string for the Iceberg table to append to - * @param dhTables The Deephaven tables to append - * @param instructions The instructions for customizations while writing, or null to use default instructions - */ - public void append( - @NotNull final String tableIdentifier, - @NotNull final Table[] dhTables, - @Nullable final IcebergWriteInstructions instructions) { - append(TableIdentifier.parse(tableIdentifier), dhTables, instructions); + public void append(@NotNull final IcebergAppend icebergAppend) { + writeImpl(TableIdentifier.parse(icebergAppend.tableIdentifier()), + icebergAppend.dhTables(), + icebergAppend.instructions(), + false, + true); } /** - * Append the provided Deephaven tables as new partitions to the existing Iceberg table in a single snapshot. All - * tables should have the same definition, else a table definition should be provided in the instructions. This will - * not change the schema of the existing table. - * - * @param tableIdentifier The identifier for the Iceberg table to append to - * @param dhTables The Deephaven tables to append - * @param instructions The instructions for customizations while writing, or null to use default instructions - */ - public void append( - @NotNull final TableIdentifier tableIdentifier, - @NotNull final Table[] dhTables, - @Nullable final IcebergWriteInstructions instructions) { - writeImpl(tableIdentifier, dhTables, instructions, false, true); - } - - /** - * Overwrite the existing Iceberg table with the provided Deephaven table in a single snapshot. This will overwrite + * Overwrite the existing Iceberg table with the provided Deephaven tables in a single snapshot. This will overwrite * the schema of the existing table to match the provided Deephaven table if they do not match. *

- * Overwriting a table while racing with other writers can lead to undefined results. + * Overwriting a table while racing with other writers can lead to failure/undefined results. * - * @param tableIdentifier The identifier string for the Iceberg table to overwrite - * @param dhTable The Deephaven table to overwrite with; an empty array will overwrite with an empty table - * @param instructions The instructions for customizations while writing, or null to use default instructions + * @param icebergOverwrite The {@link IcebergOverwrite} object containing the data/instructions for writing */ - public void overwrite( - @NotNull final String tableIdentifier, - @NotNull final Table dhTable, - @Nullable final IcebergWriteInstructions instructions) { - overwrite(tableIdentifier, new Table[] {dhTable}, instructions); - } - - /** - * Overwrite the existing Iceberg table with the provided Deephaven table in a single snapshot. This will overwrite - * the schema of the existing table to match the provided Deephaven table if they do not match. - *

- * Overwriting a table while racing with other writers can lead to undefined results. - * - * @param tableIdentifier The identifier for the Iceberg table to overwrite - * @param dhTable The Deephaven table to overwrite with; an empty array will overwrite with an empty table - * @param instructions The instructions for customizations while writing, or null to use default instructions - */ - public void overwrite( - @NotNull final TableIdentifier tableIdentifier, - @NotNull final Table dhTable, - @Nullable final IcebergWriteInstructions instructions) { - overwrite(tableIdentifier, new Table[] {dhTable}, instructions); - } - - /** - * Overwrite the existing Iceberg table with the provided Deephaven table in a single snapshot. All tables should - * have the same definition, else a table definition should be provided in the instructions. This will overwrite the - * schema of the existing table to match the provided Deephaven table if they do not match. - *

- * Overwriting a table while racing with other writers can lead to undefined results. - * - * @param tableIdentifier The identifier string for the Iceberg table to overwrite - * @param dhTables The Deephaven tables to overwrite with; an empty array will overwrite with an empty table - * @param instructions The instructions for customizations while writing, or null to use default instructions - */ - public void overwrite( - @NotNull final String tableIdentifier, - @NotNull final Table[] dhTables, - @Nullable final IcebergWriteInstructions instructions) { - overwrite(TableIdentifier.parse(tableIdentifier), dhTables, instructions); - } - - /** - * Overwrite the existing Iceberg table with the provided Deephaven tables in a single snapshot. All tables should - * have the same definition, else a table definition should be provided in the instructions. This will overwrite the - * schema of the existing table to match the provided Deephaven table if they do not match. - *

- * Overwriting a table while racing with other writers can lead to undefined results. - * - * @param tableIdentifier The identifier for the Iceberg table to overwrite - * @param dhTables The Deephaven tables to overwrite with; an empty array will overwrite with an empty table - * @param instructions The instructions for customizations while writing, or null to use default instructions - */ - public void overwrite( - @NotNull final TableIdentifier tableIdentifier, - @NotNull final Table[] dhTables, - @Nullable final IcebergWriteInstructions instructions) { - writeImpl(tableIdentifier, dhTables, instructions, true, true); - } - - /** - * Writes data from a Deephaven table 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. - * - * @param tableIdentifier The identifier string for the Iceberg table to write to. - * @param dhTable The Deephaven table containing the data to be written. - * @param instructions The instructions for customizations while writing, or null to use default instructions. - * - * @return A list of {@link DataFile} objects representing the written data files. - */ - public List writeDataFiles( - @NotNull final String tableIdentifier, - @NotNull final Table dhTable, - @Nullable final IcebergWriteInstructions instructions) { - return writeDataFiles(tableIdentifier, new Table[] {dhTable}, instructions); - } - - /** - * Writes data from a Deephaven table 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. - * - * @param tableIdentifier The identifier for the Iceberg table to write to. - * @param dhTable The Deephaven table containing the data to be written. - * @param instructions The instructions for customizations while writing, or null to use default instructions. - * - * @return A list of {@link DataFile} objects representing the written data files. - */ - public List writeDataFiles( - @NotNull final TableIdentifier tableIdentifier, - @NotNull final Table dhTable, - @Nullable final IcebergWriteInstructions instructions) { - return writeDataFiles(tableIdentifier, new Table[] {dhTable}, instructions); + public void overwrite(@NotNull final IcebergOverwrite icebergOverwrite) { + writeImpl(TableIdentifier.parse(icebergOverwrite.tableIdentifier()), + icebergOverwrite.dhTables(), + icebergOverwrite.instructions(), + true, + true); } /** * Writes data from Deephaven 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. All tables - * should have the same definition, else a table definition should be provided in the instructions. + * of data files that were written. Users can use this list to create a transaction/snapshot if needed. * - * @param tableIdentifier The identifier string for the Iceberg table to write to. - * @param dhTables The Deephaven tables containing the data to be written. - * @param instructions The instructions for customizations while writing, or null to use default instructions. - * - * @return A list of {@link DataFile} objects representing the written data files. + * @param icebergWriteDataFiles The {@link IcebergWriteDataFiles} object containing the data/instructions for + * writing */ - public List writeDataFiles( - @NotNull final String tableIdentifier, - @NotNull final Table[] dhTables, - @Nullable final IcebergWriteInstructions instructions) { - return writeDataFiles(TableIdentifier.parse(tableIdentifier), dhTables, instructions); - } - - /** - * Writes data from Deephaven 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. All tables - * should have the same definition, else a table definition should be provided in the instructions. - * - * @param tableIdentifier The identifier for the Iceberg table to write to. - * @param dhTables The Deephaven tables containing the data to be written. - * @param instructions The instructions for customizations while writing, or null to use default instructions. - * - * @return A list of {@link DataFile} objects representing the written data files. - */ - public List writeDataFiles( - @NotNull final TableIdentifier tableIdentifier, - @NotNull final Table[] dhTables, - @Nullable final IcebergWriteInstructions instructions) { - return writeImpl(tableIdentifier, dhTables, instructions, false, false); + public List writeDataFiles(@NotNull final IcebergWriteDataFiles icebergWriteDataFiles) { + return writeImpl(TableIdentifier.parse(icebergWriteDataFiles.tableIdentifier()), + icebergWriteDataFiles.dhTables(), + icebergWriteDataFiles.instructions(), + false, + false); } /** @@ -1056,20 +899,20 @@ public List writeDataFiles( */ private List writeImpl( @NotNull final TableIdentifier tableIdentifier, - @NotNull Table[] dhTables, - @Nullable final IcebergWriteInstructions instructions, + @NotNull List

dhTables, + @NotNull final IcebergWriteInstructions instructions, final boolean overwrite, final boolean addSnapshot) { if (overwrite && !addSnapshot) { throw new IllegalArgumentException("Cannot overwrite an Iceberg table without adding a snapshot"); } - if (dhTables.length == 0) { + if (dhTables.isEmpty()) { if (!overwrite) { // Nothing to append return Collections.emptyList(); } // Overwrite with an empty table - dhTables = new Table[] {TableTools.emptyTable(0)}; + dhTables = List.of(TableTools.emptyTable(0)); } IcebergParquetWriteInstructions writeInstructions = verifyWriteInstructions(instructions); @@ -1082,9 +925,10 @@ private List writeImpl( useDefinition = writeInstructions.tableDefinition().get(); } else { // Verify that all tables have the same definition - final TableDefinition firstDefinition = dhTables[0].getDefinition(); - for (int idx = 1; idx < dhTables.length; idx++) { - if (!firstDefinition.equals(dhTables[idx].getDefinition())) { + final TableDefinition firstDefinition = dhTables.get(0).getDefinition(); + final int numTables = dhTables.size(); + for (int idx = 1; idx < numTables; idx++) { + if (!firstDefinition.equals(dhTables.get(idx).getDefinition())) { throw new IllegalArgumentException( "All Deephaven tables must have the same definition, else table definition should be " + "provided when writing multiple tables with different definitions"); @@ -1160,10 +1004,7 @@ private List writeImpl( } private static IcebergParquetWriteInstructions verifyWriteInstructions( - @Nullable final IcebergWriteInstructions instructions) { - if (instructions == null) { - return IcebergParquetWriteInstructions.DEFAULT; - } + @NotNull final IcebergWriteInstructions instructions) { // We ony support writing to Parquet files if (!(instructions instanceof IcebergParquetWriteInstructions)) { throw new IllegalArgumentException("Unsupported instructions of class " + instructions.getClass() + " for" + @@ -1220,10 +1061,10 @@ private CompletedParquetWrite(final URI destination, final long numRows, final l @NotNull private static List writeParquet( @NotNull final org.apache.iceberg.Table icebergTable, - @NotNull final Table[] dhTables, + @NotNull final Collection
dhTables, @NotNull final IcebergParquetWriteInstructions writeInstructions) { // Build the parquet instructions - final List parquetFilesWritten = new ArrayList<>(dhTables.length); + final List parquetFilesWritten = new ArrayList<>(dhTables.size()); final ParquetInstructions.OnWriteCompleted onWriteCompleted = (destination, numRows, numBytes) -> parquetFilesWritten .add(new CompletedParquetWrite(destination, numRows, numBytes)); @@ -1258,10 +1099,15 @@ private static void commit( final boolean schemaVerified) { final Transaction icebergTransaction = icebergTable.newTransaction(); if (overwrite) { - final OverwriteFiles overwriteFiles = icebergTransaction.newOverwrite(); + // Fail if the table gets changed concurrently + final Snapshot currentSnapshot = icebergTable.currentSnapshot(); + final OverwriteFiles overwriteFiles = icebergTransaction.newOverwrite() + .validateFromSnapshot(currentSnapshot.snapshotId()) + .validateNoConflictingDeletes() + .validateNoConflictingData(); // Delete all the existing data files in the table - try (final Stream dataFiles = allDataFiles(icebergTable, icebergTable.currentSnapshot())) { + try (final Stream dataFiles = allDataFiles(icebergTable, currentSnapshot)) { dataFiles.forEach(overwriteFiles::deleteFile); } appendFiles.forEach(overwriteFiles::addFile); diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergOverwrite.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergOverwrite.java new file mode 100644 index 00000000000..4af3224dfe3 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergOverwrite.java @@ -0,0 +1,51 @@ +// +// 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 java.util.List; + +@Value.Immutable +@BuildableStyle +public abstract class IcebergOverwrite { + /** + * The identifier string for the Iceberg table to overwrite + */ + public abstract String tableIdentifier(); + + /** + * The Deephaven tables to overwrite with. All tables should have the same definition, else a table definition + * should be provided in the {@link #instructions()}. An empty list will overwrite with an empty table. + */ + public abstract List
dhTables(); + + /** + * The instructions for customizations while writing, defaults to {@link IcebergParquetWriteInstructions#DEFAULT}. + */ + @Value.Default + public IcebergWriteInstructions instructions() { + return IcebergParquetWriteInstructions.DEFAULT; + } + + public static Builder builder() { + return ImmutableIcebergOverwrite.builder(); + } + + public interface Builder { + Builder tableIdentifier(String tableIdentifier); + + Builder addDhTables(Table element); + + Builder addDhTables(Table... elements); + + Builder addAllDhTables(Iterable elements); + + Builder instructions(IcebergWriteInstructions instructions); + + IcebergOverwrite build(); + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteDataFiles.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteDataFiles.java new file mode 100644 index 00000000000..494c8f2073f --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteDataFiles.java @@ -0,0 +1,51 @@ +// +// 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 java.util.List; + +@Value.Immutable +@BuildableStyle +public abstract class IcebergWriteDataFiles { + /** + * The identifier string for the Iceberg table to write to. + */ + public abstract String tableIdentifier(); + + /** + * The Deephaven tables to be written. All tables should have the same definition, else a table definition should be + * provided in the {@link #instructions()}. + */ + public abstract List
dhTables(); + + /** + * The instructions for customizations while writing, defaults to {@link IcebergParquetWriteInstructions#DEFAULT}. + */ + @Value.Default + public IcebergWriteInstructions instructions() { + return IcebergParquetWriteInstructions.DEFAULT; + } + + public static Builder builder() { + return ImmutableIcebergWriteDataFiles.builder(); + } + + public interface Builder { + Builder tableIdentifier(String tableIdentifier); + + Builder addDhTables(Table element); + + Builder addDhTables(Table... elements); + + Builder addAllDhTables(Iterable elements); + + Builder instructions(IcebergWriteInstructions instructions); + + IcebergWriteDataFiles build(); + } +} diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java index 08a369626e2..668a4efcaf0 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java @@ -11,16 +11,18 @@ import io.deephaven.engine.util.TableTools; import io.deephaven.iceberg.base.IcebergUtils; import io.deephaven.iceberg.junit5.CatalogAdapterBase; +import io.deephaven.iceberg.util.IcebergAppend; +import io.deephaven.iceberg.util.IcebergOverwrite; import io.deephaven.iceberg.util.IcebergParquetWriteInstructions; import io.deephaven.iceberg.util.IcebergWriteInstructions; import io.deephaven.parquet.table.ParquetTools; -import junit.framework.TestCase; 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.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import java.util.Iterator; @@ -69,14 +71,21 @@ void appendTableBasicTest() { "doubleCol = (double) 2.5 * i + 10"); final String tableIdentifier = "MyNamespace.MyTable"; try { - catalogAdapter.append(tableIdentifier, source, null); + catalogAdapter.append(IcebergAppend.builder() + .tableIdentifier(tableIdentifier) + .addDhTables(source) + .build()); } catch (RuntimeException e) { assertThat(e.getMessage()).contains("Table does not exist"); } final IcebergWriteInstructions writeInstructions = IcebergParquetWriteInstructions.builder() .createTableIfNotExist(true) .build(); - catalogAdapter.append(tableIdentifier, source, writeInstructions); + catalogAdapter.append(IcebergAppend.builder() + .tableIdentifier(tableIdentifier) + .addDhTables(source) + .instructions(writeInstructions) + .build()); Table fromIceberg = catalogAdapter.readTable(tableIdentifier, null); assertTableEquals(source, fromIceberg); verifySnapshots(tableIdentifier, List.of("append")); @@ -88,7 +97,11 @@ void appendTableBasicTest() { final IcebergWriteInstructions writeInstructionsLZ4 = IcebergParquetWriteInstructions.builder() .compressionCodecName("LZ4") .build(); - catalogAdapter.append(tableIdentifier, moreData, writeInstructionsLZ4); + catalogAdapter.append(IcebergAppend.builder() + .tableIdentifier(tableIdentifier) + .addDhTables(moreData) + .instructions(writeInstructionsLZ4) + .build()); fromIceberg = catalogAdapter.readTable(tableIdentifier, null); final Table expected = TableTools.merge(moreData, source); assertTableEquals(expected, fromIceberg); @@ -98,7 +111,11 @@ void appendTableBasicTest() { final Table emptyTable = TableTools.emptyTable(0) .update("intCol = (int) 4 * i + 30", "doubleCol = (double) 4.5 * i + 30"); - catalogAdapter.append(tableIdentifier, emptyTable, writeInstructions); + catalogAdapter.append(IcebergAppend.builder() + .tableIdentifier(tableIdentifier) + .addDhTables(emptyTable) + .instructions(writeInstructions) + .build()); fromIceberg = catalogAdapter.readTable(tableIdentifier, null); assertTableEquals(expected, fromIceberg); verifySnapshots(tableIdentifier, List.of("append", "append", "append")); @@ -110,8 +127,11 @@ void appendTableBasicTest() { final IcebergWriteInstructions writeInstructionsGZIP = IcebergParquetWriteInstructions.builder() .compressionCodecName("GZIP") .build(); - catalogAdapter.append(tableIdentifier, new Table[] {someMoreData, moreData, emptyTable}, - writeInstructionsGZIP); + catalogAdapter.append(IcebergAppend.builder() + .tableIdentifier(tableIdentifier) + .addDhTables(someMoreData, moreData, emptyTable) + .instructions(writeInstructionsGZIP) + .build()); fromIceberg = catalogAdapter.readTable(tableIdentifier, null); final Table expected2 = TableTools.merge(someMoreData, moreData, expected); assertTableEquals(expected2, fromIceberg); @@ -125,14 +145,21 @@ void overwriteTablesBasicTest() { "doubleCol = (double) 2.5 * i + 10"); final String tableIdentifier = "MyNamespace.MyTable"; try { - catalogAdapter.overwrite(tableIdentifier, source, null); + catalogAdapter.overwrite(IcebergOverwrite.builder() + .tableIdentifier(tableIdentifier) + .addDhTables(source) + .build()); } catch (RuntimeException e) { assertThat(e.getMessage()).contains("Table does not exist"); } final IcebergWriteInstructions writeInstructions = IcebergParquetWriteInstructions.builder() .createTableIfNotExist(true) .build(); - catalogAdapter.overwrite(tableIdentifier, source, writeInstructions); + catalogAdapter.overwrite(IcebergOverwrite.builder() + .tableIdentifier(tableIdentifier) + .addDhTables(source) + .instructions(writeInstructions) + .build()); Table fromIceberg = catalogAdapter.readTable(tableIdentifier, null); assertTableEquals(source, fromIceberg); verifySnapshots(tableIdentifier, List.of("append")); @@ -141,7 +168,11 @@ void overwriteTablesBasicTest() { final Table moreData = TableTools.emptyTable(5) .update("intCol = (int) 3 * i + 20", "doubleCol = (double) 3.5 * i + 20"); - catalogAdapter.overwrite(tableIdentifier, moreData, writeInstructions); + catalogAdapter.overwrite(IcebergOverwrite.builder() + .tableIdentifier(tableIdentifier) + .addDhTables(moreData) + .instructions(writeInstructions) + .build()); fromIceberg = catalogAdapter.readTable(tableIdentifier, null); assertTableEquals(moreData, fromIceberg); verifySnapshots(tableIdentifier, List.of("append", "overwrite")); @@ -150,7 +181,11 @@ void overwriteTablesBasicTest() { final Table emptyTable = TableTools.emptyTable(0) .update("intCol = (int) 4 * i + 30", "doubleCol = (double) 4.5 * i + 30"); - catalogAdapter.overwrite(tableIdentifier, emptyTable, writeInstructions); + catalogAdapter.overwrite(IcebergOverwrite.builder() + .tableIdentifier(tableIdentifier) + .addDhTables(emptyTable) + .instructions(writeInstructions) + .build()); fromIceberg = catalogAdapter.readTable(tableIdentifier, null); assertTableEquals(emptyTable, fromIceberg); verifySnapshots(tableIdentifier, List.of("append", "overwrite", "overwrite")); @@ -159,8 +194,11 @@ void overwriteTablesBasicTest() { final Table someMoreData = TableTools.emptyTable(3) .update("intCol = (int) 5 * i + 40", "doubleCol = (double) 5.5 * i + 40"); - catalogAdapter.overwrite(tableIdentifier, new Table[] {someMoreData, moreData, emptyTable}, - writeInstructions); + catalogAdapter.overwrite(IcebergOverwrite.builder() + .tableIdentifier(tableIdentifier) + .addDhTables(someMoreData, moreData, emptyTable) + .instructions(writeInstructions) + .build()); fromIceberg = catalogAdapter.readTable(tableIdentifier, null); final Table expected2 = TableTools.merge(someMoreData, moreData); assertTableEquals(expected2, fromIceberg); @@ -189,7 +227,11 @@ void overwriteWithDifferentDefinition() { .build(); { - catalogAdapter.append(tableIdentifier, source, writeInstructionsWithSchemaMatching); + catalogAdapter.append(IcebergAppend.builder() + .tableIdentifier(tableIdentifier) + .addDhTables(source) + .instructions(writeInstructionsWithSchemaMatching) + .build()); final Table fromIceberg = catalogAdapter.readTable(tableIdentifier, null); assertTableEquals(source, fromIceberg); verifySnapshots(tableIdentifier, List.of("append")); @@ -198,7 +240,11 @@ void overwriteWithDifferentDefinition() { final Table differentSource = TableTools.emptyTable(10) .update("intCol = (int) 2 * i + 10"); try { - catalogAdapter.overwrite(tableIdentifier, differentSource, writeInstructionsWithSchemaMatching); + catalogAdapter.overwrite(IcebergOverwrite.builder() + .tableIdentifier(tableIdentifier) + .addDhTables(differentSource) + .instructions(writeInstructionsWithSchemaMatching) + .build()); } catch (RuntimeException e) { assertThat(e.getMessage()).contains("Schema verification failed"); } @@ -207,7 +253,11 @@ void overwriteWithDifferentDefinition() { final IcebergWriteInstructions writeInstructionsWithoutSchemaMatching = IcebergParquetWriteInstructions.builder().build(); { - catalogAdapter.overwrite(tableIdentifier, differentSource, writeInstructionsWithoutSchemaMatching); + catalogAdapter.overwrite(IcebergOverwrite.builder() + .tableIdentifier(tableIdentifier) + .addDhTables(differentSource) + .instructions(writeInstructionsWithoutSchemaMatching) + .build()); final Table fromIceberg = catalogAdapter.readTable(tableIdentifier, null); assertTableEquals(differentSource, fromIceberg); verifySnapshots(tableIdentifier, List.of("append", "overwrite")); @@ -217,17 +267,23 @@ void overwriteWithDifferentDefinition() { { final Table moreData = TableTools.emptyTable(5) .update("intCol = (int) 3 * i + 20"); - catalogAdapter.append(tableIdentifier, moreData, writeInstructionsWithoutSchemaMatching); + catalogAdapter.append(IcebergAppend.builder() + .tableIdentifier(tableIdentifier) + .addDhTables(moreData) + .instructions(writeInstructionsWithoutSchemaMatching) + .build()); final Table fromIceberg = catalogAdapter.readTable(tableIdentifier, null); final Table expected = TableTools.merge(moreData, differentSource); assertTableEquals(expected, fromIceberg); verifySnapshots(tableIdentifier, List.of("append", "overwrite", "append")); - } // Overwrite with an empty list { - catalogAdapter.overwrite(tableIdentifier, new Table[] {}, writeInstructionsWithoutSchemaMatching); + catalogAdapter.overwrite(IcebergOverwrite.builder() + .tableIdentifier(tableIdentifier) + .instructions(writeInstructionsWithoutSchemaMatching) + .build()); final Table fromIceberg = catalogAdapter.readTable(tableIdentifier, null); assertTableEquals(TableTools.emptyTable(0), fromIceberg); verifySnapshots(tableIdentifier, List.of("append", "overwrite", "append", "overwrite")); @@ -245,7 +301,11 @@ void appendWithDifferentDefinition() { final IcebergWriteInstructions writeInstructions = IcebergParquetWriteInstructions.builder() .createTableIfNotExist(true) .build(); - catalogAdapter.append(tableIdentifier, source, writeInstructions); + catalogAdapter.append(IcebergAppend.builder() + .tableIdentifier(tableIdentifier) + .addDhTables(source) + .instructions(writeInstructions) + .build()); Table fromIceberg = catalogAdapter.readTable(tableIdentifier, null); assertTableEquals(source, fromIceberg); verifySnapshots(tableIdentifier, List.of("append")); @@ -253,7 +313,11 @@ void appendWithDifferentDefinition() { final Table differentSource = TableTools.emptyTable(10) .update("shortCol = (short) 2 * i + 10"); try { - catalogAdapter.append(tableIdentifier, differentSource, writeInstructions); + catalogAdapter.append(IcebergAppend.builder() + .tableIdentifier(tableIdentifier) + .addDhTables(differentSource) + .instructions(writeInstructions) + .build()); } catch (RuntimeException e) { assertThat(e.getMessage()).contains("Schema verification failed"); } @@ -261,7 +325,11 @@ void appendWithDifferentDefinition() { // Append a table with just the int column, should be compatible with the existing schema final Table compatibleSource = TableTools.emptyTable(10) .update("intCol = (int) 5 * i + 10"); - catalogAdapter.append(tableIdentifier, compatibleSource, writeInstructions); + catalogAdapter.append(IcebergAppend.builder() + .tableIdentifier(tableIdentifier) + .addDhTables(compatibleSource) + .instructions(writeInstructions) + .build()); fromIceberg = catalogAdapter.readTable(tableIdentifier, null); final Table expected = TableTools.merge(compatibleSource.update("doubleCol = NULL_DOUBLE"), source); assertTableEquals(expected, fromIceberg); @@ -271,14 +339,21 @@ void appendWithDifferentDefinition() { final Table moreData = TableTools.emptyTable(5) .update("intCol = (int) 3 * i + 20", "doubleCol = (double) 3.5 * i + 20"); - catalogAdapter.append(tableIdentifier, moreData, writeInstructions); + catalogAdapter.append(IcebergAppend.builder() + .tableIdentifier(tableIdentifier) + .addDhTables(moreData) + .instructions(writeInstructions) + .build()); fromIceberg = catalogAdapter.readTable(tableIdentifier, null); final Table expected2 = TableTools.merge(moreData, expected); assertTableEquals(expected2, fromIceberg); verifySnapshots(tableIdentifier, List.of("append", "append", "append")); // Append an empty list - catalogAdapter.append(tableIdentifier, new Table[] {}, writeInstructions); + catalogAdapter.append(IcebergAppend.builder() + .tableIdentifier(tableIdentifier) + .instructions(writeInstructions) + .build()); fromIceberg = catalogAdapter.readTable(tableIdentifier, null); assertTableEquals(expected2, fromIceberg); verifySnapshots(tableIdentifier, List.of("append", "append", "append")); @@ -293,7 +368,11 @@ void appendMultipleTablesWithDefinitionTest() { .createTableIfNotExist(true) .verifySchema(true) .build(); - catalogAdapter.append("MyNamespace.MyTable", source, writeInstructions); + catalogAdapter.append(IcebergAppend.builder() + .tableIdentifier("MyNamespace.MyTable") + .addDhTables(source) + .instructions(writeInstructions) + .build()); Table fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); assertTableEquals(source, fromIceberg); @@ -302,13 +381,16 @@ void appendMultipleTablesWithDefinitionTest() { "doubleCol = (double) 3.5 * i + 20", "shortCol = (short) 3 * i + 20"); final Table appendTable2 = TableTools.emptyTable(5) - .update( - "charCol = (char) 65 + i%26", + .update("charCol = (char) 65 + i % 26", "intCol = (int) 4 * i + 30", "doubleCol = (double) 4.5 * i + 30"); try { - catalogAdapter.append("MyNamespace.MyTable", new Table[] {appendTable1, appendTable2}, writeInstructions); + catalogAdapter.append(IcebergAppend.builder() + .tableIdentifier("MyNamespace.MyTable") + .addDhTables(appendTable1, appendTable2) + .instructions(writeInstructions) + .build()); } catch (RuntimeException e) { assertThat(e.getMessage()).contains("All Deephaven tables must have the same definition"); } @@ -322,8 +404,11 @@ void appendMultipleTablesWithDefinitionTest() { .verifySchema(true) .tableDefinition(writeDefinition) .build(); - catalogAdapter.append("MyNamespace.MyTable", new Table[] {appendTable1, appendTable2}, - writeInstructionsWithDefinition); + catalogAdapter.append(IcebergAppend.builder() + .tableIdentifier("MyNamespace.MyTable") + .addDhTables(appendTable1, appendTable2) + .instructions(writeInstructionsWithDefinition) + .build()); fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); final Table expected = TableTools.merge( appendTable1.dropColumns("shortCol"), @@ -350,7 +435,7 @@ void appendToCatalogTableWithAllDataTypesTest() { final TableIdentifier myTableId = TableIdentifier.of(myNamespace, "MyTableWithAllDataTypes"); catalogAdapter.catalog().createTable(myTableId, schema); - final Table data = TableTools.emptyTable(10) + final Table source = TableTools.emptyTable(10) .update( "booleanCol = i % 2 == 0", "doubleCol = (double) 2.5 * i + 10", @@ -363,9 +448,12 @@ void appendToCatalogTableWithAllDataTypesTest() { "localDateCol = java.time.LocalDate.now()", "localTimeCol = java.time.LocalTime.now()", "binaryCol = new byte[] {(byte) i}"); - catalogAdapter.append(myTableId.toString(), data, null); + catalogAdapter.append(IcebergAppend.builder() + .tableIdentifier(myTableId.toString()) + .addDhTables(source) + .build()); final Table fromIceberg = catalogAdapter.readTable(myTableId, null); - assertTableEquals(data, fromIceberg); + assertTableEquals(source, fromIceberg); } @Test @@ -381,10 +469,15 @@ void testFailureInWrite() { .build(); final Namespace myNamespace = Namespace.of("MyNamespace"); final TableIdentifier myTableId = TableIdentifier.of(myNamespace, "MyTable"); + final String tableIdString = myTableId.toString(); try { - catalogAdapter.append(myTableId, badSource, writeInstructions); - TestCase.fail("Exception expected for invalid formula in table"); + catalogAdapter.append(IcebergAppend.builder() + .tableIdentifier(tableIdString) + .addDhTables(badSource) + .instructions(writeInstructions) + .build()); + Assertions.fail("Exception expected for invalid formula in table"); } catch (UncheckedDeephavenException e) { assertThat(e.getCause() instanceof FormulaEvaluationException).isTrue(); } @@ -394,13 +487,21 @@ void testFailureInWrite() { final Table goodSource = TableTools.emptyTable(5) .update("stringCol = Long.toString(ii)", "intCol = (int) i"); - catalogAdapter.append(myTableId, goodSource, writeInstructions); - Table fromIceberg = catalogAdapter.readTable(myTableId, null); + catalogAdapter.append(IcebergAppend.builder() + .tableIdentifier(tableIdString) + .addDhTables(goodSource) + .instructions(writeInstructions) + .build()); + Table fromIceberg = catalogAdapter.readTable(tableIdString, null); assertTableEquals(goodSource, fromIceberg); try { - catalogAdapter.append(myTableId, badSource, writeInstructions); - TestCase.fail("Exception expected for invalid formula in table"); + catalogAdapter.append(IcebergAppend.builder() + .tableIdentifier(tableIdString) + .addDhTables(badSource) + .instructions(writeInstructions) + .build()); + Assertions.fail("Exception expected for invalid formula in table"); } catch (UncheckedDeephavenException e) { assertThat(e.getCause() instanceof FormulaEvaluationException).isTrue(); } @@ -422,7 +523,11 @@ void testColumnRenameWhileWriting() { .verifySchema(true) .build(); - catalogAdapter.append("MyNamespace.MyTable", source, writeInstructions); + catalogAdapter.append(IcebergAppend.builder() + .tableIdentifier("MyNamespace.MyTable") + .addDhTables(source) + .instructions(writeInstructions) + .build()); // TODO: This is failing because we don't map columns based on the column ID when reading. Uncomment when this // is fixed. // final Table fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); @@ -452,7 +557,11 @@ void testColumnRenameWhileWriting() { .putDhToIcebergColumnRenames("newIntCol", "intCol") .putDhToIcebergColumnRenames("newDoubleCol", "doubleCol") .build(); - catalogAdapter.append("MyNamespace.MyTable", moreData, writeInstructions); + catalogAdapter.append(IcebergAppend.builder() + .tableIdentifier("MyNamespace.MyTable") + .addDhTables(moreData) + .instructions(writeInstructions) + .build()); // Verify that the column names in the parquet file are same as the table written { From 946def05ba36d7ab2742efcb0e0cd223da8b913d Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Fri, 18 Oct 2024 12:41:40 -0500 Subject: [PATCH 25/28] Fix for failing parquet reads --- .../io/deephaven/parquet/table/ParquetTableWriter.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) 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 d9354c636ab..32f749fb825 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 @@ -212,11 +212,13 @@ private static long write( final TrackingRowSet tableRowSet = t.getRowSet(); final Map> columnSourceMap = t.getColumnSourceMap(); final long numBytesWritten; - try (final ParquetFileWriter parquetFileWriter = getParquetFileWriter(computedCache, definition, - tableRowSet, columnSourceMap, dest, destOutputStream, writeInstructions, tableMeta, - tableInfoBuilder, metadataFileWriter)) { + { + 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.getCount(); } destOutputStream.done(); From bd8535cadddf0c98edb2feb7c74af84189781db5 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Fri, 18 Oct 2024 14:15:39 -0500 Subject: [PATCH 26/28] Added more tests for writeDataFile --- .../iceberg/util/IcebergCatalogAdapter.java | 7 +- .../deephaven/iceberg/CatalogAdapterTest.java | 109 +++++++++++++----- 2 files changed, 85 insertions(+), 31 deletions(-) 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 ab96ae61b2e..d790764603c 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 @@ -979,7 +979,7 @@ private List writeImpl( writeParquet(icebergTable, dhTables, writeInstructions); final List appendFiles = dataFilesFromParquet(parquetFileInfo); if (addSnapshot) { - commit(icebergTable, newSpecAndSchema, appendFiles, overwrite && !newTableCreated, verifySchema); + commit(icebergTable, newSpecAndSchema, appendFiles, overwrite, verifySchema); } return appendFiles; } catch (final Throwable throwable) { @@ -1098,9 +1098,10 @@ private static void commit( final boolean overwrite, final boolean schemaVerified) { final Transaction icebergTransaction = icebergTable.newTransaction(); - if (overwrite) { + final Snapshot currentSnapshot = icebergTable.currentSnapshot(); + // For a null current snapshot, we are creating a new table. So we can just append instead of overwriting. + if (overwrite && currentSnapshot != null) { // Fail if the table gets changed concurrently - final Snapshot currentSnapshot = icebergTable.currentSnapshot(); final OverwriteFiles overwriteFiles = icebergTransaction.newOverwrite() .validateFromSnapshot(currentSnapshot.snapshotId()) .validateNoConflictingDeletes() diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java index 668a4efcaf0..bc0e7da12ce 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java @@ -14,8 +14,10 @@ import io.deephaven.iceberg.util.IcebergAppend; import io.deephaven.iceberg.util.IcebergOverwrite; import io.deephaven.iceberg.util.IcebergParquetWriteInstructions; +import io.deephaven.iceberg.util.IcebergWriteDataFiles; import io.deephaven.iceberg.util.IcebergWriteInstructions; import io.deephaven.parquet.table.ParquetTools; +import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; @@ -533,18 +535,8 @@ void testColumnRenameWhileWriting() { // final Table fromIceberg = catalogAdapter.readTable("MyNamespace.MyTable", null); // assertTableEquals(source, fromIceberg); - // Verify that the column names in the parquet file are same as the table written final TableIdentifier tableIdentifier = TableIdentifier.of("MyNamespace", "MyTable"); - final String firstParquetFilePath; - { - final org.apache.iceberg.Table table = catalogAdapter.catalog().loadTable(tableIdentifier); - final List dataFileList = - IcebergUtils.allDataFiles(table, table.currentSnapshot()).collect(Collectors.toList()); - assertThat(dataFileList).hasSize(1); - firstParquetFilePath = dataFileList.get(0).path().toString(); - final Table fromParquet = ParquetTools.readTable(firstParquetFilePath); - assertTableEquals(source, fromParquet); - } + verifyDataFiles(tableIdentifier, List.of(source)); // TODO Verify that the column ID is set correctly after #6156 is merged @@ -563,25 +555,86 @@ void testColumnRenameWhileWriting() { .instructions(writeInstructions) .build()); - // Verify that the column names in the parquet file are same as the table written + // Verify the data files in the table. Note that we are assuming an order here. + verifyDataFiles(tableIdentifier, List.of(moreData, source)); + + // TODO Verify that the column ID is set correctly after #6156 is merged + } + + /** + * 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); + assertTableEquals(dhTable, fromParquet); + } + } + + @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 String tableIdentifier = "MyNamespace.MyTable"; + try { + catalogAdapter.writeDataFiles(IcebergWriteDataFiles.builder() + .tableIdentifier(tableIdentifier) + .addDhTables(source) + .build()); + } catch (RuntimeException e) { + assertThat(e.getMessage()).contains("Table does not exist"); + } + final IcebergWriteInstructions writeInstructions = IcebergParquetWriteInstructions.builder() + .createTableIfNotExist(true) + .build(); + final List dataFilesWritten = catalogAdapter.writeDataFiles(IcebergWriteDataFiles.builder() + .tableIdentifier(tableIdentifier) + .addDhTables(source, anotherSource) + .instructions(writeInstructions) + .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"); + catalogAdapter.append(IcebergAppend.builder() + .tableIdentifier(tableIdentifier) + .addDhTables(moreData) + .instructions(writeInstructions) + .build()); { - final org.apache.iceberg.Table table = catalogAdapter.catalog().loadTable(tableIdentifier); - final List dataFileList = - IcebergUtils.allDataFiles(table, table.currentSnapshot()).collect(Collectors.toList()); - assertThat(dataFileList).hasSize(2); - String secondParquetFilePath = null; - for (final DataFile df : dataFileList) { - final String parquetFilePath = df.path().toString(); - if (!parquetFilePath.equals(firstParquetFilePath)) { - secondParquetFilePath = parquetFilePath; - break; - } - } - assertThat(secondParquetFilePath).isNotNull(); - final Table fromParquet = ParquetTools.readTable(secondParquetFilePath); - assertTableEquals(moreData, fromParquet); + final Table fromIceberg = catalogAdapter.readTable(tableIdentifier, null); + assertTableEquals(moreData, fromIceberg); + verifySnapshots(tableIdentifier, List.of("append")); + verifyDataFiles(TableIdentifier.parse(tableIdentifier), List.of(moreData)); } - // TODO Verify that the column ID is set correctly after #6156 is merged + // Now commit those data files to the table + final org.apache.iceberg.Table icebergTable = + catalogAdapter.catalog().loadTable(TableIdentifier.parse(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.parse(tableIdentifier), List.of(source, anotherSource, moreData)); } } From 78bd605fdc3a131ba0647c04af89144dd01ef685 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Fri, 18 Oct 2024 14:35:22 -0500 Subject: [PATCH 27/28] Added tests for on write callback --- .../table/ParquetTableReadWriteTest.java | 48 +++++++++++++++++++ 1 file changed, 48 insertions(+) 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..09edbfcf3bb 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,54 @@ public void testReadingParquetFilesWithDifferentPageSizes() { assertTableEquals(expected, fromDisk); } + private static class CompletedParquetWrite { + private final URI destination; + private final long numRows; + private final long numBytes; + + private CompletedParquetWrite(final URI destination, final long numRows, final long numBytes) { + this.destination = destination; + this.numRows = numRows; + this.numBytes = numBytes; + } + } + + @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 = + (destination, numRows, numBytes) -> parquetFilesWritten + .add(new CompletedParquetWrite(destination, numRows, numBytes)); + 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); From e2aba1f69ecafe210a865e5f12bbcb96411ee93c Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Tue, 22 Oct 2024 10:05:56 -0500 Subject: [PATCH 28/28] Added support for writing partitioned tables --- .../IcebergParquetWriteInstructionsTest.java | 10 +- .../deephaven/iceberg/base/IcebergUtils.java | 32 ++- .../deephaven/iceberg/util/IcebergAppend.java | 23 +++ .../iceberg/util/IcebergCatalogAdapter.java | 108 ++++++++-- .../iceberg/util/IcebergOverwrite.java | 23 +++ .../util/IcebergParquetWriteInstructions.java | 4 +- .../iceberg/util/IcebergWriteDataFiles.java | 23 +++ .../deephaven/iceberg/CatalogAdapterTest.java | 72 ++++++- py/server/deephaven/experimental/iceberg.py | 195 +++++++++++++++++- py/server/tests/test_iceberg.py | 81 ++++++++ 10 files changed, 545 insertions(+), 26 deletions(-) diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java index d24a2691ccf..9030188cc92 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructionsTest.java @@ -3,6 +3,8 @@ // 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; @@ -24,6 +26,7 @@ void defaults() { assertThat(instructions.compressionCodecName()).isEqualTo("SNAPPY"); assertThat(instructions.maximumDictionaryKeys()).isEqualTo(1048576); assertThat(instructions.maximumDictionarySize()).isEqualTo(1048576); + assertThat(instructions.targetPageSize()).isEqualTo(65536); } @Test @@ -160,9 +163,13 @@ void toParquetInstructionTest() { .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, fieldIdToName); + null, definition, fieldIdToName); assertThat(parquetInstructions.getCompressionCodecName()).isEqualTo("GZIP"); assertThat(parquetInstructions.getMaximumDictionaryKeys()).isEqualTo(100); @@ -172,5 +179,6 @@ void toParquetInstructionTest() { 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 index 2494dbf2c90..56a114d3e57 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/base/IcebergUtils.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/base/IcebergUtils.java @@ -8,8 +8,10 @@ import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.iceberg.util.IcebergWriteInstructions; import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; 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; @@ -161,7 +163,7 @@ public static class SpecAndSchema { private final PartitionSpec partitionSpec; private final Schema schema; - private SpecAndSchema(final PartitionSpec partitionSpec, final Schema schema) { + public SpecAndSchema(final PartitionSpec partitionSpec, final Schema schema) { this.partitionSpec = partitionSpec; this.schema = schema; } @@ -324,4 +326,32 @@ public static void verifyAppendCompatibility( partitionSpec + ", deephaven table definition: " + tableDefinition); } } + + /** + * Creates a list of {@link PartitionData} objects from a list of partition paths using the provided partition spec. + * Also, validates internally that the partition paths are compatible with the partition spec. + * + * @param partitionSpec The partition spec to use for validation + * @param partitionPaths The list of partition paths to create PartitionData objects from + * + * @return A list of PartitionData objects + */ + public static List partitionDataFromPaths( + final PartitionSpec partitionSpec, + final Collection partitionPaths) { + if (!partitionSpec.isPartitioned() && !partitionPaths.isEmpty()) { + throw new IllegalArgumentException("Partition paths should be empty for un-partitioned tables"); + } + final List partitionDataList = new ArrayList<>(partitionPaths.size()); + for (final String partitionPath : partitionPaths) { + // Following will internally validate the structure and values of the partition path + try { + partitionDataList.add(DataFiles.data(partitionSpec, partitionPath)); + } catch (final Throwable e) { + throw new IllegalArgumentException("Failed to parse partition path: " + partitionPath + " using" + + " partition spec " + partitionSpec, e); + } + } + return partitionDataList; + } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergAppend.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergAppend.java index 07e079f816f..fa6cbc35126 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergAppend.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergAppend.java @@ -23,6 +23,15 @@ public abstract class IcebergAppend { */ public abstract List
dhTables(); + /** + * 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". + *

+ * Users must provide partition path for each table in {@link #dhTables()} in the same order if appending a + * partitioned table. For appending non-partitioned tables, this should be an empty list. + */ + public abstract List partitionPaths(); + /** * The instructions for customizations while writing, defaults to {@link IcebergParquetWriteInstructions#DEFAULT}. */ @@ -44,8 +53,22 @@ public interface Builder { Builder addAllDhTables(Iterable elements); + // TODO Discuss about the API for partition paths, and add tests + Builder addPartitionPaths(String element); + + Builder addPartitionPaths(String... elements); + + Builder addAllPartitionPaths(Iterable elements); + Builder instructions(IcebergWriteInstructions instructions); IcebergAppend build(); } + + @Value.Check + final void countCheckPartitionPaths() { + if (!partitionPaths().isEmpty() && partitionPaths().size() != dhTables().size()) { + throw new IllegalArgumentException("Partition path must be provided for each table"); + } + } } 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 d790764603c..518440802b7 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 @@ -33,6 +33,7 @@ import org.apache.iceberg.DataFiles; import org.apache.iceberg.FileFormat; import org.apache.iceberg.OverwriteFiles; +import org.apache.iceberg.PartitionData; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -64,6 +65,7 @@ import static io.deephaven.iceberg.base.IcebergUtils.convertToDHType; import static io.deephaven.iceberg.base.IcebergUtils.allDataFiles; import static io.deephaven.iceberg.base.IcebergUtils.createSpecAndSchema; +import static io.deephaven.iceberg.base.IcebergUtils.partitionDataFromPaths; import static io.deephaven.iceberg.base.IcebergUtils.verifyAppendCompatibility; import static io.deephaven.iceberg.base.IcebergUtils.verifyOverwriteCompatibility; @@ -849,6 +851,7 @@ public Catalog catalog() { public void append(@NotNull final IcebergAppend icebergAppend) { writeImpl(TableIdentifier.parse(icebergAppend.tableIdentifier()), icebergAppend.dhTables(), + icebergAppend.partitionPaths(), icebergAppend.instructions(), false, true); @@ -865,6 +868,7 @@ public void append(@NotNull final IcebergAppend icebergAppend) { public void overwrite(@NotNull final IcebergOverwrite icebergOverwrite) { writeImpl(TableIdentifier.parse(icebergOverwrite.tableIdentifier()), icebergOverwrite.dhTables(), + icebergOverwrite.partitionPaths(), icebergOverwrite.instructions(), true, true); @@ -880,6 +884,7 @@ public void overwrite(@NotNull final IcebergOverwrite icebergOverwrite) { public List writeDataFiles(@NotNull final IcebergWriteDataFiles icebergWriteDataFiles) { return writeImpl(TableIdentifier.parse(icebergWriteDataFiles.tableIdentifier()), icebergWriteDataFiles.dhTables(), + icebergWriteDataFiles.partitionPaths(), icebergWriteDataFiles.instructions(), false, false); @@ -900,6 +905,7 @@ public List writeDataFiles(@NotNull final IcebergWriteDataFiles iceber private List writeImpl( @NotNull final TableIdentifier tableIdentifier, @NotNull List

dhTables, + @NotNull final List partitionPaths, @NotNull final IcebergWriteInstructions instructions, final boolean overwrite, final boolean addSnapshot) { @@ -941,11 +947,20 @@ private List writeImpl( // Try loading the table from the catalog, or create if required final org.apache.iceberg.Table icebergTable; final SpecAndSchema newSpecAndSchema; + final List partitionData; final boolean newNamespaceCreated; final boolean newTableCreated; if (catalog.tableExists(tableIdentifier)) { icebergTable = catalog.loadTable(tableIdentifier); - newSpecAndSchema = createSpecAndSchema(useDefinition, writeInstructions); + if (overwrite) { + verifyPartitionPathsIfOverwritingOrCreating(partitionPaths, writeInstructions); + newSpecAndSchema = createSpecAndSchema(useDefinition, writeInstructions); + } else { + verifyPartitionPathsIfAppending(icebergTable, partitionPaths); + // Same spec and schema as the existing table + newSpecAndSchema = new SpecAndSchema(icebergTable.spec(), icebergTable.schema()); + } + partitionData = partitionDataFromPaths(newSpecAndSchema.partitionSpec(), partitionPaths); newNamespaceCreated = false; newTableCreated = false; if (verifySchema) { @@ -964,8 +979,10 @@ private List writeImpl( } } } else if (writeInstructions.createTableIfNotExist()) { + verifyPartitionPathsIfOverwritingOrCreating(partitionPaths, writeInstructions); newNamespaceCreated = createNamespaceIfNotExists(tableIdentifier.namespace()); newSpecAndSchema = createSpecAndSchema(useDefinition, writeInstructions); + partitionData = partitionDataFromPaths(newSpecAndSchema.partitionSpec(), partitionPaths); icebergTable = createNewIcebergTable(tableIdentifier, newSpecAndSchema, writeInstructions); newTableCreated = true; } else { @@ -973,11 +990,11 @@ private List writeImpl( "Table does not exist: " + tableIdentifier + ", update the instructions " + "to create the table if it does not exist and try again."); } - + final PartitionSpec newPartitionSpec = newSpecAndSchema.partitionSpec(); try { final List parquetFileInfo = - writeParquet(icebergTable, dhTables, writeInstructions); - final List appendFiles = dataFilesFromParquet(parquetFileInfo); + writeParquet(icebergTable, newPartitionSpec, dhTables, partitionPaths, writeInstructions); + final List appendFiles = dataFilesFromParquet(parquetFileInfo, partitionData, newPartitionSpec); if (addSnapshot) { commit(icebergTable, newSpecAndSchema, appendFiles, overwrite, verifySchema); } @@ -1003,6 +1020,32 @@ private List writeImpl( } } + private static void verifyPartitionPathsIfOverwritingOrCreating( + @NotNull final Collection partitionPaths, + @NotNull final IcebergBaseInstructions instructions) { + if (!partitionPaths.isEmpty()) { + final TableDefinition tableDefinition = instructions.tableDefinition() + .orElseThrow(() -> new IllegalArgumentException("Table definition expected")); + if (tableDefinition.getColumnStream().noneMatch(ColumnDefinition::isPartitioning)) { + throw new IllegalArgumentException("Cannot write un-partitioned table to partition paths. Please " + + "remove partition paths from the Iceberg instructions or provide a table definition with " + + "partitioning columns."); + } + } + } + + private static void verifyPartitionPathsIfAppending( + final org.apache.iceberg.Table icebergTable, + final Collection partitionPaths) { + if (icebergTable.spec().isPartitioned() && partitionPaths.isEmpty()) { + throw new IllegalArgumentException("Cannot write data to an existing partitioned table without " + + "providing partition paths. Please provide partition paths in the Iceberg instructions."); + } else if (!icebergTable.spec().isPartitioned() && !partitionPaths.isEmpty()) { + throw new IllegalArgumentException("Cannot write data to an existing un-partitioned table with " + + "partition paths. Please remove partition paths from the Iceberg instructions."); + } + } + private static IcebergParquetWriteInstructions verifyWriteInstructions( @NotNull final IcebergWriteInstructions instructions) { // We ony support writing to Parquet files @@ -1061,28 +1104,47 @@ private CompletedParquetWrite(final URI destination, final long numRows, final l @NotNull private static List writeParquet( @NotNull final org.apache.iceberg.Table icebergTable, - @NotNull final Collection
dhTables, + @NotNull final PartitionSpec partitionSpec, + @NotNull final List
dhTables, + @NotNull final List partitionPaths, @NotNull final IcebergParquetWriteInstructions writeInstructions) { // Build the parquet instructions final List parquetFilesWritten = new ArrayList<>(dhTables.size()); final ParquetInstructions.OnWriteCompleted onWriteCompleted = (destination, numRows, numBytes) -> parquetFilesWritten .add(new CompletedParquetWrite(destination, numRows, numBytes)); + + // Remove all partitioning columns from the table definition before writing parquet files + TableDefinition parquetTableDefinition = writeInstructions.tableDefinition() + .orElseThrow(() -> new IllegalArgumentException("Table definition expected")); + if (partitionSpec.isPartitioned()) { + parquetTableDefinition = TableDefinition.of( + parquetTableDefinition.getColumnStream() + .filter(columnDefinition -> !columnDefinition.isPartitioning()) + .collect(Collectors.toList())); + } final ParquetInstructions parquetInstructions = writeInstructions.toParquetInstructions( - onWriteCompleted, icebergTable.schema().idToName()); + onWriteCompleted, parquetTableDefinition, icebergTable.schema().idToName()); // Write the data to parquet files int count = 0; - for (final Table dhTable : dhTables) { + for (int idx = 0; idx < dhTables.size(); idx++) { + final Table dhTable = dhTables.get(idx); if (dhTable.numColumns() == 0) { // Skip writing empty tables with no columns continue; } - final String filename = String.format( + final String fileName = String.format( "00000-%d-%s.parquet", count++, UUID.randomUUID()); - final String newDataLocation = icebergTable.locationProvider().newDataLocation(filename); + final String relativePath; + if (partitionSpec.isPartitioned()) { + relativePath = String.format("%s/%s", partitionPaths.get(idx), fileName); + } else { + relativePath = fileName; + } + final String newDataLocation = icebergTable.locationProvider().newDataLocation(relativePath); ParquetTools.writeTable(dhTable, newDataLocation, parquetInstructions); } return parquetFilesWritten; @@ -1149,15 +1211,23 @@ private static void commit( * Generate a list of {@link DataFile} objects from a list of parquet files written. */ private static List dataFilesFromParquet( - @NotNull final Collection parquetFilesWritten) { - // TODO This assumes no partition data is written, is that okay? - return parquetFilesWritten.stream() - .map(parquetFileWritten -> DataFiles.builder(PartitionSpec.unpartitioned()) - .withPath(parquetFileWritten.destination.toString()) - .withFormat(FileFormat.PARQUET) - .withRecordCount(parquetFileWritten.numRows) - .withFileSizeInBytes(parquetFileWritten.numBytes) - .build()) - .collect(Collectors.toList()); + @NotNull final List parquetFilesWritten, + @NotNull final List partitionDataList, + @NotNull final PartitionSpec partitionSpec) { + final int numFiles = parquetFilesWritten.size(); + final List dataFiles = new ArrayList<>(numFiles); + 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/IcebergOverwrite.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergOverwrite.java index 4af3224dfe3..cbbd6ef15ef 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergOverwrite.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergOverwrite.java @@ -23,6 +23,15 @@ public abstract class IcebergOverwrite { */ public abstract List
dhTables(); + /** + * 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". + *

+ * Users must provide partition path for each table in {@link #dhTables()} in the same order if overwriting with a + * partitioned table. For overwriting with non-partitioned tables, this should be an empty list. + */ + public abstract List partitionPaths(); + /** * The instructions for customizations while writing, defaults to {@link IcebergParquetWriteInstructions#DEFAULT}. */ @@ -44,8 +53,22 @@ public interface Builder { Builder addAllDhTables(Iterable elements); + // TODO think about the API for partition paths + Builder addPartitionPaths(String element); + + Builder addPartitionPaths(String... elements); + + Builder addAllPartitionPaths(Iterable elements); + Builder instructions(IcebergWriteInstructions instructions); IcebergOverwrite build(); } + + @Value.Check + final void countCheckPartitionPaths() { + if (!partitionPaths().isEmpty() && partitionPaths().size() != dhTables().size()) { + throw new IllegalArgumentException("Partition path must be provided for each table"); + } + } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java index c1277105043..89cf196684d 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergParquetWriteInstructions.java @@ -78,17 +78,19 @@ public int targetPageSize() { * Convert this {@link IcebergParquetWriteInstructions} 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(); - tableDefinition().ifPresent(builder::setTableDefinition); dataInstructions().ifPresent(builder::setSpecialInstructions); // Add parquet writing specific instructions. + builder.setTableDefinition(tableDefinition); builder.addFieldIdMapping(fieldIdToName); builder.setCompressionCodecName(compressionCodecName()); builder.setMaximumDictionaryKeys(maximumDictionaryKeys()); diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteDataFiles.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteDataFiles.java index 494c8f2073f..978e92ef91e 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteDataFiles.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergWriteDataFiles.java @@ -23,6 +23,15 @@ public abstract class IcebergWriteDataFiles { */ public abstract List

dhTables(); + /** + * 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". + *

+ * Users must provide partition path for each table in {@link #dhTables()} in the same order if writing to a + * partitioned table. For writing to a non-partitioned tables, this should be an empty list. + */ + public abstract List partitionPaths(); + /** * The instructions for customizations while writing, defaults to {@link IcebergParquetWriteInstructions#DEFAULT}. */ @@ -44,8 +53,22 @@ public interface Builder { Builder addAllDhTables(Iterable elements); + // TODO think about the API for partition paths + Builder addPartitionPaths(String element); + + Builder addPartitionPaths(String... elements); + + Builder addAllPartitionPaths(Iterable elements); + Builder instructions(IcebergWriteInstructions instructions); IcebergWriteDataFiles build(); } + + @Value.Check + final void countCheckPartitionPaths() { + if (!partitionPaths().isEmpty() && partitionPaths().size() != dhTables().size()) { + throw new IllegalArgumentException("Partition path must be provided for each table"); + } + } } diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java index bc0e7da12ce..f756082fd62 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java @@ -7,6 +7,7 @@ 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.util.TableTools; import io.deephaven.iceberg.base.IcebergUtils; @@ -24,7 +25,6 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import java.util.Iterator; @@ -33,6 +33,7 @@ import static io.deephaven.engine.testutil.TstUtils.assertTableEquals; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.fail; class CatalogAdapterTest extends CatalogAdapterBase { @Test @@ -479,7 +480,7 @@ void testFailureInWrite() { .addDhTables(badSource) .instructions(writeInstructions) .build()); - Assertions.fail("Exception expected for invalid formula in table"); + fail("Exception expected for invalid formula in table"); } catch (UncheckedDeephavenException e) { assertThat(e.getCause() instanceof FormulaEvaluationException).isTrue(); } @@ -503,7 +504,7 @@ void testFailureInWrite() { .addDhTables(badSource) .instructions(writeInstructions) .build()); - Assertions.fail("Exception expected for invalid formula in table"); + fail("Exception expected for invalid formula in table"); } catch (UncheckedDeephavenException e) { assertThat(e.getCause() instanceof FormulaEvaluationException).isTrue(); } @@ -596,6 +597,7 @@ void writeDataFilesBasicTest() { .tableIdentifier(tableIdentifier) .addDhTables(source) .build()); + fail("Exception expected for table not existing"); } catch (RuntimeException e) { assertThat(e.getMessage()).contains("Table does not exist"); } @@ -637,4 +639,68 @@ void writeDataFilesBasicTest() { verifySnapshots(tableIdentifier, List.of("append", "append")); verifyDataFiles(TableIdentifier.parse(tableIdentifier), List.of(source, anotherSource, moreData)); } + + @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=apple", "PC=boy"); + final String tableIdentifier = "MyNamespace.MyTable"; + + try { + final IcebergWriteInstructions writeInstructions = IcebergParquetWriteInstructions.builder() + .createTableIfNotExist(true) + .build(); + catalogAdapter.append(IcebergAppend.builder() + .tableIdentifier(tableIdentifier) + .addDhTables(part1, part2) + .addAllPartitionPaths(partitionPaths) + .instructions(writeInstructions) + .build()); + fail("Exception expected since no partitioning table definition is provided"); + } catch (RuntimeException e) { + assertThat(e.getMessage()).contains("table definition"); + } + + final TableDefinition tableDefinition = TableDefinition.of( + ColumnDefinition.ofInt("intCol"), + ColumnDefinition.ofDouble("doubleCol"), + ColumnDefinition.ofString("PC").withPartitioning()); + final IcebergWriteInstructions writeInstructions = IcebergParquetWriteInstructions.builder() + .createTableIfNotExist(true) + .tableDefinition(tableDefinition) + .build(); + catalogAdapter.append(IcebergAppend.builder() + .tableIdentifier(tableIdentifier) + .addDhTables(part1, part2) + .addAllPartitionPaths(partitionPaths) + .instructions(writeInstructions) + .build()); + final Table fromIceberg = catalogAdapter.readTable(tableIdentifier, null); + assertThat(catalogAdapter.getTableDefinition(tableIdentifier, null)).isEqualTo(tableDefinition); + assertThat(fromIceberg.getDefinition()).isEqualTo(tableDefinition); + assertThat(fromIceberg).isInstanceOf(PartitionAwareSourceTable.class); + final Table expected = TableTools.merge( + part1.update("PC = `apple`"), + part2.update("PC = `boy`")); + 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=cat"; + catalogAdapter.append(IcebergAppend.builder() + .tableIdentifier(tableIdentifier) + .addDhTables(part3) + .addPartitionPaths(partitionPath) + .instructions(writeInstructions) + .build()); + final Table fromIceberg2 = catalogAdapter.readTable(tableIdentifier, null); + final Table expected2 = TableTools.merge(part3.update("PC = `cat`"), expected); + assertTableEquals(expected2, fromIceberg2.select()); + } } diff --git a/py/server/deephaven/experimental/iceberg.py b/py/server/deephaven/experimental/iceberg.py index 9b488c6d51b..a8807ff5d5e 100644 --- a/py/server/deephaven/experimental/iceberg.py +++ b/py/server/deephaven/experimental/iceberg.py @@ -14,8 +14,12 @@ from deephaven.jcompat import j_hashmap _JIcebergReadInstructions = jpy.get_type("io.deephaven.iceberg.util.IcebergReadInstructions") +_JIcebergParquetWriteInstructions = jpy.get_type("io.deephaven.iceberg.util.IcebergParquetWriteInstructions") _JIcebergCatalogAdapter = jpy.get_type("io.deephaven.iceberg.util.IcebergCatalogAdapter") _JIcebergTools = jpy.get_type("io.deephaven.iceberg.util.IcebergTools") +_JIcebergAppend = jpy.get_type("io.deephaven.iceberg.util.IcebergAppend") +_JIcebergOverwrite = jpy.get_type("io.deephaven.iceberg.util.IcebergOverwrite") +_JIcebergWriteDataFile = jpy.get_type("io.deephaven.iceberg.util.IcebergWriteDataFiles") # IcebergToolsS3 is an optional library try: @@ -78,6 +82,102 @@ def j_object(self) -> jpy.JType: return self._j_object +class IcebergParquetWriteInstructions(JObjectWrapper): + """ + This class specifies the instructions for writing Iceberg tables as Parquet data files. These include column rename + instructions, table definitions, special data instructions for loading data files from the cloud, etc. + """ + + j_object_type = _JIcebergParquetWriteInstructions + + def __init__(self, + compression_codec_name: Optional[str] = None, + maximum_dictionary_keys: Optional[int] = None, + maximum_dictionary_size: Optional[int] = None, + target_page_size: Optional[int] = None, + create_table_if_not_exist: Optional[bool] = None, + verify_schema: Optional[bool] = None, + dh_to_iceberg_column_renames: Optional[Dict[str, str]] = None, + table_definition: Optional[TableDefinitionLike] = None, + data_instructions: Optional[s3.S3Instructions] = None): + """ + Initializes the instructions using the provided parameters. + + Args: + compression_codec_name (Optional[str]): the compression codec to use. Allowed values include "UNCOMPRESSED", + "SNAPPY", "GZIP", "LZO", "LZ4", "LZ4_RAW", "ZSTD", etc. If not specified, defaults to "SNAPPY". + maximum_dictionary_keys (Optional[int]): the maximum number of unique keys the writer should add to a + dictionary page before switching to non-dictionary encoding, never evaluated for non-String columns, + defaults to 2^20 (1,048,576) + maximum_dictionary_size (Optional[int]): the maximum number of bytes the writer should add to the dictionary + before switching to non-dictionary encoding, never evaluated for non-String columns, defaults to + 2^20 (1,048,576) + target_page_size (Optional[int]): the target page size in bytes, if not specified, defaults to + 2^20 bytes (1 MiB) + create_table_if_not_exist (Optional[bool]): if true, the table will be created if it does not exist, + defaults to false + verify_schema (Optional[bool]): Specifies whether to verify that the partition spec and schema of the table + being written are consistent with the Iceberg table. Verification behavior differs based on the + operation type: + - Appending Data or Writing Data Files: Verification is enabled by default. It ensures that: + - All columns from the Deephaven table are present in the Iceberg table and have compatible types. + - All required columns in the Iceberg table are present in the Deephaven table. + - The set of partitioning columns in both the Iceberg and Deephaven tables are identical. + - Overwriting Data: Verification is disabled by default. When enabled, it ensures that the + schema and partition spec of the table being written are identical to those of the Iceberg table. + dh_to_iceberg_column_renames (Optional[Dict[str, str]]): A dictionary from Deephaven to Iceberg column names + to use when writing deephaven tables to Iceberg tables. + table_definition (Optional[TableDefinitionLike]): the table definition; if omitted, + the definition is inferred from the Iceberg schema. Setting a definition guarantees the returned table + will have that definition. This is useful for specifying a subset of the Iceberg schema columns. + data_instructions (Optional[s3.S3Instructions]): Special instructions for reading data files, useful when + reading files from a non-local file system, like S3. + + Raises: + DHError: If unable to build the instructions object. + """ + + try: + builder = self.j_object_type.builder() + + if compression_codec_name is not None: + builder.compressionCodecName(compression_codec_name) + + if maximum_dictionary_keys is not None: + builder.maximumDictionaryKeys(maximum_dictionary_keys) + + if maximum_dictionary_size is not None: + builder.maximumDictionarySize(maximum_dictionary_size) + + if target_page_size is not None: + builder.targetPageSize(target_page_size) + + if create_table_if_not_exist is not None: + builder.createTableIfNotExist(create_table_if_not_exist) + + if verify_schema is not None: + builder.verifySchema(verify_schema) + + if dh_to_iceberg_column_renames is not None: + for dh_name, iceberg_name in dh_to_iceberg_column_renames.items(): + builder.putDhToIcebergColumnRenames(dh_name, iceberg_name) + + if table_definition is not None: + builder.tableDefinition(TableDefinition(table_definition).j_table_definition) + + if data_instructions is not None: + 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 IcebergCatalogAdapter(JObjectWrapper): """ This class provides an interface for interacting with Iceberg catalogs. It allows listing namespaces, tables and @@ -133,7 +233,11 @@ def snapshots(self, table_identifier: str) -> Table: return self.j_object.listSnapshotsAsTable(table_identifier) - def read_table(self, table_identifier: str, instructions: Optional[IcebergReadInstructions] = None, snapshot_id: Optional[int] = None) -> Table: + def read_table( + self, + table_identifier: str, + instructions: Optional[IcebergReadInstructions] = None, + snapshot_id: Optional[int] = None) -> Table: """ Reads the table from the catalog using the provided instructions. Optionally, a snapshot id can be provided to read a specific snapshot of the table. @@ -158,6 +262,95 @@ def read_table(self, table_identifier: str, instructions: Optional[IcebergReadIn return Table(self.j_object.readTable(table_identifier, snapshot_id, instructions_object)) return Table(self.j_object.readTable(table_identifier, instructions_object)) + def append(self, + table_identifier: str, + tables: List[Table], + partition_paths: Optional[List[str]] = None, + instructions: Optional[IcebergParquetWriteInstructions] = None): + # TODO Review javadoc in this file once again + """ + Append the provided Deephaven table as a new partition to the existing Iceberg table in a single snapshot. This + will not change the schema of the existing table. + + Args: + table_identifier (str): the identifier string for iceberg table to append to. + tables (List[Table]): the tables to append. + partition_paths (Optional[List[str]]): the partitioning path at which data would be appended, for example, + "year=2021/month=01". If omitted, we will try to append data to the table without partitioning. + instructions (Optional[IcebergParquetWriteInstructions]): the instructions for customizations while writing. + """ + builder = _JIcebergAppend.builder().tableIdentifier(table_identifier) + + for table in tables: + builder.addDhTables(table.j_table) + + for partition_path in partition_paths: + builder.addPartitionPaths(partition_path) + + if instructions is not None: + builder.instructions(instructions.j_object) + + return self.j_object.append(builder.build()) + + def overwrite(self, + table_identifier: str, + tables: List[Table], + partition_paths: Optional[List[str]] = None, + instructions: Optional[IcebergParquetWriteInstructions] = None): + """ + Overwrite the existing Iceberg table with the provided Deephaven tables in a single snapshot. This will + overwrite the schema of the existing table to match the provided Deephaven table if they do not match. + Overwriting a table while racing with other writers can lead to failure/undefined results. + + Args: + table_identifier (str): the identifier string for iceberg table to overwrite. + tables (List[Table]): the tables to overwrite. + partition_paths (Optional[List[str]]): the partitioning path at which data would be overwritten, for example, + "year=2021/month=01". If omitted, we will try to overwrite data to the table without partitioning. + instructions (Optional[IcebergParquetWriteInstructions]): the instructions for customizations while writing. + """ + builder = _JIcebergOverwrite.builder().tableIdentifier(table_identifier) + + for table in tables: + builder.addDhTables(table.j_table) + + for partition_path in partition_paths: + builder.addPartitionPaths(partition_path) + + if instructions is not None: + builder.instructions(instructions.j_object) + + return self.j_object.overwrite(builder.build()) + + def write_data_file(self, + table_identifier: str, + tables: List[Table], + partition_paths: Optional[List[str]] = None, + instructions: Optional[IcebergParquetWriteInstructions] = None): + """ + Writes data from Deephaven 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. + + Args: + table_identifier (str): the identifier string for iceberg table to write to. + tables (List[Table]): the tables to write. + partition_paths (Optional[List[str]]): the partitioning path at which data would be written, for example, + "year=2021/month=01". If omitted, we will try to write data to the table without partitioning. + instructions (Optional[IcebergParquetWriteInstructions]): the instructions for customizations while writing. + """ + builder = _JIcebergWriteDataFile.builder().tableIdentifier(table_identifier) + + for table in tables: + builder.addDhTables(table.j_table) + + for partition_path in partition_paths: + builder.addPartitionPaths(partition_path) + + if instructions is not None: + builder.instructions(instructions.j_object) + + return self.j_object.writeDataFiles(builder.build()) + @property def j_object(self) -> jpy.JType: return self.j_catalog_adapter diff --git a/py/server/tests/test_iceberg.py b/py/server/tests/test_iceberg.py index 13331dffa5b..5a521fd3bfd 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) """ @@ -72,3 +73,83 @@ def test_instruction_create_with_table_definition_list(self): self.assertTrue(col_names[1] == "x") self.assertTrue(col_names[2] == "y") self.assertTrue(col_names[3] == "z") + + def test_write_instruction_create_default(self): + iceberg_write_instructions = iceberg.IcebergParquetWriteInstructions() + self.assertEqual(iceberg_write_instructions.j_object.compressionCodecName(), "SNAPPY") + self.assertEqual(iceberg_write_instructions.j_object.maximumDictionaryKeys(), 1048576) + self.assertEqual(iceberg_write_instructions.j_object.maximumDictionarySize(), 1048576) + self.assertEqual(iceberg_write_instructions.j_object.targetPageSize(), 65536) + self.assertFalse(iceberg_write_instructions.j_object.createTableIfNotExist()) + + def test_write_instruction_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" + ) + iceberg_write_instructions = iceberg.IcebergParquetWriteInstructions(data_instructions=s3_instructions) + + def test_write_instruction_create_with_col_renames(self): + renames = { + "dh_name_a": "ice_name_a", + "dh_name_b": "ice_name_b", + "dh_name_c": "ice_name_c" + } + iceberg_write_instructions = iceberg.IcebergParquetWriteInstructions(dh_to_iceberg_column_renames=renames) + + col_rename_dict = j_map_to_dict(iceberg_write_instructions.j_object.dhToIcebergColumnRenames()) + self.assertTrue(col_rename_dict["dh_name_a"] == "ice_name_a") + self.assertTrue(col_rename_dict["dh_name_b"] == "ice_name_b") + self.assertTrue(col_rename_dict["dh_name_c"] == "ice_name_c") + + def test_write_instruction_create_with_table_definition_dict(self): + table_def = { + "x": dtypes.int32, + "y": dtypes.double, + "z": dtypes.double, + } + + iceberg_write_instructions = iceberg.IcebergParquetWriteInstructions(table_definition=table_def) + col_names = j_list_to_list(iceberg_write_instructions.j_object.tableDefinition().get().getColumnNames()) + self.assertTrue(col_names[0] == "x") + self.assertTrue(col_names[1] == "y") + self.assertTrue(col_names[2] == "z") + + def test_write_instruction_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), + ] + + iceberg_write_instructions = iceberg.IcebergParquetWriteInstructions(table_definition=table_def) + col_names = j_list_to_list(iceberg_write_instructions.j_object.tableDefinition().get().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_write_instruction_create_with_compression_codec(self): + iceberg_write_instructions = iceberg.IcebergParquetWriteInstructions(compression_codec_name="GZIP") + self.assertEqual(iceberg_write_instructions.j_object.compressionCodecName(), "GZIP") + + def test_write_instruction_create_with_max_dictionary_keys(self): + iceberg_write_instructions = iceberg.IcebergParquetWriteInstructions(maximum_dictionary_keys=1024) + self.assertEqual(iceberg_write_instructions.j_object.maximumDictionaryKeys(), 1024) + + def test_write_instruction_create_with_max_dictionary_size(self): + iceberg_write_instructions = iceberg.IcebergParquetWriteInstructions(maximum_dictionary_size=8192) + self.assertEqual(iceberg_write_instructions.j_object.maximumDictionarySize(), 8192) + + def test_write_instruction_create_with_target_page_size(self): + iceberg_write_instructions = iceberg.IcebergParquetWriteInstructions(target_page_size=4096) + self.assertEqual(iceberg_write_instructions.j_object.targetPageSize(), 4096) + + def test_write_instruction_create_with_create_table_if_not_exist(self): + iceberg_write_instructions = iceberg.IcebergParquetWriteInstructions(create_table_if_not_exist=True) + self.assertTrue(iceberg_write_instructions.j_object.createTableIfNotExist()) + + def test_write_instruction_create_with_verify_schema(self): + iceberg_write_instructions = iceberg.IcebergParquetWriteInstructions(verify_schema=True) + self.assertTrue(iceberg_write_instructions.j_object.verifySchema())